You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@cassandra.apache.org by sl...@apache.org on 2016/05/06 11:47:39 UTC

[1/9] cassandra git commit: Refactor MV code

Repository: cassandra
Updated Branches:
  refs/heads/cassandra-3.0 c19066ee0 -> 86ba22747
  refs/heads/cassandra-3.7 4e364d71e -> a87fd715d
  refs/heads/trunk 1dd33eca1 -> 89a645ac4


http://git-wip-us.apache.org/repos/asf/cassandra/blob/86ba2274/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 1b823aa..845a6ab 100644
--- a/src/java/org/apache/cassandra/db/view/View.java
+++ b/src/java/org/apache/cassandra/db/view/View.java
@@ -32,17 +32,15 @@ import org.apache.cassandra.cql3.statements.SelectStatement;
 import org.apache.cassandra.db.*;
 import org.apache.cassandra.config.*;
 import org.apache.cassandra.cql3.ColumnIdentifier;
-import org.apache.cassandra.db.AbstractReadCommandBuilder.SinglePartitionSliceBuilder;
 import org.apache.cassandra.db.compaction.CompactionManager;
-import org.apache.cassandra.db.partitions.AbstractBTreePartition;
-import org.apache.cassandra.db.partitions.PartitionIterator;
-import org.apache.cassandra.db.partitions.PartitionUpdate;
+import org.apache.cassandra.db.partitions.*;
 import org.apache.cassandra.db.rows.*;
 import org.apache.cassandra.schema.KeyspaceMetadata;
 import org.apache.cassandra.service.ClientState;
 import org.apache.cassandra.service.pager.QueryPager;
 import org.apache.cassandra.transport.Server;
 import org.apache.cassandra.utils.FBUtilities;
+import org.apache.cassandra.utils.btree.BTreeSet;
 import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
 
@@ -50,46 +48,18 @@ import org.slf4j.LoggerFactory;
  * A View copies data from a base table into a view table which can be queried independently from the
  * base. Every update which targets the base table must be fed through the {@link ViewManager} to ensure
  * that if a view needs to be updated, the updates are properly created and fed into the view.
- *
- * This class does the job of translating the base row to the view row.
- *
- * It handles reading existing state and figuring out what tombstones need to be generated.
- *
- * {@link View#createMutations(AbstractBTreePartition, TemporalRow.Set, boolean)} is the "main method"
- *
  */
 public class View
 {
     private static final Logger logger = LoggerFactory.getLogger(View.class);
 
-    /**
-     * The columns should all be updated together, so we use this object as group.
-     */
-    private static class Columns
-    {
-        //These are the base column definitions in terms of the *views* partitioning.
-        //Meaning we can see (for example) the partition key of the view contains a clustering key
-        //from the base table.
-        public final List<ColumnDefinition> partitionDefs;
-        public final List<ColumnDefinition> primaryKeyDefs;
-        public final List<ColumnDefinition> baseComplexColumns;
-
-        private Columns(List<ColumnDefinition> partitionDefs, List<ColumnDefinition> primaryKeyDefs, List<ColumnDefinition> baseComplexColumns)
-        {
-            this.partitionDefs = partitionDefs;
-            this.primaryKeyDefs = primaryKeyDefs;
-            this.baseComplexColumns = baseComplexColumns;
-        }
-    }
-
     public final String name;
     private volatile ViewDefinition definition;
 
     private final ColumnFamilyStore baseCfs;
 
-    private Columns columns;
+    public volatile List<ColumnDefinition> baseNonPKColumnsInViewPK;
 
-    private final boolean viewPKIncludesOnlyBasePKColumns;
     private final boolean includeAllColumns;
     private ViewBuilder builder;
 
@@ -104,12 +74,11 @@ public class View
                 ColumnFamilyStore baseCfs)
     {
         this.baseCfs = baseCfs;
-
-        name = definition.viewName;
-        includeAllColumns = definition.includeAllColumns;
-
-        viewPKIncludesOnlyBasePKColumns = updateDefinition(definition);
+        this.name = definition.viewName;
+        this.includeAllColumns = definition.includeAllColumns;
         this.rawSelect = definition.select;
+
+        updateDefinition(definition);
     }
 
     public ViewDefinition getDefinition()
@@ -118,513 +87,100 @@ public class View
     }
 
     /**
-     * Lookup column definitions in the base table that correspond to the view columns (should be 1:1)
-     *
-     * Notify caller if all primary keys in the view are ALL primary keys in the base. We do this to simplify
-     * tombstone checks.
-     *
-     * @param columns a list of columns to lookup in the base table
-     * @param definitions lists to populate for the base table definitions
-     * @return true if all view PKs are also Base PKs
-     */
-    private boolean resolveAndAddColumns(Iterable<ColumnIdentifier> columns, List<ColumnDefinition>... definitions)
-    {
-        boolean allArePrimaryKeys = true;
-        for (ColumnIdentifier identifier : columns)
-        {
-            ColumnDefinition cdef = baseCfs.metadata.getColumnDefinition(identifier);
-            assert cdef != null : "Could not resolve column " + identifier.toString();
-
-            for (List<ColumnDefinition> list : definitions)
-            {
-                list.add(cdef);
-            }
-
-            allArePrimaryKeys = allArePrimaryKeys && cdef.isPrimaryKeyColumn();
-        }
-
-        return allArePrimaryKeys;
-    }
-
-    /**
      * This updates the columns stored which are dependent on the base CFMetaData.
      *
      * @return true if the view contains only columns which are part of the base's primary key; false if there is at
      *         least one column which is not.
      */
-    public boolean updateDefinition(ViewDefinition definition)
+    public void updateDefinition(ViewDefinition definition)
     {
         this.definition = definition;
 
         CFMetaData viewCfm = definition.metadata;
-        List<ColumnDefinition> partitionDefs = new ArrayList<>(viewCfm.partitionKeyColumns().size());
-        List<ColumnDefinition> primaryKeyDefs = new ArrayList<>(viewCfm.partitionKeyColumns().size()
-                                                                + viewCfm.clusteringColumns().size());
-        List<ColumnDefinition> baseComplexColumns = new ArrayList<>();
-
-        // We only add the partition columns to the partitions list, but both partition columns and clustering
-        // columns are added to the primary keys list
-        boolean partitionAllPrimaryKeyColumns = resolveAndAddColumns(Iterables.transform(viewCfm.partitionKeyColumns(), cd -> cd.name), primaryKeyDefs, partitionDefs);
-        boolean clusteringAllPrimaryKeyColumns = resolveAndAddColumns(Iterables.transform(viewCfm.clusteringColumns(), cd -> cd.name), primaryKeyDefs);
-
-        for (ColumnDefinition cdef : baseCfs.metadata.allColumns())
+        List<ColumnDefinition> nonPKDefPartOfViewPK = new ArrayList<>();
+        for (ColumnDefinition baseColumn : baseCfs.metadata.allColumns())
         {
-            if (cdef.isComplex() && definition.includes(cdef.name))
-            {
-                baseComplexColumns.add(cdef);
-            }
+            ColumnDefinition viewColumn = getViewColumn(baseColumn);
+            if (viewColumn != null && !baseColumn.isPrimaryKeyColumn() && viewColumn.isPrimaryKeyColumn())
+                nonPKDefPartOfViewPK.add(baseColumn);
         }
-
-        this.columns = new Columns(partitionDefs, primaryKeyDefs, baseComplexColumns);
-
-        return partitionAllPrimaryKeyColumns && clusteringAllPrimaryKeyColumns;
+        this.baseNonPKColumnsInViewPK = nonPKDefPartOfViewPK;
     }
 
     /**
-     * Check to see if the update could possibly modify a view. Cases where the view may be updated are:
-     * <ul>
-     *     <li>View selects all columns</li>
-     *     <li>Update contains any range tombstones</li>
-     *     <li>Update touches one of the columns included in the view</li>
-     * </ul>
-     *
-     * If the update contains any range tombstones, there is a possibility that it will not touch a range that is
-     * currently included in the view.
-     *
-     * @return true if {@param partition} modifies a column included in the view
+     * The view column corresponding to the provided base column. This <b>can</b>
+     * return {@code null} if the column is denormalized in the view.
      */
-    public boolean updateAffectsView(AbstractBTreePartition partition)
+    public ColumnDefinition getViewColumn(ColumnDefinition baseColumn)
     {
-        ReadQuery selectQuery = getReadQuery();
-
-        if (!partition.metadata().cfId.equals(definition.baseTableId))
-            return false;
-
-        if (!selectQuery.selectsKey(partition.partitionKey()))
-            return false;
-
-        // If there are range tombstones, tombstones will also need to be generated for the view
-        // This requires a query of the base rows and generating tombstones for all of those values
-        if (!partition.deletionInfo().isLive())
-            return true;
-
-        // Check each row for deletion or update
-        for (Row row : partition)
-        {
-            if (!selectQuery.selectsClustering(partition.partitionKey(), row.clustering()))
-                continue;
-
-            if (includeAllColumns || !row.deletion().isLive())
-                return true;
-
-            if (row.primaryKeyLivenessInfo().isLive(FBUtilities.nowInSeconds()))
-                return true;
-
-            for (ColumnData data : row)
-            {
-                if (definition.metadata.getColumnDefinition(data.column().name) != null)
-                    return true;
-            }
-        }
-
-        return false;
-    }
-
-    /**
-     * Creates the clustering columns for the view based on the specified row and resolver policy
-     *
-     * @param temporalRow The current row
-     * @param resolver The policy to use when selecting versions of cells use
-     * @return The clustering object to use for the view
-     */
-    private Clustering viewClustering(TemporalRow temporalRow, TemporalRow.Resolver resolver)
-    {
-        CFMetaData viewCfm = definition.metadata;
-        int numViewClustering = viewCfm.clusteringColumns().size();
-        CBuilder clustering = CBuilder.create(viewCfm.comparator);
-        for (int i = 0; i < numViewClustering; i++)
-        {
-            ColumnDefinition definition = viewCfm.clusteringColumns().get(i);
-            clustering.add(temporalRow.clusteringValue(definition, resolver));
-        }
-
-        return clustering.build();
-    }
-
-    /**
-     * @return Mutation containing a range tombstone for a base partition key and TemporalRow.
-     */
-    private PartitionUpdate createTombstone(TemporalRow temporalRow,
-                                            DecoratedKey partitionKey,
-                                            Row.Deletion deletion,
-                                            TemporalRow.Resolver resolver,
-                                            int nowInSec)
-    {
-        CFMetaData viewCfm = definition.metadata;
-        Row.Builder builder = BTreeRow.unsortedBuilder(nowInSec);
-        builder.newRow(viewClustering(temporalRow, resolver));
-        builder.addRowDeletion(deletion);
-        return PartitionUpdate.singleRowUpdate(viewCfm, partitionKey, builder.build());
-    }
-
-    /**
-     * @return PartitionUpdate containing a complex tombstone for a TemporalRow, and the collection's column identifier.
-     */
-    private PartitionUpdate createComplexTombstone(TemporalRow temporalRow,
-                                                   DecoratedKey partitionKey,
-                                                   ColumnDefinition deletedColumn,
-                                                   DeletionTime deletionTime,
-                                                   TemporalRow.Resolver resolver,
-                                                   int nowInSec)
-    {
-        CFMetaData viewCfm = definition.metadata;
-        Row.Builder builder = BTreeRow.unsortedBuilder(nowInSec);
-        builder.newRow(viewClustering(temporalRow, resolver));
-        builder.addComplexDeletion(deletedColumn, deletionTime);
-        return PartitionUpdate.singleRowUpdate(viewCfm, partitionKey, builder.build());
+        return definition.metadata.getColumnDefinition(baseColumn.name);
     }
 
     /**
-     * @return View's DecoratedKey or null, if one of the view's primary key components has an invalid resolution from
-     *         the TemporalRow and its Resolver
+     * The base column corresponding to the provided view column. This should
+     * never return {@code null} since a view can't have its "own" columns.
      */
-    private DecoratedKey viewPartitionKey(TemporalRow temporalRow, TemporalRow.Resolver resolver)
+    public ColumnDefinition getBaseColumn(ColumnDefinition viewColumn)
     {
-        List<ColumnDefinition> partitionDefs = this.columns.partitionDefs;
-        Object[] partitionKey = new Object[partitionDefs.size()];
-
-        for (int i = 0; i < partitionKey.length; i++)
-        {
-            ByteBuffer value = temporalRow.clusteringValue(partitionDefs.get(i), resolver);
-
-            if (value == null)
-                return null;
-
-            partitionKey[i] = value;
-        }
-
-        CFMetaData metadata = definition.metadata;
-        return metadata.decorateKey(CFMetaData.serializePartitionKey(metadata
-                                                                     .getKeyValidatorAsClusteringComparator()
-                                                                     .make(partitionKey)));
+        ColumnDefinition baseColumn = baseCfs.metadata.getColumnDefinition(viewColumn.name);
+        assert baseColumn != null;
+        return baseColumn;
     }
 
     /**
-     * @return mutation which contains the tombstone for the referenced TemporalRow, or null if not necessary.
-     * TemporalRow's can reference at most one view row; there will be at most one row to be tombstoned, so only one
-     * mutation is necessary
-     */
-    private PartitionUpdate createRangeTombstoneForRow(TemporalRow temporalRow)
-    {
-        // Primary Key and Clustering columns do not generate tombstones
-        if (viewPKIncludesOnlyBasePKColumns)
-            return null;
-
-        boolean hasUpdate = false;
-        List<ColumnDefinition> primaryKeyDefs = this.columns.primaryKeyDefs;
-        for (ColumnDefinition viewPartitionKeys : primaryKeyDefs)
-        {
-            if (!viewPartitionKeys.isPrimaryKeyColumn() && temporalRow.clusteringValue(viewPartitionKeys, TemporalRow.oldValueIfUpdated) != null)
-                hasUpdate = true;
-        }
-
-        if (!hasUpdate)
-            return null;
-
-        TemporalRow.Resolver resolver = TemporalRow.earliest;
-        return createTombstone(temporalRow,
-                               viewPartitionKey(temporalRow, resolver),
-                               Row.Deletion.shadowable(new DeletionTime(temporalRow.viewClusteringTimestamp(), temporalRow.nowInSec)),
-                               resolver,
-                               temporalRow.nowInSec);
-    }
-
-    /**
-     * @return Mutation which is the transformed base table mutation for the view.
-     */
-    private PartitionUpdate createUpdatesForInserts(TemporalRow temporalRow)
-    {
-        TemporalRow.Resolver resolver = TemporalRow.latest;
-
-        DecoratedKey partitionKey = viewPartitionKey(temporalRow, resolver);
-        CFMetaData viewCfm = definition.metadata;
-
-        if (partitionKey == null)
-        {
-            // Not having a partition key means we aren't updating anything
-            return null;
-        }
-
-        Row.Builder regularBuilder = BTreeRow.unsortedBuilder(temporalRow.nowInSec);
-
-        CBuilder clustering = CBuilder.create(viewCfm.comparator);
-        for (int i = 0; i < viewCfm.clusteringColumns().size(); i++)
-        {
-            ColumnDefinition column = viewCfm.clusteringColumns().get(i);
-            ByteBuffer value = temporalRow.clusteringValue(column, resolver);
-
-            // handle single-column deletions correctly to avoid nulls in the view primary key
-            if (value == null)
-                return null;
-
-            clustering.add(value);
-        }
-        regularBuilder.newRow(clustering.build());
-        regularBuilder.addPrimaryKeyLivenessInfo(LivenessInfo.create(viewCfm,
-                                                                     temporalRow.viewClusteringTimestamp(),
-                                                                     temporalRow.viewClusteringTtl(),
-                                                                     temporalRow.viewClusteringLocalDeletionTime()));
-
-        for (ColumnDefinition columnDefinition : viewCfm.allColumns())
-        {
-            if (columnDefinition.isPrimaryKeyColumn())
-                continue;
-
-            for (Cell cell : temporalRow.values(columnDefinition, resolver))
-            {
-                regularBuilder.addCell(cell);
-            }
-        }
-
-        Row row = regularBuilder.build();
-
-        // although we check for empty rows in updateAppliesToView(), if there are any good rows in the PartitionUpdate,
-        // all rows in the partition will be processed, and we need to handle empty/non-live rows here (CASSANDRA-10614)
-        if (row.isEmpty())
-            return null;
-
-        return PartitionUpdate.singleRowUpdate(viewCfm, partitionKey, row);
-    }
-
-    /**
-     * @param partition Update which possibly contains deletion info for which to generate view tombstones.
-     * @return    View Tombstones which delete all of the rows which have been removed from the base table with
-     *            {@param partition}
-     */
-    private Collection<Mutation> createForDeletionInfo(TemporalRow.Set rowSet, AbstractBTreePartition partition)
-    {
-        final TemporalRow.Resolver resolver = TemporalRow.earliest;
-
-        DeletionInfo deletionInfo = partition.deletionInfo();
-
-        List<Mutation> mutations = new ArrayList<>();
-
-        // Check the complex columns to see if there are any which may have tombstones we need to create for the view
-        if (!columns.baseComplexColumns.isEmpty())
-        {
-            for (Row row : partition)
-            {
-                if (!row.hasComplexDeletion())
-                    continue;
-
-                TemporalRow temporalRow = rowSet.getClustering(row.clustering());
-
-                assert temporalRow != null;
-
-                for (ColumnDefinition definition : columns.baseComplexColumns)
-                {
-                    ComplexColumnData columnData = row.getComplexColumnData(definition);
-
-                    if (columnData != null)
-                    {
-                        DeletionTime time = columnData.complexDeletion();
-                        if (!time.isLive())
-                        {
-                            DecoratedKey targetKey = viewPartitionKey(temporalRow, resolver);
-                            if (targetKey != null)
-                                mutations.add(new Mutation(createComplexTombstone(temporalRow, targetKey, definition, time, resolver, temporalRow.nowInSec)));
-                        }
-                    }
-                }
-            }
-        }
-
-        ReadCommand command = null;
-
-        if (!deletionInfo.isLive())
-        {
-            // We have to generate tombstones for all of the affected rows, but we don't have the information in order
-            // to create them. This requires that we perform a read for the entire range that is being tombstoned, and
-            // generate a tombstone for each. This may be slow, because a single range tombstone can cover up to an
-            // entire partition of data which is not distributed on a single partition node.
-            DecoratedKey dk = rowSet.dk;
-
-            if (!deletionInfo.getPartitionDeletion().isLive())
-            {
-                command = getSelectStatement().internalReadForView(dk, rowSet.nowInSec);
-            }
-            else
-            {
-                SinglePartitionSliceBuilder builder = new SinglePartitionSliceBuilder(baseCfs, dk);
-                Iterator<RangeTombstone> tombstones = deletionInfo.rangeIterator(false);
-                while (tombstones.hasNext())
-                {
-                    RangeTombstone tombstone = tombstones.next();
-
-                    builder.addSlice(tombstone.deletedSlice());
-                }
-
-                command = builder.build();
-            }
-        }
-
-        if (command == null)
-        {
-            ReadQuery selectQuery = getReadQuery();
-            SinglePartitionSliceBuilder builder = null;
-            for (Row row : partition)
-            {
-                if (!row.deletion().isLive())
-                {
-                    if (!selectQuery.selectsClustering(rowSet.dk, row.clustering()))
-                        continue;
-
-                    if (builder == null)
-                        builder = new SinglePartitionSliceBuilder(baseCfs, rowSet.dk);
-                    builder.addSlice(Slice.make(row.clustering()));
-                }
-            }
-
-            if (builder != null)
-                command = builder.build();
-        }
-
-        if (command != null)
-        {
-            ReadQuery selectQuery = getReadQuery();
-            assert selectQuery.selectsKey(rowSet.dk);
-
-            // We may have already done this work for another MV update so check
-            if (!rowSet.hasTombstonedExisting())
-            {
-                QueryPager pager = command.getPager(null, Server.CURRENT_VERSION);
-
-                // Add all of the rows which were recovered from the query to the row set
-                while (!pager.isExhausted())
-                {
-                    try (ReadOrderGroup orderGroup = pager.startOrderGroup();
-                         PartitionIterator iter = pager.fetchPageInternal(128, orderGroup))
-                    {
-                        if (!iter.hasNext())
-                            break;
-
-                        try (RowIterator rowIterator = iter.next())
-                        {
-                            while (rowIterator.hasNext())
-                            {
-                                Row row = rowIterator.next();
-                                if (selectQuery.selectsClustering(rowSet.dk, row.clustering()))
-                                    rowSet.addRow(row, false);
-                            }
-                        }
-                    }
-                }
-
-                //Incase we fetched nothing, avoid re checking on another MV update
-                rowSet.setTombstonedExisting();
-            }
-
-            // If the temporal row has been deleted by the deletion info, we generate the corresponding range tombstone
-            // for the view.
-            for (TemporalRow temporalRow : rowSet)
-            {
-                DeletionTime deletionTime = temporalRow.deletionTime(partition);
-                if (!deletionTime.isLive())
-                {
-                    DecoratedKey value = viewPartitionKey(temporalRow, resolver);
-                    if (value != null)
-                    {
-                        PartitionUpdate update = createTombstone(temporalRow, value, Row.Deletion.regular(deletionTime), resolver, temporalRow.nowInSec);
-                        if (update != null)
-                            mutations.add(new Mutation(update));
-                    }
-                }
-            }
-        }
-
-        return !mutations.isEmpty() ? mutations : null;
-    }
-
-    /**
-     * Read and update temporal rows in the set which have corresponding values stored on the local node
+     * Whether the view might be affected by the provided update.
+     * <p>
+     * Note that having this method return {@code true} is not an absolute guarantee that the view will be
+     * updated, just that it most likely will, but a {@code false} return guarantees it won't be affected).
+     *
+     * @param partitionKey the partition key that is updated.
+     * @param update the update being applied.
+     * @return {@code false} if we can guarantee that inserting {@code update} for key {@code partitionKey}
+     * won't affect the view in any way, {@code true} otherwise.
      */
-    private void readLocalRows(TemporalRow.Set rowSet)
+    public boolean mayBeAffectedBy(DecoratedKey partitionKey, Row update)
     {
-        long start = System.currentTimeMillis();
-        SinglePartitionSliceBuilder builder = new SinglePartitionSliceBuilder(baseCfs, rowSet.dk);
-
-        for (TemporalRow temporalRow : rowSet)
-            builder.addSlice(temporalRow.baseSlice());
+        // We can guarantee that the view won't be affected if:
+        //  - the clustering is excluded by the view filter (note that this isn't true of the filter on regular columns:
+        //    even if an update don't match a view condition on a regular column, that update can still invalidate an pre-existing
+        //    entry).
+        //  - or the update don't modify any of the columns impacting the view (where "impacting" the view means that column is
+        //    neither included in the view, nor used by the view filter).
+        if (!getReadQuery().selectsClustering(partitionKey, update.clustering()))
+            return false;
 
-        QueryPager pager = builder.build().getPager(null, Server.CURRENT_VERSION);
+        // 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;
 
-        while (!pager.isExhausted())
+        for (ColumnData data : update)
         {
-            try (ReadOrderGroup orderGroup = pager.startOrderGroup();
-                 PartitionIterator iter = pager.fetchPageInternal(128, orderGroup))
-            {
-                while (iter.hasNext())
-                {
-                    try (RowIterator rows = iter.next())
-                    {
-                        while (rows.hasNext())
-                        {
-                            rowSet.addRow(rows.next(), false);
-                        }
-                    }
-                }
-            }
+            if (definition.metadata.getColumnDefinition(data.column().name) != null)
+                return true;
         }
-        baseCfs.metric.viewReadTime.update(System.currentTimeMillis() - start, TimeUnit.MILLISECONDS);
-    }
-
-    /**
-     * @return Set of rows which are contained in the partition update {@param partition}
-     */
-    private TemporalRow.Set separateRows(AbstractBTreePartition partition, Set<ColumnIdentifier> viewPrimaryKeyCols)
-    {
-
-        TemporalRow.Set rowSet = new TemporalRow.Set(baseCfs, viewPrimaryKeyCols, partition.partitionKey().getKey());
-
-        for (Row row : partition)
-            rowSet.addRow(row, true);
-
-        return rowSet;
+        return false;
     }
 
     /**
-     * Splits the partition update up and adds the existing state to each row.
-     * This data can be reused for multiple MV updates on the same base table
+     * Whether a given base row matches the view filter (and thus if is should have a corresponding entry).
+     * <p>
+     * Note that this differs from {@link #mayBeAffectedBy} in that the provide row <b>must</b> be the current
+     * state of the base row, not just some updates to it. This method also has no false positive: a base
+     * row either do or don't match the view filter.
      *
-     * @param partition the mutation
-     * @param isBuilding If the view is currently being built, we do not query the values which are already stored,
-     *                   since all of the update will already be present in the base table.
-     * @return The set of temoral rows contained in this update
+     * @param partitionKey the partition key that is updated.
+     * @param baseRow the current state of a particular base row.
+     * @param nowInSec the current time in seconds (to decide what is live and what isn't).
+     * @return {@code true} if {@code baseRow} matches the view filters, {@code false} otherwise.
      */
-    public TemporalRow.Set getTemporalRowSet(AbstractBTreePartition partition, TemporalRow.Set existing, boolean isBuilding)
+    public boolean matchesViewFilter(DecoratedKey partitionKey, Row baseRow, int nowInSec)
     {
-        if (!updateAffectsView(partition))
-            return existing;
-
-        Set<ColumnIdentifier> columns = new HashSet<>(this.columns.primaryKeyDefs.size());
-        for (ColumnDefinition def : this.columns.primaryKeyDefs)
-            columns.add(def.name);
-
-        TemporalRow.Set rowSet;
-        if (existing == null)
-        {
-            rowSet = separateRows(partition, columns);
-
-            // If we are building the view, we do not want to add old values; they will always be the same
-            if (!isBuilding)
-                readLocalRows(rowSet);
-        }
-        else
-        {
-            rowSet = existing.withNewViewPrimaryKey(columns);
-        }
-
-        return rowSet;
+        return getReadQuery().selectsClustering(partitionKey, baseRow.clustering())
+            && getSelectStatement().rowFilterForInternalCalls().isSatisfiedBy(baseCfs.metadata, partitionKey, baseRow, nowInSec);
     }
 
     /**
@@ -656,61 +212,6 @@ public class View
         return query;
     }
 
-    /**
-     * @param isBuilding If the view is currently being built, we do not query the values which are already stored,
-     *                   since all of the update will already be present in the base table.
-     * @return View mutations which represent the changes necessary as long as previously created mutations for the view
-     *         have been applied successfully. This is based solely on the changes that are necessary given the current
-     *         state of the base table and the newly applying partition data.
-     */
-    public Collection<Mutation> createMutations(AbstractBTreePartition partition, TemporalRow.Set rowSet, boolean isBuilding)
-    {
-        if (!updateAffectsView(partition))
-            return null;
-
-        ReadQuery selectQuery = getReadQuery();
-        Collection<Mutation> mutations = null;
-        for (TemporalRow temporalRow : rowSet)
-        {
-            // In updateAffectsView, we check the partition to see if there is at least one row that matches the
-            // filters and is live.  If there is more than one row in the partition, we need to re-check each one
-            // individually.
-            if (partition.rowCount() != 1 && !selectQuery.selectsClustering(partition.partitionKey(), temporalRow.baseClustering()))
-                continue;
-
-            // If we are building, there is no need to check for partition tombstones; those values will not be present
-            // in the partition data
-            if (!isBuilding)
-            {
-                PartitionUpdate partitionTombstone = createRangeTombstoneForRow(temporalRow);
-                if (partitionTombstone != null)
-                {
-                    if (mutations == null) mutations = new LinkedList<>();
-                    mutations.add(new Mutation(partitionTombstone));
-                }
-            }
-
-            PartitionUpdate insert = createUpdatesForInserts(temporalRow);
-            if (insert != null)
-            {
-                if (mutations == null) mutations = new LinkedList<>();
-                mutations.add(new Mutation(insert));
-            }
-        }
-
-        if (!isBuilding)
-        {
-            Collection<Mutation> deletion = createForDeletionInfo(rowSet, partition);
-            if (deletion != null && !deletion.isEmpty())
-            {
-                if (mutations == null) mutations = new LinkedList<>();
-                mutations.addAll(deletion);
-            }
-        }
-
-        return mutations;
-    }
-
     public synchronized void build()
     {
         if (this.builder != null)

http://git-wip-us.apache.org/repos/asf/cassandra/blob/86ba2274/src/java/org/apache/cassandra/db/view/ViewBuilder.java
----------------------------------------------------------------------
diff --git a/src/java/org/apache/cassandra/db/view/ViewBuilder.java b/src/java/org/apache/cassandra/db/view/ViewBuilder.java
index 35b023b..b2b409b 100644
--- a/src/java/org/apache/cassandra/db/view/ViewBuilder.java
+++ b/src/java/org/apache/cassandra/db/view/ViewBuilder.java
@@ -19,6 +19,7 @@
 package org.apache.cassandra.db.view;
 
 import java.util.Collection;
+import java.util.Collections;
 import java.util.UUID;
 import java.util.concurrent.TimeUnit;
 import java.util.concurrent.atomic.AtomicLong;
@@ -35,9 +36,8 @@ import org.apache.cassandra.db.compaction.CompactionInfo;
 import org.apache.cassandra.db.compaction.CompactionManager;
 import org.apache.cassandra.db.compaction.OperationType;
 import org.apache.cassandra.db.lifecycle.SSTableSet;
-import org.apache.cassandra.db.partitions.FilteredPartition;
-import org.apache.cassandra.db.partitions.PartitionIterator;
-import org.apache.cassandra.db.rows.RowIterator;
+import org.apache.cassandra.db.partitions.*;
+import org.apache.cassandra.db.rows.*;
 import org.apache.cassandra.dht.Range;
 import org.apache.cassandra.dht.Token;
 import org.apache.cassandra.io.sstable.ReducingKeyIterator;
@@ -76,28 +76,22 @@ public class ViewBuilder extends CompactionInfo.Holder
         if (!selectQuery.selectsKey(key))
             return;
 
-        QueryPager pager = view.getSelectStatement().internalReadForView(key, FBUtilities.nowInSeconds()).getPager(null, Server.CURRENT_VERSION);
+        int nowInSec = FBUtilities.nowInSeconds();
+        SinglePartitionReadCommand command = view.getSelectStatement().internalReadForView(key, nowInSec);
 
-        while (!pager.isExhausted())
+        // We're rebuilding everything from what's on disk, so we read everything, consider that as new updates
+        // and pretend that there is nothing pre-existing.
+        UnfilteredRowIterator empty = UnfilteredRowIterators.noRowsIterator(baseCfs.metadata, key, Rows.EMPTY_STATIC_ROW, DeletionTime.LIVE, false);
+
+        Collection<Mutation> mutations;
+        try (ReadOrderGroup orderGroup = command.startOrderGroup();
+             UnfilteredRowIterator data = UnfilteredPartitionIterators.getOnlyElement(command.executeLocally(orderGroup), command))
         {
-           try (ReadOrderGroup orderGroup = pager.startOrderGroup();
-                PartitionIterator partitionIterator = pager.fetchPageInternal(128, orderGroup))
-           {
-               if (!partitionIterator.hasNext())
-                   return;
-
-               try (RowIterator rowIterator = partitionIterator.next())
-               {
-                   FilteredPartition partition = FilteredPartition.create(rowIterator);
-                   TemporalRow.Set temporalRows = view.getTemporalRowSet(partition, null, true);
-
-                   Collection<Mutation> mutations = view.createMutations(partition, temporalRows, true);
-
-                   if (mutations != null)
-                       StorageProxy.mutateMV(key.getKey(), mutations, true, noBase);
-               }
-           }
+            mutations = baseCfs.keyspace.viewManager.forTable(baseCfs.metadata).generateViewUpdates(Collections.singleton(view), data, empty, nowInSec);
         }
+
+        if (!mutations.isEmpty())
+            StorageProxy.mutateMV(key.getKey(), mutations, true, noBase);
     }
 
     public void run()

http://git-wip-us.apache.org/repos/asf/cassandra/blob/86ba2274/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 9fe0544..fd04b97 100644
--- a/src/java/org/apache/cassandra/db/view/ViewManager.java
+++ b/src/java/org/apache/cassandra/db/view/ViewManager.java
@@ -19,23 +19,21 @@ package org.apache.cassandra.db.view;
 
 import java.nio.ByteBuffer;
 import java.util.*;
-import java.util.concurrent.ConcurrentNavigableMap;
-import java.util.concurrent.ConcurrentSkipListMap;
-import java.util.concurrent.atomic.AtomicLong;
+import java.util.concurrent.ConcurrentMap;
+import java.util.concurrent.ConcurrentHashMap;
 import java.util.concurrent.locks.Lock;
 
-import com.google.common.collect.Lists;
 import com.google.common.util.concurrent.Striped;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
 
+import org.apache.cassandra.config.CFMetaData;
 import org.apache.cassandra.config.DatabaseDescriptor;
 import org.apache.cassandra.config.ViewDefinition;
 import org.apache.cassandra.db.*;
-import org.apache.cassandra.db.commitlog.ReplayPosition;
-import org.apache.cassandra.db.partitions.PartitionUpdate;
-import org.apache.cassandra.dht.Token;
-import org.apache.cassandra.service.StorageProxy;
-import org.slf4j.Logger;
-import org.slf4j.LoggerFactory;
+import org.apache.cassandra.db.rows.*;
+import org.apache.cassandra.db.partitions.*;
+
 
 /**
  * Manages {@link View}'s for a single {@link ColumnFamilyStore}. All of the views for that table are created when this
@@ -45,110 +43,30 @@ import org.slf4j.LoggerFactory;
  * any views {@link ViewManager#updatesAffectView(Collection, boolean)}, provide locks to prevent multiple
  * updates from creating incoherent updates in the view {@link ViewManager#acquireLockFor(ByteBuffer)}, and
  * to affect change on the view.
+ *
+ * TODO: I think we can get rid of that class. For addition/removal of view by names, we could move it Keyspace. And we
+ * not sure it's even worth keeping viewsByName as none of the related operation are performance sensitive so we could
+ * find the view by iterating over the CFStore.viewManager directly.
+ * For the lock, it could move to Keyspace too, but I don't remmenber why it has to be at the keyspace level and if it
+ * can be at the table level, maybe that's where it should be.
  */
 public class ViewManager
 {
     private static final Logger logger = LoggerFactory.getLogger(ViewManager.class);
 
-    public class ForStore
-    {
-        private final ConcurrentNavigableMap<String, View> viewsByName;
-
-        public ForStore()
-        {
-            this.viewsByName = new ConcurrentSkipListMap<>();
-        }
-
-        public Iterable<View> allViews()
-        {
-            return viewsByName.values();
-        }
-
-        public Iterable<ColumnFamilyStore> allViewsCfs()
-        {
-            List<ColumnFamilyStore> viewColumnFamilies = new ArrayList<>();
-            for (View view : allViews())
-                viewColumnFamilies.add(keyspace.getColumnFamilyStore(view.getDefinition().viewName));
-            return viewColumnFamilies;
-        }
-
-        public void forceBlockingFlush()
-        {
-            for (ColumnFamilyStore viewCfs : allViewsCfs())
-                viewCfs.forceBlockingFlush();
-        }
-
-        public void dumpMemtables()
-        {
-            for (ColumnFamilyStore viewCfs : allViewsCfs())
-                viewCfs.dumpMemtable();
-        }
-
-        public void truncateBlocking(long truncatedAt)
-        {
-            for (ColumnFamilyStore viewCfs : allViewsCfs())
-            {
-                ReplayPosition replayAfter = viewCfs.discardSSTables(truncatedAt);
-                SystemKeyspace.saveTruncationRecord(viewCfs, truncatedAt, replayAfter);
-            }
-        }
-
-        public void addView(View view)
-        {
-            viewsByName.put(view.name, view);
-        }
-
-        public void removeView(String name)
-        {
-            viewsByName.remove(name);
-        }
-    }
-
     private static final Striped<Lock> LOCKS = Striped.lazyWeakLock(DatabaseDescriptor.getConcurrentViewWriters() * 1024);
 
     private static final boolean enableCoordinatorBatchlog = Boolean.getBoolean("cassandra.mv_enable_coordinator_batchlog");
 
-    private final ConcurrentNavigableMap<UUID, ForStore> viewManagersByStore;
-    private final ConcurrentNavigableMap<String, View> viewsByName;
+    private final ConcurrentMap<String, View> viewsByName = new ConcurrentHashMap<>();
+    private final ConcurrentMap<UUID, TableViews> viewsByBaseTable = new ConcurrentHashMap<>();
     private final Keyspace keyspace;
 
     public ViewManager(Keyspace keyspace)
     {
-        this.viewManagersByStore = new ConcurrentSkipListMap<>();
-        this.viewsByName = new ConcurrentSkipListMap<>();
         this.keyspace = keyspace;
     }
 
-    /**
-     * Calculates and pushes updates to the views replicas. The replicas are determined by
-     * {@link ViewUtils#getViewNaturalEndpoint(String, Token, Token)}.
-     */
-    public void pushViewReplicaUpdates(PartitionUpdate update, boolean writeCommitLog, AtomicLong baseComplete)
-    {
-        List<Mutation> mutations = null;
-        TemporalRow.Set temporalRows = null;
-        for (Map.Entry<String, View> view : viewsByName.entrySet())
-        {
-            // Make sure that we only get mutations from views which are affected since the set includes all views for a
-            // keyspace. This will prevent calling getTemporalRowSet for the wrong base table.
-            if (view.getValue().updateAffectsView(update))
-            {
-                temporalRows = view.getValue().getTemporalRowSet(update, temporalRows, false);
-
-                Collection<Mutation> viewMutations = view.getValue().createMutations(update, temporalRows, false);
-                if (viewMutations != null && !viewMutations.isEmpty())
-                {
-                    if (mutations == null)
-                        mutations = Lists.newLinkedList();
-                    mutations.addAll(viewMutations);
-                }
-            }
-        }
-
-        if (mutations != null)
-            StorageProxy.mutateMV(update.partitionKey().getKey(), mutations, writeCommitLog, baseComplete);
-    }
-
     public boolean updatesAffectView(Collection<? extends IMutation> mutations, boolean coordinatorBatchlog)
     {
         if (coordinatorBatchlog && !enableCoordinatorBatchlog)
@@ -156,25 +74,22 @@ public class ViewManager
 
         for (IMutation mutation : mutations)
         {
-            for (PartitionUpdate cf : mutation.getPartitionUpdates())
+            for (PartitionUpdate update : mutation.getPartitionUpdates())
             {
-                assert keyspace.getName().equals(cf.metadata().ksName);
+                assert keyspace.getName().equals(update.metadata().ksName);
 
                 if (coordinatorBatchlog && keyspace.getReplicationStrategy().getReplicationFactor() == 1)
                     continue;
 
-                for (View view : allViews())
-                {
-                    if (view.updateAffectsView(cf))
-                        return true;
-                }
+                if (!forTable(update.metadata()).updatedViews(update).isEmpty())
+                    return true;
             }
         }
 
         return false;
     }
 
-    public Iterable<View> allViews()
+    private Iterable<View> allViews()
     {
         return viewsByName.values();
     }
@@ -222,7 +137,7 @@ public class ViewManager
     public void addView(ViewDefinition definition)
     {
         View view = new View(definition, keyspace.getColumnFamilyStore(definition.baseTableId));
-        forTable(view.getDefinition().baseTableId).addView(view);
+        forTable(view.getDefinition().baseTableMetadata()).add(view);
         viewsByName.put(definition.viewName, view);
     }
 
@@ -233,7 +148,7 @@ public class ViewManager
         if (view == null)
             return;
 
-        forTable(view.getDefinition().baseTableId).removeView(name);
+        forTable(view.getDefinition().baseTableMetadata()).removeByName(name);
         SystemKeyspace.setViewRemoved(keyspace.getName(), view.name);
     }
 
@@ -243,17 +158,18 @@ public class ViewManager
             view.build();
     }
 
-    public ForStore forTable(UUID baseId)
+    public TableViews forTable(CFMetaData metadata)
     {
-        ForStore forStore = viewManagersByStore.get(baseId);
-        if (forStore == null)
+        UUID baseId = metadata.cfId;
+        TableViews views = viewsByBaseTable.get(baseId);
+        if (views == null)
         {
-            forStore = new ForStore();
-            ForStore previous = viewManagersByStore.put(baseId, forStore);
+            views = new TableViews(metadata);
+            TableViews previous = viewsByBaseTable.putIfAbsent(baseId, views);
             if (previous != null)
-                forStore = previous;
+                views = previous;
         }
-        return forStore;
+        return views;
     }
 
     public static Lock acquireLockFor(ByteBuffer key)

http://git-wip-us.apache.org/repos/asf/cassandra/blob/86ba2274/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
new file mode 100644
index 0000000..af025cb
--- /dev/null
+++ b/src/java/org/apache/cassandra/db/view/ViewUpdateGenerator.java
@@ -0,0 +1,549 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.cassandra.db.view;
+
+import java.nio.ByteBuffer;
+import java.util.*;
+
+import com.google.common.collect.Iterators;
+import com.google.common.collect.PeekingIterator;
+
+import org.apache.cassandra.config.CFMetaData;
+import org.apache.cassandra.config.ColumnDefinition;
+import org.apache.cassandra.config.ViewDefinition;
+import org.apache.cassandra.db.*;
+import org.apache.cassandra.db.rows.*;
+import org.apache.cassandra.db.partitions.*;
+import org.apache.cassandra.db.marshal.AbstractType;
+import org.apache.cassandra.db.marshal.CompositeType;
+
+/**
+ * Creates the updates to apply to a view given the existing rows in the base
+ * table and the updates that we're applying to them (this handles updates
+ * on a single partition only).
+ *
+ * This class is used by passing the updates made to the base table to
+ * {@link #addBaseTableUpdate} and calling {@link #generateViewUpdates} once all updates have
+ * been handled to get the resulting view mutations.
+ */
+public class ViewUpdateGenerator
+{
+    private final View view;
+    private final int nowInSec;
+
+    private final CFMetaData baseMetadata;
+    private final DecoratedKey baseDecoratedKey;
+    private final ByteBuffer[] basePartitionKey;
+
+    private final CFMetaData viewMetadata;
+
+    private final Map<DecoratedKey, PartitionUpdate> updates = new HashMap<>();
+
+    // Reused internally to build a new entry
+    private final ByteBuffer[] currentViewEntryPartitionKey;
+    private final Row.Builder currentViewEntryBuilder;
+
+    /**
+     * The type of type update action to perform to the view for a given base table
+     * update.
+     */
+    private enum UpdateAction
+    {
+        NONE,            // There was no view entry and none should be added
+        NEW_ENTRY,       // There was no entry but there is one post-update
+        DELETE_OLD,      // There was an entry but there is nothing after update
+        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}.
+     *
+     * @param view the view for which this will be building updates for.
+     * @param basePartitionKey the partition key for the base table partition for which
+     * we'll handle updates for.
+     * @param nowInSec the current time in seconds. Used to decide if data are live or not
+     * and as base reference for new deletions.
+     */
+    public ViewUpdateGenerator(View view, DecoratedKey basePartitionKey, int nowInSec)
+    {
+        this.view = view;
+        this.nowInSec = nowInSec;
+
+        this.baseMetadata = view.getDefinition().baseTableMetadata();
+        this.baseDecoratedKey = basePartitionKey;
+        this.basePartitionKey = extractKeyComponents(basePartitionKey, baseMetadata.getKeyValidator());
+
+        this.viewMetadata = view.getDefinition().metadata;
+
+        this.currentViewEntryPartitionKey = new ByteBuffer[viewMetadata.partitionKeyColumns().size()];
+        this.currentViewEntryBuilder = BTreeRow.sortedBuilder();
+    }
+
+    private static ByteBuffer[] extractKeyComponents(DecoratedKey partitionKey, AbstractType<?> type)
+    {
+        return type instanceof CompositeType
+             ? ((CompositeType)type).split(partitionKey.getKey())
+             : new ByteBuffer[]{ partitionKey.getKey() };
+    }
+
+    /**
+     * Adds to this generator the updates to be made to the view given a base table row
+     * before and after an update.
+     *
+     * @param existingBaseRow the base table row as it is before an update.
+     * @param mergedBaseRow the base table row after the update is applied (note that
+     * this is not just the new update, but rather the resulting row).
+     */
+    public void addBaseTableUpdate(Row existingBaseRow, Row mergedBaseRow)
+    {
+        switch (updateAction(existingBaseRow, mergedBaseRow))
+        {
+            case NONE:
+                return;
+            case NEW_ENTRY:
+                createEntry(mergedBaseRow);
+                return;
+            case DELETE_OLD:
+                deleteOldEntry(existingBaseRow);
+                return;
+            case UPDATE_EXISTING:
+                updateEntry(existingBaseRow, mergedBaseRow);
+                return;
+            case SWITCH_ENTRY:
+                createEntry(mergedBaseRow);
+                deleteOldEntry(existingBaseRow);
+                return;
+        }
+    }
+
+    /**
+     * Returns the updates that needs to be done to the view given the base table updates
+     * passed to {@link #generateViewMutations}.
+     *
+     * @return the updates to do to the view.
+     */
+    public Collection<PartitionUpdate> generateViewUpdates()
+    {
+        return updates.values();
+    }
+
+    /**
+     * Compute which type of action needs to be performed to the view for a base table row
+     * before and after an update.
+     */
+    private UpdateAction updateAction(Row existingBaseRow, Row mergedBaseRow)
+    {
+        // Having existing empty is useful, it just means we'll insert a brand new entry for mergedBaseRow,
+        // but if we have no update at all, we shouldn't get there.
+        assert !mergedBaseRow.isEmpty();
+
+        // Note that none of the base PK columns will differ since we're intrinsically dealing
+        // with the same base row. So we have to check 3 things:
+        //   1) that the clustering doesn't have a null, which can happen for compact tables. If that's the case,
+        //      there is no corresponding entries.
+        //   2) if there is a column not part of the base PK in the view PK, whether it is changed by the update.
+        //   3) whether mergedBaseRow actually match the view SELECT filter
+
+        if (baseMetadata.isCompactTable())
+        {
+            Clustering clustering = mergedBaseRow.clustering();
+            for (int i = 0; i < clustering.size(); i++)
+            {
+                if (clustering.get(i) == null)
+                    return UpdateAction.NONE;
+            }
+        }
+
+        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.
+
+            // Note that we allow existingBaseRow to be null and treat it as empty (see MultiViewUpdateBuilder.generateViewsMutations).
+            boolean existingHasLiveData = existingBaseRow != null && existingBaseRow.hasLiveData(nowInSec);
+            boolean mergedHasLiveData = mergedBaseRow.hasLiveData(nowInSec);
+            return existingHasLiveData
+                 ? (mergedHasLiveData ? UpdateAction.UPDATE_EXISTING : UpdateAction.DELETE_OLD)
+                 : (mergedHasLiveData ? UpdateAction.NEW_ENTRY : UpdateAction.NONE);
+        }
+
+        ColumnDefinition baseColumn = view.baseNonPKColumnsInViewPK.get(0);
+        assert !baseColumn.isComplex() : "A complex column couldn't be part of the view PK";
+        Cell before = existingBaseRow == null ? null : existingBaseRow.getCell(baseColumn);
+        Cell after = mergedBaseRow.getCell(baseColumn);
+
+        // If the update didn't modified this column, the cells will be the same object so it's worth checking
+        if (before == after)
+            return before == null ? UpdateAction.NONE : UpdateAction.UPDATE_EXISTING;
+
+        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
+             : UpdateAction.SWITCH_ENTRY;
+    }
+
+    private boolean matchesViewFilter(Row baseRow)
+    {
+        return view.matchesViewFilter(baseDecoratedKey, baseRow, nowInSec);
+    }
+
+    private boolean isLive(Cell cell)
+    {
+        return cell != null && cell.isLive(nowInSec);
+    }
+
+    /**
+     * Creates a view entry corresponding to the provided base row.
+     * <p>
+     * This method checks that the base row does match the view filter before applying it.
+     */
+    private void createEntry(Row baseRow)
+    {
+        // Before create a new entry, make sure it matches the view filter
+        if (!matchesViewFilter(baseRow))
+            return;
+
+        startNewUpdate(baseRow);
+        currentViewEntryBuilder.addPrimaryKeyLivenessInfo(computeLivenessInfoForEntry(baseRow));
+        currentViewEntryBuilder.addRowDeletion(baseRow.deletion());
+
+        for (ColumnData data : baseRow)
+        {
+            ColumnDefinition viewColumn = view.getViewColumn(data.column());
+            // If that base table column is not denormalized in the view, we had nothing to do.
+            // Alose, if it's part of the view PK it's already been taken into account in the clustering.
+            if (viewColumn == null || viewColumn.isPrimaryKeyColumn())
+                continue;
+
+            addColumnData(viewColumn, data);
+        }
+
+        submitUpdate();
+    }
+
+    /**
+     * Creates the updates to apply to the existing view entry given the base table row before
+     * and after the update, assuming that the update hasn't changed to which view entry the
+     * row correspond (that is, we know the columns composing the view PK haven't changed).
+     * <p>
+     * This method checks that the base row (before and after) does match the view filter before
+     * applying anything.
+     */
+    private void updateEntry(Row existingBaseRow, Row mergedBaseRow)
+    {
+        // While we know existingBaseRow and mergedBaseRow are corresponding to the same view entry,
+        // they may not match the view filter.
+        if (!matchesViewFilter(existingBaseRow))
+        {
+            createEntry(mergedBaseRow);
+            return;
+        }
+        if (!matchesViewFilter(mergedBaseRow))
+        {
+            deleteOldEntryInternal(existingBaseRow);
+            return;
+        }
+
+        startNewUpdate(mergedBaseRow);
+
+        // In theory, it may be the PK liveness and row deletion hasn't been change by the update
+        // and we could condition the 2 additions below. In practice though, it's as fast (if not
+        // faster) to compute those info than to check if they have changed so we keep it simple.
+        currentViewEntryBuilder.addPrimaryKeyLivenessInfo(computeLivenessInfoForEntry(mergedBaseRow));
+        currentViewEntryBuilder.addRowDeletion(mergedBaseRow.deletion());
+
+        // 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
+        // mergedBaseRow (note that including more cells than we strictly should isn't a problem
+        // for correction, so even if the code change and pointer equality don't work anymore, it'll
+        // only a slightly inefficiency which we can fix then).
+        // Note: we could alternatively use Rows.diff() for this, but because it is a bit more generic
+        // than what we need here, it's also a bit less efficient (it allocates more in particular),
+        // and this might be called a lot of time for view updates. So, given that this is not a whole
+        // lot of code anyway, it's probably doing the diff manually.
+        PeekingIterator<ColumnData> existingIter = Iterators.peekingIterator(existingBaseRow.iterator());
+        for (ColumnData mergedData : mergedBaseRow)
+        {
+            ColumnDefinition baseColumn = mergedData.column();
+            ColumnDefinition viewColumn = view.getViewColumn(baseColumn);
+            // If that base table column is not denormalized in the view, we had nothing to do.
+            // Alose, if it's part of the view PK it's already been taken into account in the clustering.
+            if (viewColumn == null || viewColumn.isPrimaryKeyColumn())
+                continue;
+
+            ColumnData existingData = null;
+            // Find if there is data for that column in the existing row
+            while (existingIter.hasNext())
+            {
+                int cmp = baseColumn.compareTo(existingIter.peek().column());
+                if (cmp < 0)
+                    break;
+
+                ColumnData next = existingIter.next();
+                if (cmp == 0)
+                {
+                    existingData = next;
+                    break;
+                }
+            }
+
+            if (existingData == null)
+            {
+                addColumnData(viewColumn, mergedData);
+                continue;
+            }
+
+            if (mergedData == existingData)
+                continue;
+
+            if (baseColumn.isComplex())
+            {
+                ComplexColumnData mergedComplexData = (ComplexColumnData)mergedData;
+                ComplexColumnData existingComplexData = (ComplexColumnData)existingData;
+                if (mergedComplexData.complexDeletion().supersedes(existingComplexData.complexDeletion()))
+                    currentViewEntryBuilder.addComplexDeletion(viewColumn, mergedComplexData.complexDeletion());
+
+                PeekingIterator<Cell> existingCells = Iterators.peekingIterator(existingComplexData.iterator());
+                for (Cell mergedCell : mergedComplexData)
+                {
+                    Cell existingCell = null;
+                    // Find if there is corresponding cell in the existing row
+                    while (existingCells.hasNext())
+                    {
+                        int cmp = baseColumn.cellPathComparator().compare(mergedCell.path(), existingCells.peek().path());
+                        if (cmp > 0)
+                            break;
+
+                        Cell next = existingCells.next();
+                        if (cmp == 0)
+                        {
+                            existingCell = next;
+                            break;
+                        }
+                    }
+
+                    if (mergedCell != existingCell)
+                        addCell(viewColumn, mergedCell);
+                }
+            }
+            else
+            {
+                // Note that we've already eliminated the case where merged == existing
+                addCell(viewColumn, (Cell)mergedData);
+            }
+        }
+
+        submitUpdate();
+    }
+
+    /**
+     * Deletes the view entry corresponding to the provided base row.
+     * <p>
+     * This method checks that the base row does match the view filter before bothering.
+     */
+    private void deleteOldEntry(Row existingBaseRow)
+    {
+        // 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);
+    }
+
+    private void deleteOldEntryInternal(Row existingBaseRow)
+    {
+        startNewUpdate(existingBaseRow);
+        DeletionTime dt = new DeletionTime(computeTimestampForEntryDeletion(existingBaseRow), nowInSec);
+        currentViewEntryBuilder.addRowDeletion(Row.Deletion.shadowable(dt));
+        submitUpdate();
+    }
+
+    /**
+     * Computes the partition key and clustering for a new view entry, and setup the internal
+     * row builder for the new row.
+     *
+     * This assumes that there is corresponding entry, i.e. no values for the partition key and
+     * clustering are null (since we have eliminated that case through updateAction).
+     */
+    private void startNewUpdate(Row baseRow)
+    {
+        ByteBuffer[] clusteringValues = new ByteBuffer[viewMetadata.clusteringColumns().size()];
+        for (ColumnDefinition viewColumn : viewMetadata.primaryKeyColumns())
+        {
+            ColumnDefinition baseColumn = view.getBaseColumn(viewColumn);
+            ByteBuffer value = getValueForPK(baseColumn, baseRow);
+            if (viewColumn.isPartitionKey())
+                currentViewEntryPartitionKey[viewColumn.position()] = value;
+            else
+                clusteringValues[viewColumn.position()] = value;
+        }
+
+        currentViewEntryBuilder.newRow(new Clustering(clusteringValues));
+    }
+
+    private LivenessInfo computeLivenessInfoForEntry(Row baseRow)
+    {
+        /*
+         * We need to compute both the timestamp and expiration.
+         *
+         * 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())
+        {
+            int ttl = baseLiveness.ttl();
+            int expirationTime = baseLiveness.localExpirationTime();
+            for (Cell cell : baseRow.cells())
+            {
+                if (cell.ttl() > ttl)
+                {
+                    ttl = cell.ttl();
+                    expirationTime = cell.localDeletionTime();
+                }
+            }
+            return ttl == baseLiveness.ttl()
+                 ? baseLiveness
+                 : LivenessInfo.create(baseLiveness.timestamp(), ttl, expirationTime);
+        }
+
+        ColumnDefinition baseColumn = view.baseNonPKColumnsInViewPK.get(0);
+        Cell cell = baseRow.getCell(baseColumn);
+        assert isLive(cell) : "We shouldn't have got there is the base row had no associated entry";
+
+        long timestamp = Math.max(baseLiveness.timestamp(), cell.timestamp());
+        return LivenessInfo.create(timestamp, cell.ttl(), cell.localDeletionTime());
+    }
+
+    private long computeTimestampForEntryDeletion(Row baseRow)
+    {
+        // 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)
+        {
+            if (!view.getDefinition().includes(data.column().name))
+                continue;
+
+            timestamp = Math.max(timestamp, data.maxTimestamp());
+        }
+        return timestamp;
+    }
+
+    private void addColumnData(ColumnDefinition viewColumn, ColumnData baseTableData)
+    {
+        assert viewColumn.isComplex() == baseTableData.column().isComplex();
+        if (!viewColumn.isComplex())
+        {
+            addCell(viewColumn, (Cell)baseTableData);
+            return;
+        }
+
+        ComplexColumnData complexData = (ComplexColumnData)baseTableData;
+        currentViewEntryBuilder.addComplexDeletion(viewColumn, complexData.complexDeletion());
+        for (Cell cell : complexData)
+            addCell(viewColumn, cell);
+    }
+
+    private void addCell(ColumnDefinition viewColumn, Cell baseTableCell)
+    {
+        assert !viewColumn.isPrimaryKeyColumn();
+        currentViewEntryBuilder.addCell(baseTableCell.withUpdatedColumn(viewColumn));
+    }
+
+    /**
+     * Finish building the currently updated view entry and add it to the other built
+     * updates.
+     */
+    private void submitUpdate()
+    {
+        Row row = currentViewEntryBuilder.build();
+        // I'm not sure we can reach there is there is nothing is updated, but adding an empty row breaks things
+        // and it costs us nothing to be prudent here.
+        if (row.isEmpty())
+            return;
+
+        DecoratedKey partitionKey = makeCurrentPartitionKey();
+        PartitionUpdate update = updates.get(partitionKey);
+        if (update == null)
+        {
+            // We can't really know which columns of the view will be updated nor how many row will be updated for this key
+            // so we rely on hopefully sane defaults.
+            update = new PartitionUpdate(viewMetadata, partitionKey, viewMetadata.partitionColumns(), 4);
+            updates.put(partitionKey, update);
+        }
+        update.add(row);
+    }
+
+    private DecoratedKey makeCurrentPartitionKey()
+    {
+        ByteBuffer rawKey = viewMetadata.partitionKeyColumns().size() == 1
+                          ? currentViewEntryPartitionKey[0]
+                          : CompositeType.build(currentViewEntryPartitionKey);
+
+        return viewMetadata.decorateKey(rawKey);
+    }
+
+    private ByteBuffer getValueForPK(ColumnDefinition column, Row row)
+    {
+        switch (column.kind)
+        {
+            case PARTITION_KEY:
+                return basePartitionKey[column.position()];
+            case CLUSTERING:
+                return row.clustering().get(column.position());
+            default:
+                // This shouldn't NPE as we shouldn't get there if the value can be null (or there is a bug in updateAction())
+                return row.getCell(column).value();
+        }
+    }
+}

http://git-wip-us.apache.org/repos/asf/cassandra/blob/86ba2274/test/unit/org/apache/cassandra/cql3/ViewTest.java
----------------------------------------------------------------------
diff --git a/test/unit/org/apache/cassandra/cql3/ViewTest.java b/test/unit/org/apache/cassandra/cql3/ViewTest.java
index ac4becb..4a1dc07 100644
--- a/test/unit/org/apache/cassandra/cql3/ViewTest.java
+++ b/test/unit/org/apache/cassandra/cql3/ViewTest.java
@@ -98,7 +98,7 @@ public class ViewTest extends CQLTester
     @Test
     public void testPartitionTombstone() throws Throwable
     {
-        createTable("CREATE TABLE %s (k1 int, c1 int , val int, PRIMARY KEY (k1))");
+        createTable("CREATE TABLE %s (k1 int, c1 int , val int, PRIMARY KEY (k1, c1))");
 
         execute("USE " + keyspace());
         executeNet(protocolVersion, "USE " + keyspace());
@@ -108,8 +108,8 @@ public class ViewTest extends CQLTester
         updateView("INSERT INTO %s (k1, c1, val) VALUES (1, 2, 200)");
         updateView("INSERT INTO %s (k1, c1, val) VALUES (1, 3, 300)");
 
-        Assert.assertEquals(1, execute("select * from %s").size());
-        Assert.assertEquals(1, execute("select * from view1").size());
+        Assert.assertEquals(2, execute("select * from %s").size());
+        Assert.assertEquals(2, execute("select * from view1").size());
 
         updateView("DELETE FROM %s WHERE k1 = 1");
 
@@ -814,18 +814,58 @@ public class ViewTest extends CQLTester
 
         createView("mv", "CREATE MATERIALIZED VIEW %s AS SELECT * FROM %%s WHERE c IS NOT NULL AND a IS NOT NULL AND b IS NOT NULL PRIMARY KEY (c, a, b)");
 
-        updateView("INSERT INTO %s (a, b, c, d) VALUES (?, ?, ?, ?) USING TTL 5", 1, 1, 1, 1);
+        updateView("INSERT INTO %s (a, b, c, d) VALUES (?, ?, ?, ?) USING TTL 3", 1, 1, 1, 1);
 
-        Thread.sleep(TimeUnit.SECONDS.toMillis(3));
+        Thread.sleep(TimeUnit.SECONDS.toMillis(1));
         updateView("INSERT INTO %s (a, b, c) VALUES (?, ?, ?)", 1, 1, 2);
 
-        Thread.sleep(TimeUnit.SECONDS.toMillis(3));
+        Thread.sleep(TimeUnit.SECONDS.toMillis(5));
         List<Row> results = executeNet(protocolVersion, "SELECT d FROM mv WHERE c = 2 AND a = 1 AND b = 1").all();
         Assert.assertEquals(1, results.size());
         Assert.assertTrue("There should be a null result given back due to ttl expiry", results.get(0).isNull(0));
     }
 
     @Test
+    public void ttlExpirationTest() throws Throwable
+    {
+        createTable("CREATE TABLE %s (" +
+                    "a int," +
+                    "b int," +
+                    "c int," +
+                    "d int," +
+                    "PRIMARY KEY (a, b))");
+
+        executeNet(protocolVersion, "USE " + keyspace());
+
+        createView("mv", "CREATE MATERIALIZED VIEW %s AS SELECT * FROM %%s WHERE c IS NOT NULL AND a IS NOT NULL AND b IS NOT NULL PRIMARY KEY (c, a, b)");
+
+        updateView("INSERT INTO %s (a, b, c, d) VALUES (?, ?, ?, ?) USING TTL 3", 1, 1, 1, 1);
+
+        Thread.sleep(TimeUnit.SECONDS.toMillis(4));
+        Assert.assertEquals(0, executeNet(protocolVersion, "SELECT * FROM mv WHERE c = 1 AND a = 1 AND b = 1").all().size());
+    }
+
+    @Test
+    public void rowDeletionTest() throws Throwable
+    {
+        createTable("CREATE TABLE %s (" +
+                    "a int," +
+                    "b int," +
+                    "c int," +
+                    "d int," +
+                    "PRIMARY KEY (a, b))");
+
+        executeNet(protocolVersion, "USE " + keyspace());
+
+        createView("mv", "CREATE MATERIALIZED VIEW %s AS SELECT * FROM %%s WHERE c IS NOT NULL AND a IS NOT NULL AND b IS NOT NULL PRIMARY KEY (c, a, b)");
+
+        String table = keyspace() + "." + currentTable();
+        updateView("DELETE FROM " + table + " USING TIMESTAMP 6 WHERE a = 1 AND b = 1;");
+        updateView("INSERT INTO %s (a, b, c, d) VALUES (?, ?, ?, ?) USING TIMESTAMP 3", 1, 1, 1, 1);
+        Assert.assertEquals(0, executeNet(protocolVersion, "SELECT * FROM mv WHERE c = 1 AND a = 1 AND b = 1").all().size());
+    }
+
+    @Test
     public void conflictingTimestampTest() throws Throwable
     {
         createTable("CREATE TABLE %s (" +

http://git-wip-us.apache.org/repos/asf/cassandra/blob/86ba2274/test/unit/org/apache/cassandra/db/rows/RowsTest.java
----------------------------------------------------------------------
diff --git a/test/unit/org/apache/cassandra/db/rows/RowsTest.java b/test/unit/org/apache/cassandra/db/rows/RowsTest.java
index dede867..b47bea2 100644
--- a/test/unit/org/apache/cassandra/db/rows/RowsTest.java
+++ b/test/unit/org/apache/cassandra/db/rows/RowsTest.java
@@ -239,7 +239,8 @@ public class RowsTest
                                                       BufferCell.live(kcvm, m, secondToTs(now), BB1, CellPath.create(BB1)),
                                                       BufferCell.live(kcvm, m, secondToTs(now), BB2, CellPath.create(BB2)));
         expectedCells.forEach(originalBuilder::addCell);
-        Row.Deletion rowDeletion = new Row.Deletion(new DeletionTime(ts, now), false);
+        // We need to use ts-1 so the deletion doesn't shadow what we've created
+        Row.Deletion rowDeletion = new Row.Deletion(new DeletionTime(ts-1, now), false);
         originalBuilder.addRowDeletion(rowDeletion);
 
         RowBuilder builder = new RowBuilder();
@@ -267,7 +268,8 @@ public class RowsTest
                                                       BufferCell.live(kcvm, m, ts, BB1, CellPath.create(BB1)),
                                                       BufferCell.live(kcvm, m, ts, BB2, CellPath.create(BB2)));
         expectedCells.forEach(builder::addCell);
-        Row.Deletion rowDeletion = new Row.Deletion(new DeletionTime(ts, now), false);
+        // We need to use ts-1 so the deletion doesn't shadow what we've created
+        Row.Deletion rowDeletion = new Row.Deletion(new DeletionTime(ts-1, now), false);
         builder.addRowDeletion(rowDeletion);
 
         StatsCollector collector = new StatsCollector();


[3/9] cassandra git commit: Refactor MV code

Posted by sl...@apache.org.
http://git-wip-us.apache.org/repos/asf/cassandra/blob/86ba2274/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 1b823aa..845a6ab 100644
--- a/src/java/org/apache/cassandra/db/view/View.java
+++ b/src/java/org/apache/cassandra/db/view/View.java
@@ -32,17 +32,15 @@ import org.apache.cassandra.cql3.statements.SelectStatement;
 import org.apache.cassandra.db.*;
 import org.apache.cassandra.config.*;
 import org.apache.cassandra.cql3.ColumnIdentifier;
-import org.apache.cassandra.db.AbstractReadCommandBuilder.SinglePartitionSliceBuilder;
 import org.apache.cassandra.db.compaction.CompactionManager;
-import org.apache.cassandra.db.partitions.AbstractBTreePartition;
-import org.apache.cassandra.db.partitions.PartitionIterator;
-import org.apache.cassandra.db.partitions.PartitionUpdate;
+import org.apache.cassandra.db.partitions.*;
 import org.apache.cassandra.db.rows.*;
 import org.apache.cassandra.schema.KeyspaceMetadata;
 import org.apache.cassandra.service.ClientState;
 import org.apache.cassandra.service.pager.QueryPager;
 import org.apache.cassandra.transport.Server;
 import org.apache.cassandra.utils.FBUtilities;
+import org.apache.cassandra.utils.btree.BTreeSet;
 import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
 
@@ -50,46 +48,18 @@ import org.slf4j.LoggerFactory;
  * A View copies data from a base table into a view table which can be queried independently from the
  * base. Every update which targets the base table must be fed through the {@link ViewManager} to ensure
  * that if a view needs to be updated, the updates are properly created and fed into the view.
- *
- * This class does the job of translating the base row to the view row.
- *
- * It handles reading existing state and figuring out what tombstones need to be generated.
- *
- * {@link View#createMutations(AbstractBTreePartition, TemporalRow.Set, boolean)} is the "main method"
- *
  */
 public class View
 {
     private static final Logger logger = LoggerFactory.getLogger(View.class);
 
-    /**
-     * The columns should all be updated together, so we use this object as group.
-     */
-    private static class Columns
-    {
-        //These are the base column definitions in terms of the *views* partitioning.
-        //Meaning we can see (for example) the partition key of the view contains a clustering key
-        //from the base table.
-        public final List<ColumnDefinition> partitionDefs;
-        public final List<ColumnDefinition> primaryKeyDefs;
-        public final List<ColumnDefinition> baseComplexColumns;
-
-        private Columns(List<ColumnDefinition> partitionDefs, List<ColumnDefinition> primaryKeyDefs, List<ColumnDefinition> baseComplexColumns)
-        {
-            this.partitionDefs = partitionDefs;
-            this.primaryKeyDefs = primaryKeyDefs;
-            this.baseComplexColumns = baseComplexColumns;
-        }
-    }
-
     public final String name;
     private volatile ViewDefinition definition;
 
     private final ColumnFamilyStore baseCfs;
 
-    private Columns columns;
+    public volatile List<ColumnDefinition> baseNonPKColumnsInViewPK;
 
-    private final boolean viewPKIncludesOnlyBasePKColumns;
     private final boolean includeAllColumns;
     private ViewBuilder builder;
 
@@ -104,12 +74,11 @@ public class View
                 ColumnFamilyStore baseCfs)
     {
         this.baseCfs = baseCfs;
-
-        name = definition.viewName;
-        includeAllColumns = definition.includeAllColumns;
-
-        viewPKIncludesOnlyBasePKColumns = updateDefinition(definition);
+        this.name = definition.viewName;
+        this.includeAllColumns = definition.includeAllColumns;
         this.rawSelect = definition.select;
+
+        updateDefinition(definition);
     }
 
     public ViewDefinition getDefinition()
@@ -118,513 +87,100 @@ public class View
     }
 
     /**
-     * Lookup column definitions in the base table that correspond to the view columns (should be 1:1)
-     *
-     * Notify caller if all primary keys in the view are ALL primary keys in the base. We do this to simplify
-     * tombstone checks.
-     *
-     * @param columns a list of columns to lookup in the base table
-     * @param definitions lists to populate for the base table definitions
-     * @return true if all view PKs are also Base PKs
-     */
-    private boolean resolveAndAddColumns(Iterable<ColumnIdentifier> columns, List<ColumnDefinition>... definitions)
-    {
-        boolean allArePrimaryKeys = true;
-        for (ColumnIdentifier identifier : columns)
-        {
-            ColumnDefinition cdef = baseCfs.metadata.getColumnDefinition(identifier);
-            assert cdef != null : "Could not resolve column " + identifier.toString();
-
-            for (List<ColumnDefinition> list : definitions)
-            {
-                list.add(cdef);
-            }
-
-            allArePrimaryKeys = allArePrimaryKeys && cdef.isPrimaryKeyColumn();
-        }
-
-        return allArePrimaryKeys;
-    }
-
-    /**
      * This updates the columns stored which are dependent on the base CFMetaData.
      *
      * @return true if the view contains only columns which are part of the base's primary key; false if there is at
      *         least one column which is not.
      */
-    public boolean updateDefinition(ViewDefinition definition)
+    public void updateDefinition(ViewDefinition definition)
     {
         this.definition = definition;
 
         CFMetaData viewCfm = definition.metadata;
-        List<ColumnDefinition> partitionDefs = new ArrayList<>(viewCfm.partitionKeyColumns().size());
-        List<ColumnDefinition> primaryKeyDefs = new ArrayList<>(viewCfm.partitionKeyColumns().size()
-                                                                + viewCfm.clusteringColumns().size());
-        List<ColumnDefinition> baseComplexColumns = new ArrayList<>();
-
-        // We only add the partition columns to the partitions list, but both partition columns and clustering
-        // columns are added to the primary keys list
-        boolean partitionAllPrimaryKeyColumns = resolveAndAddColumns(Iterables.transform(viewCfm.partitionKeyColumns(), cd -> cd.name), primaryKeyDefs, partitionDefs);
-        boolean clusteringAllPrimaryKeyColumns = resolveAndAddColumns(Iterables.transform(viewCfm.clusteringColumns(), cd -> cd.name), primaryKeyDefs);
-
-        for (ColumnDefinition cdef : baseCfs.metadata.allColumns())
+        List<ColumnDefinition> nonPKDefPartOfViewPK = new ArrayList<>();
+        for (ColumnDefinition baseColumn : baseCfs.metadata.allColumns())
         {
-            if (cdef.isComplex() && definition.includes(cdef.name))
-            {
-                baseComplexColumns.add(cdef);
-            }
+            ColumnDefinition viewColumn = getViewColumn(baseColumn);
+            if (viewColumn != null && !baseColumn.isPrimaryKeyColumn() && viewColumn.isPrimaryKeyColumn())
+                nonPKDefPartOfViewPK.add(baseColumn);
         }
-
-        this.columns = new Columns(partitionDefs, primaryKeyDefs, baseComplexColumns);
-
-        return partitionAllPrimaryKeyColumns && clusteringAllPrimaryKeyColumns;
+        this.baseNonPKColumnsInViewPK = nonPKDefPartOfViewPK;
     }
 
     /**
-     * Check to see if the update could possibly modify a view. Cases where the view may be updated are:
-     * <ul>
-     *     <li>View selects all columns</li>
-     *     <li>Update contains any range tombstones</li>
-     *     <li>Update touches one of the columns included in the view</li>
-     * </ul>
-     *
-     * If the update contains any range tombstones, there is a possibility that it will not touch a range that is
-     * currently included in the view.
-     *
-     * @return true if {@param partition} modifies a column included in the view
+     * The view column corresponding to the provided base column. This <b>can</b>
+     * return {@code null} if the column is denormalized in the view.
      */
-    public boolean updateAffectsView(AbstractBTreePartition partition)
+    public ColumnDefinition getViewColumn(ColumnDefinition baseColumn)
     {
-        ReadQuery selectQuery = getReadQuery();
-
-        if (!partition.metadata().cfId.equals(definition.baseTableId))
-            return false;
-
-        if (!selectQuery.selectsKey(partition.partitionKey()))
-            return false;
-
-        // If there are range tombstones, tombstones will also need to be generated for the view
-        // This requires a query of the base rows and generating tombstones for all of those values
-        if (!partition.deletionInfo().isLive())
-            return true;
-
-        // Check each row for deletion or update
-        for (Row row : partition)
-        {
-            if (!selectQuery.selectsClustering(partition.partitionKey(), row.clustering()))
-                continue;
-
-            if (includeAllColumns || !row.deletion().isLive())
-                return true;
-
-            if (row.primaryKeyLivenessInfo().isLive(FBUtilities.nowInSeconds()))
-                return true;
-
-            for (ColumnData data : row)
-            {
-                if (definition.metadata.getColumnDefinition(data.column().name) != null)
-                    return true;
-            }
-        }
-
-        return false;
-    }
-
-    /**
-     * Creates the clustering columns for the view based on the specified row and resolver policy
-     *
-     * @param temporalRow The current row
-     * @param resolver The policy to use when selecting versions of cells use
-     * @return The clustering object to use for the view
-     */
-    private Clustering viewClustering(TemporalRow temporalRow, TemporalRow.Resolver resolver)
-    {
-        CFMetaData viewCfm = definition.metadata;
-        int numViewClustering = viewCfm.clusteringColumns().size();
-        CBuilder clustering = CBuilder.create(viewCfm.comparator);
-        for (int i = 0; i < numViewClustering; i++)
-        {
-            ColumnDefinition definition = viewCfm.clusteringColumns().get(i);
-            clustering.add(temporalRow.clusteringValue(definition, resolver));
-        }
-
-        return clustering.build();
-    }
-
-    /**
-     * @return Mutation containing a range tombstone for a base partition key and TemporalRow.
-     */
-    private PartitionUpdate createTombstone(TemporalRow temporalRow,
-                                            DecoratedKey partitionKey,
-                                            Row.Deletion deletion,
-                                            TemporalRow.Resolver resolver,
-                                            int nowInSec)
-    {
-        CFMetaData viewCfm = definition.metadata;
-        Row.Builder builder = BTreeRow.unsortedBuilder(nowInSec);
-        builder.newRow(viewClustering(temporalRow, resolver));
-        builder.addRowDeletion(deletion);
-        return PartitionUpdate.singleRowUpdate(viewCfm, partitionKey, builder.build());
-    }
-
-    /**
-     * @return PartitionUpdate containing a complex tombstone for a TemporalRow, and the collection's column identifier.
-     */
-    private PartitionUpdate createComplexTombstone(TemporalRow temporalRow,
-                                                   DecoratedKey partitionKey,
-                                                   ColumnDefinition deletedColumn,
-                                                   DeletionTime deletionTime,
-                                                   TemporalRow.Resolver resolver,
-                                                   int nowInSec)
-    {
-        CFMetaData viewCfm = definition.metadata;
-        Row.Builder builder = BTreeRow.unsortedBuilder(nowInSec);
-        builder.newRow(viewClustering(temporalRow, resolver));
-        builder.addComplexDeletion(deletedColumn, deletionTime);
-        return PartitionUpdate.singleRowUpdate(viewCfm, partitionKey, builder.build());
+        return definition.metadata.getColumnDefinition(baseColumn.name);
     }
 
     /**
-     * @return View's DecoratedKey or null, if one of the view's primary key components has an invalid resolution from
-     *         the TemporalRow and its Resolver
+     * The base column corresponding to the provided view column. This should
+     * never return {@code null} since a view can't have its "own" columns.
      */
-    private DecoratedKey viewPartitionKey(TemporalRow temporalRow, TemporalRow.Resolver resolver)
+    public ColumnDefinition getBaseColumn(ColumnDefinition viewColumn)
     {
-        List<ColumnDefinition> partitionDefs = this.columns.partitionDefs;
-        Object[] partitionKey = new Object[partitionDefs.size()];
-
-        for (int i = 0; i < partitionKey.length; i++)
-        {
-            ByteBuffer value = temporalRow.clusteringValue(partitionDefs.get(i), resolver);
-
-            if (value == null)
-                return null;
-
-            partitionKey[i] = value;
-        }
-
-        CFMetaData metadata = definition.metadata;
-        return metadata.decorateKey(CFMetaData.serializePartitionKey(metadata
-                                                                     .getKeyValidatorAsClusteringComparator()
-                                                                     .make(partitionKey)));
+        ColumnDefinition baseColumn = baseCfs.metadata.getColumnDefinition(viewColumn.name);
+        assert baseColumn != null;
+        return baseColumn;
     }
 
     /**
-     * @return mutation which contains the tombstone for the referenced TemporalRow, or null if not necessary.
-     * TemporalRow's can reference at most one view row; there will be at most one row to be tombstoned, so only one
-     * mutation is necessary
-     */
-    private PartitionUpdate createRangeTombstoneForRow(TemporalRow temporalRow)
-    {
-        // Primary Key and Clustering columns do not generate tombstones
-        if (viewPKIncludesOnlyBasePKColumns)
-            return null;
-
-        boolean hasUpdate = false;
-        List<ColumnDefinition> primaryKeyDefs = this.columns.primaryKeyDefs;
-        for (ColumnDefinition viewPartitionKeys : primaryKeyDefs)
-        {
-            if (!viewPartitionKeys.isPrimaryKeyColumn() && temporalRow.clusteringValue(viewPartitionKeys, TemporalRow.oldValueIfUpdated) != null)
-                hasUpdate = true;
-        }
-
-        if (!hasUpdate)
-            return null;
-
-        TemporalRow.Resolver resolver = TemporalRow.earliest;
-        return createTombstone(temporalRow,
-                               viewPartitionKey(temporalRow, resolver),
-                               Row.Deletion.shadowable(new DeletionTime(temporalRow.viewClusteringTimestamp(), temporalRow.nowInSec)),
-                               resolver,
-                               temporalRow.nowInSec);
-    }
-
-    /**
-     * @return Mutation which is the transformed base table mutation for the view.
-     */
-    private PartitionUpdate createUpdatesForInserts(TemporalRow temporalRow)
-    {
-        TemporalRow.Resolver resolver = TemporalRow.latest;
-
-        DecoratedKey partitionKey = viewPartitionKey(temporalRow, resolver);
-        CFMetaData viewCfm = definition.metadata;
-
-        if (partitionKey == null)
-        {
-            // Not having a partition key means we aren't updating anything
-            return null;
-        }
-
-        Row.Builder regularBuilder = BTreeRow.unsortedBuilder(temporalRow.nowInSec);
-
-        CBuilder clustering = CBuilder.create(viewCfm.comparator);
-        for (int i = 0; i < viewCfm.clusteringColumns().size(); i++)
-        {
-            ColumnDefinition column = viewCfm.clusteringColumns().get(i);
-            ByteBuffer value = temporalRow.clusteringValue(column, resolver);
-
-            // handle single-column deletions correctly to avoid nulls in the view primary key
-            if (value == null)
-                return null;
-
-            clustering.add(value);
-        }
-        regularBuilder.newRow(clustering.build());
-        regularBuilder.addPrimaryKeyLivenessInfo(LivenessInfo.create(viewCfm,
-                                                                     temporalRow.viewClusteringTimestamp(),
-                                                                     temporalRow.viewClusteringTtl(),
-                                                                     temporalRow.viewClusteringLocalDeletionTime()));
-
-        for (ColumnDefinition columnDefinition : viewCfm.allColumns())
-        {
-            if (columnDefinition.isPrimaryKeyColumn())
-                continue;
-
-            for (Cell cell : temporalRow.values(columnDefinition, resolver))
-            {
-                regularBuilder.addCell(cell);
-            }
-        }
-
-        Row row = regularBuilder.build();
-
-        // although we check for empty rows in updateAppliesToView(), if there are any good rows in the PartitionUpdate,
-        // all rows in the partition will be processed, and we need to handle empty/non-live rows here (CASSANDRA-10614)
-        if (row.isEmpty())
-            return null;
-
-        return PartitionUpdate.singleRowUpdate(viewCfm, partitionKey, row);
-    }
-
-    /**
-     * @param partition Update which possibly contains deletion info for which to generate view tombstones.
-     * @return    View Tombstones which delete all of the rows which have been removed from the base table with
-     *            {@param partition}
-     */
-    private Collection<Mutation> createForDeletionInfo(TemporalRow.Set rowSet, AbstractBTreePartition partition)
-    {
-        final TemporalRow.Resolver resolver = TemporalRow.earliest;
-
-        DeletionInfo deletionInfo = partition.deletionInfo();
-
-        List<Mutation> mutations = new ArrayList<>();
-
-        // Check the complex columns to see if there are any which may have tombstones we need to create for the view
-        if (!columns.baseComplexColumns.isEmpty())
-        {
-            for (Row row : partition)
-            {
-                if (!row.hasComplexDeletion())
-                    continue;
-
-                TemporalRow temporalRow = rowSet.getClustering(row.clustering());
-
-                assert temporalRow != null;
-
-                for (ColumnDefinition definition : columns.baseComplexColumns)
-                {
-                    ComplexColumnData columnData = row.getComplexColumnData(definition);
-
-                    if (columnData != null)
-                    {
-                        DeletionTime time = columnData.complexDeletion();
-                        if (!time.isLive())
-                        {
-                            DecoratedKey targetKey = viewPartitionKey(temporalRow, resolver);
-                            if (targetKey != null)
-                                mutations.add(new Mutation(createComplexTombstone(temporalRow, targetKey, definition, time, resolver, temporalRow.nowInSec)));
-                        }
-                    }
-                }
-            }
-        }
-
-        ReadCommand command = null;
-
-        if (!deletionInfo.isLive())
-        {
-            // We have to generate tombstones for all of the affected rows, but we don't have the information in order
-            // to create them. This requires that we perform a read for the entire range that is being tombstoned, and
-            // generate a tombstone for each. This may be slow, because a single range tombstone can cover up to an
-            // entire partition of data which is not distributed on a single partition node.
-            DecoratedKey dk = rowSet.dk;
-
-            if (!deletionInfo.getPartitionDeletion().isLive())
-            {
-                command = getSelectStatement().internalReadForView(dk, rowSet.nowInSec);
-            }
-            else
-            {
-                SinglePartitionSliceBuilder builder = new SinglePartitionSliceBuilder(baseCfs, dk);
-                Iterator<RangeTombstone> tombstones = deletionInfo.rangeIterator(false);
-                while (tombstones.hasNext())
-                {
-                    RangeTombstone tombstone = tombstones.next();
-
-                    builder.addSlice(tombstone.deletedSlice());
-                }
-
-                command = builder.build();
-            }
-        }
-
-        if (command == null)
-        {
-            ReadQuery selectQuery = getReadQuery();
-            SinglePartitionSliceBuilder builder = null;
-            for (Row row : partition)
-            {
-                if (!row.deletion().isLive())
-                {
-                    if (!selectQuery.selectsClustering(rowSet.dk, row.clustering()))
-                        continue;
-
-                    if (builder == null)
-                        builder = new SinglePartitionSliceBuilder(baseCfs, rowSet.dk);
-                    builder.addSlice(Slice.make(row.clustering()));
-                }
-            }
-
-            if (builder != null)
-                command = builder.build();
-        }
-
-        if (command != null)
-        {
-            ReadQuery selectQuery = getReadQuery();
-            assert selectQuery.selectsKey(rowSet.dk);
-
-            // We may have already done this work for another MV update so check
-            if (!rowSet.hasTombstonedExisting())
-            {
-                QueryPager pager = command.getPager(null, Server.CURRENT_VERSION);
-
-                // Add all of the rows which were recovered from the query to the row set
-                while (!pager.isExhausted())
-                {
-                    try (ReadOrderGroup orderGroup = pager.startOrderGroup();
-                         PartitionIterator iter = pager.fetchPageInternal(128, orderGroup))
-                    {
-                        if (!iter.hasNext())
-                            break;
-
-                        try (RowIterator rowIterator = iter.next())
-                        {
-                            while (rowIterator.hasNext())
-                            {
-                                Row row = rowIterator.next();
-                                if (selectQuery.selectsClustering(rowSet.dk, row.clustering()))
-                                    rowSet.addRow(row, false);
-                            }
-                        }
-                    }
-                }
-
-                //Incase we fetched nothing, avoid re checking on another MV update
-                rowSet.setTombstonedExisting();
-            }
-
-            // If the temporal row has been deleted by the deletion info, we generate the corresponding range tombstone
-            // for the view.
-            for (TemporalRow temporalRow : rowSet)
-            {
-                DeletionTime deletionTime = temporalRow.deletionTime(partition);
-                if (!deletionTime.isLive())
-                {
-                    DecoratedKey value = viewPartitionKey(temporalRow, resolver);
-                    if (value != null)
-                    {
-                        PartitionUpdate update = createTombstone(temporalRow, value, Row.Deletion.regular(deletionTime), resolver, temporalRow.nowInSec);
-                        if (update != null)
-                            mutations.add(new Mutation(update));
-                    }
-                }
-            }
-        }
-
-        return !mutations.isEmpty() ? mutations : null;
-    }
-
-    /**
-     * Read and update temporal rows in the set which have corresponding values stored on the local node
+     * Whether the view might be affected by the provided update.
+     * <p>
+     * Note that having this method return {@code true} is not an absolute guarantee that the view will be
+     * updated, just that it most likely will, but a {@code false} return guarantees it won't be affected).
+     *
+     * @param partitionKey the partition key that is updated.
+     * @param update the update being applied.
+     * @return {@code false} if we can guarantee that inserting {@code update} for key {@code partitionKey}
+     * won't affect the view in any way, {@code true} otherwise.
      */
-    private void readLocalRows(TemporalRow.Set rowSet)
+    public boolean mayBeAffectedBy(DecoratedKey partitionKey, Row update)
     {
-        long start = System.currentTimeMillis();
-        SinglePartitionSliceBuilder builder = new SinglePartitionSliceBuilder(baseCfs, rowSet.dk);
-
-        for (TemporalRow temporalRow : rowSet)
-            builder.addSlice(temporalRow.baseSlice());
+        // We can guarantee that the view won't be affected if:
+        //  - the clustering is excluded by the view filter (note that this isn't true of the filter on regular columns:
+        //    even if an update don't match a view condition on a regular column, that update can still invalidate an pre-existing
+        //    entry).
+        //  - or the update don't modify any of the columns impacting the view (where "impacting" the view means that column is
+        //    neither included in the view, nor used by the view filter).
+        if (!getReadQuery().selectsClustering(partitionKey, update.clustering()))
+            return false;
 
-        QueryPager pager = builder.build().getPager(null, Server.CURRENT_VERSION);
+        // 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;
 
-        while (!pager.isExhausted())
+        for (ColumnData data : update)
         {
-            try (ReadOrderGroup orderGroup = pager.startOrderGroup();
-                 PartitionIterator iter = pager.fetchPageInternal(128, orderGroup))
-            {
-                while (iter.hasNext())
-                {
-                    try (RowIterator rows = iter.next())
-                    {
-                        while (rows.hasNext())
-                        {
-                            rowSet.addRow(rows.next(), false);
-                        }
-                    }
-                }
-            }
+            if (definition.metadata.getColumnDefinition(data.column().name) != null)
+                return true;
         }
-        baseCfs.metric.viewReadTime.update(System.currentTimeMillis() - start, TimeUnit.MILLISECONDS);
-    }
-
-    /**
-     * @return Set of rows which are contained in the partition update {@param partition}
-     */
-    private TemporalRow.Set separateRows(AbstractBTreePartition partition, Set<ColumnIdentifier> viewPrimaryKeyCols)
-    {
-
-        TemporalRow.Set rowSet = new TemporalRow.Set(baseCfs, viewPrimaryKeyCols, partition.partitionKey().getKey());
-
-        for (Row row : partition)
-            rowSet.addRow(row, true);
-
-        return rowSet;
+        return false;
     }
 
     /**
-     * Splits the partition update up and adds the existing state to each row.
-     * This data can be reused for multiple MV updates on the same base table
+     * Whether a given base row matches the view filter (and thus if is should have a corresponding entry).
+     * <p>
+     * Note that this differs from {@link #mayBeAffectedBy} in that the provide row <b>must</b> be the current
+     * state of the base row, not just some updates to it. This method also has no false positive: a base
+     * row either do or don't match the view filter.
      *
-     * @param partition the mutation
-     * @param isBuilding If the view is currently being built, we do not query the values which are already stored,
-     *                   since all of the update will already be present in the base table.
-     * @return The set of temoral rows contained in this update
+     * @param partitionKey the partition key that is updated.
+     * @param baseRow the current state of a particular base row.
+     * @param nowInSec the current time in seconds (to decide what is live and what isn't).
+     * @return {@code true} if {@code baseRow} matches the view filters, {@code false} otherwise.
      */
-    public TemporalRow.Set getTemporalRowSet(AbstractBTreePartition partition, TemporalRow.Set existing, boolean isBuilding)
+    public boolean matchesViewFilter(DecoratedKey partitionKey, Row baseRow, int nowInSec)
     {
-        if (!updateAffectsView(partition))
-            return existing;
-
-        Set<ColumnIdentifier> columns = new HashSet<>(this.columns.primaryKeyDefs.size());
-        for (ColumnDefinition def : this.columns.primaryKeyDefs)
-            columns.add(def.name);
-
-        TemporalRow.Set rowSet;
-        if (existing == null)
-        {
-            rowSet = separateRows(partition, columns);
-
-            // If we are building the view, we do not want to add old values; they will always be the same
-            if (!isBuilding)
-                readLocalRows(rowSet);
-        }
-        else
-        {
-            rowSet = existing.withNewViewPrimaryKey(columns);
-        }
-
-        return rowSet;
+        return getReadQuery().selectsClustering(partitionKey, baseRow.clustering())
+            && getSelectStatement().rowFilterForInternalCalls().isSatisfiedBy(baseCfs.metadata, partitionKey, baseRow, nowInSec);
     }
 
     /**
@@ -656,61 +212,6 @@ public class View
         return query;
     }
 
-    /**
-     * @param isBuilding If the view is currently being built, we do not query the values which are already stored,
-     *                   since all of the update will already be present in the base table.
-     * @return View mutations which represent the changes necessary as long as previously created mutations for the view
-     *         have been applied successfully. This is based solely on the changes that are necessary given the current
-     *         state of the base table and the newly applying partition data.
-     */
-    public Collection<Mutation> createMutations(AbstractBTreePartition partition, TemporalRow.Set rowSet, boolean isBuilding)
-    {
-        if (!updateAffectsView(partition))
-            return null;
-
-        ReadQuery selectQuery = getReadQuery();
-        Collection<Mutation> mutations = null;
-        for (TemporalRow temporalRow : rowSet)
-        {
-            // In updateAffectsView, we check the partition to see if there is at least one row that matches the
-            // filters and is live.  If there is more than one row in the partition, we need to re-check each one
-            // individually.
-            if (partition.rowCount() != 1 && !selectQuery.selectsClustering(partition.partitionKey(), temporalRow.baseClustering()))
-                continue;
-
-            // If we are building, there is no need to check for partition tombstones; those values will not be present
-            // in the partition data
-            if (!isBuilding)
-            {
-                PartitionUpdate partitionTombstone = createRangeTombstoneForRow(temporalRow);
-                if (partitionTombstone != null)
-                {
-                    if (mutations == null) mutations = new LinkedList<>();
-                    mutations.add(new Mutation(partitionTombstone));
-                }
-            }
-
-            PartitionUpdate insert = createUpdatesForInserts(temporalRow);
-            if (insert != null)
-            {
-                if (mutations == null) mutations = new LinkedList<>();
-                mutations.add(new Mutation(insert));
-            }
-        }
-
-        if (!isBuilding)
-        {
-            Collection<Mutation> deletion = createForDeletionInfo(rowSet, partition);
-            if (deletion != null && !deletion.isEmpty())
-            {
-                if (mutations == null) mutations = new LinkedList<>();
-                mutations.addAll(deletion);
-            }
-        }
-
-        return mutations;
-    }
-
     public synchronized void build()
     {
         if (this.builder != null)

http://git-wip-us.apache.org/repos/asf/cassandra/blob/86ba2274/src/java/org/apache/cassandra/db/view/ViewBuilder.java
----------------------------------------------------------------------
diff --git a/src/java/org/apache/cassandra/db/view/ViewBuilder.java b/src/java/org/apache/cassandra/db/view/ViewBuilder.java
index 35b023b..b2b409b 100644
--- a/src/java/org/apache/cassandra/db/view/ViewBuilder.java
+++ b/src/java/org/apache/cassandra/db/view/ViewBuilder.java
@@ -19,6 +19,7 @@
 package org.apache.cassandra.db.view;
 
 import java.util.Collection;
+import java.util.Collections;
 import java.util.UUID;
 import java.util.concurrent.TimeUnit;
 import java.util.concurrent.atomic.AtomicLong;
@@ -35,9 +36,8 @@ import org.apache.cassandra.db.compaction.CompactionInfo;
 import org.apache.cassandra.db.compaction.CompactionManager;
 import org.apache.cassandra.db.compaction.OperationType;
 import org.apache.cassandra.db.lifecycle.SSTableSet;
-import org.apache.cassandra.db.partitions.FilteredPartition;
-import org.apache.cassandra.db.partitions.PartitionIterator;
-import org.apache.cassandra.db.rows.RowIterator;
+import org.apache.cassandra.db.partitions.*;
+import org.apache.cassandra.db.rows.*;
 import org.apache.cassandra.dht.Range;
 import org.apache.cassandra.dht.Token;
 import org.apache.cassandra.io.sstable.ReducingKeyIterator;
@@ -76,28 +76,22 @@ public class ViewBuilder extends CompactionInfo.Holder
         if (!selectQuery.selectsKey(key))
             return;
 
-        QueryPager pager = view.getSelectStatement().internalReadForView(key, FBUtilities.nowInSeconds()).getPager(null, Server.CURRENT_VERSION);
+        int nowInSec = FBUtilities.nowInSeconds();
+        SinglePartitionReadCommand command = view.getSelectStatement().internalReadForView(key, nowInSec);
 
-        while (!pager.isExhausted())
+        // We're rebuilding everything from what's on disk, so we read everything, consider that as new updates
+        // and pretend that there is nothing pre-existing.
+        UnfilteredRowIterator empty = UnfilteredRowIterators.noRowsIterator(baseCfs.metadata, key, Rows.EMPTY_STATIC_ROW, DeletionTime.LIVE, false);
+
+        Collection<Mutation> mutations;
+        try (ReadOrderGroup orderGroup = command.startOrderGroup();
+             UnfilteredRowIterator data = UnfilteredPartitionIterators.getOnlyElement(command.executeLocally(orderGroup), command))
         {
-           try (ReadOrderGroup orderGroup = pager.startOrderGroup();
-                PartitionIterator partitionIterator = pager.fetchPageInternal(128, orderGroup))
-           {
-               if (!partitionIterator.hasNext())
-                   return;
-
-               try (RowIterator rowIterator = partitionIterator.next())
-               {
-                   FilteredPartition partition = FilteredPartition.create(rowIterator);
-                   TemporalRow.Set temporalRows = view.getTemporalRowSet(partition, null, true);
-
-                   Collection<Mutation> mutations = view.createMutations(partition, temporalRows, true);
-
-                   if (mutations != null)
-                       StorageProxy.mutateMV(key.getKey(), mutations, true, noBase);
-               }
-           }
+            mutations = baseCfs.keyspace.viewManager.forTable(baseCfs.metadata).generateViewUpdates(Collections.singleton(view), data, empty, nowInSec);
         }
+
+        if (!mutations.isEmpty())
+            StorageProxy.mutateMV(key.getKey(), mutations, true, noBase);
     }
 
     public void run()

http://git-wip-us.apache.org/repos/asf/cassandra/blob/86ba2274/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 9fe0544..fd04b97 100644
--- a/src/java/org/apache/cassandra/db/view/ViewManager.java
+++ b/src/java/org/apache/cassandra/db/view/ViewManager.java
@@ -19,23 +19,21 @@ package org.apache.cassandra.db.view;
 
 import java.nio.ByteBuffer;
 import java.util.*;
-import java.util.concurrent.ConcurrentNavigableMap;
-import java.util.concurrent.ConcurrentSkipListMap;
-import java.util.concurrent.atomic.AtomicLong;
+import java.util.concurrent.ConcurrentMap;
+import java.util.concurrent.ConcurrentHashMap;
 import java.util.concurrent.locks.Lock;
 
-import com.google.common.collect.Lists;
 import com.google.common.util.concurrent.Striped;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
 
+import org.apache.cassandra.config.CFMetaData;
 import org.apache.cassandra.config.DatabaseDescriptor;
 import org.apache.cassandra.config.ViewDefinition;
 import org.apache.cassandra.db.*;
-import org.apache.cassandra.db.commitlog.ReplayPosition;
-import org.apache.cassandra.db.partitions.PartitionUpdate;
-import org.apache.cassandra.dht.Token;
-import org.apache.cassandra.service.StorageProxy;
-import org.slf4j.Logger;
-import org.slf4j.LoggerFactory;
+import org.apache.cassandra.db.rows.*;
+import org.apache.cassandra.db.partitions.*;
+
 
 /**
  * Manages {@link View}'s for a single {@link ColumnFamilyStore}. All of the views for that table are created when this
@@ -45,110 +43,30 @@ import org.slf4j.LoggerFactory;
  * any views {@link ViewManager#updatesAffectView(Collection, boolean)}, provide locks to prevent multiple
  * updates from creating incoherent updates in the view {@link ViewManager#acquireLockFor(ByteBuffer)}, and
  * to affect change on the view.
+ *
+ * TODO: I think we can get rid of that class. For addition/removal of view by names, we could move it Keyspace. And we
+ * not sure it's even worth keeping viewsByName as none of the related operation are performance sensitive so we could
+ * find the view by iterating over the CFStore.viewManager directly.
+ * For the lock, it could move to Keyspace too, but I don't remmenber why it has to be at the keyspace level and if it
+ * can be at the table level, maybe that's where it should be.
  */
 public class ViewManager
 {
     private static final Logger logger = LoggerFactory.getLogger(ViewManager.class);
 
-    public class ForStore
-    {
-        private final ConcurrentNavigableMap<String, View> viewsByName;
-
-        public ForStore()
-        {
-            this.viewsByName = new ConcurrentSkipListMap<>();
-        }
-
-        public Iterable<View> allViews()
-        {
-            return viewsByName.values();
-        }
-
-        public Iterable<ColumnFamilyStore> allViewsCfs()
-        {
-            List<ColumnFamilyStore> viewColumnFamilies = new ArrayList<>();
-            for (View view : allViews())
-                viewColumnFamilies.add(keyspace.getColumnFamilyStore(view.getDefinition().viewName));
-            return viewColumnFamilies;
-        }
-
-        public void forceBlockingFlush()
-        {
-            for (ColumnFamilyStore viewCfs : allViewsCfs())
-                viewCfs.forceBlockingFlush();
-        }
-
-        public void dumpMemtables()
-        {
-            for (ColumnFamilyStore viewCfs : allViewsCfs())
-                viewCfs.dumpMemtable();
-        }
-
-        public void truncateBlocking(long truncatedAt)
-        {
-            for (ColumnFamilyStore viewCfs : allViewsCfs())
-            {
-                ReplayPosition replayAfter = viewCfs.discardSSTables(truncatedAt);
-                SystemKeyspace.saveTruncationRecord(viewCfs, truncatedAt, replayAfter);
-            }
-        }
-
-        public void addView(View view)
-        {
-            viewsByName.put(view.name, view);
-        }
-
-        public void removeView(String name)
-        {
-            viewsByName.remove(name);
-        }
-    }
-
     private static final Striped<Lock> LOCKS = Striped.lazyWeakLock(DatabaseDescriptor.getConcurrentViewWriters() * 1024);
 
     private static final boolean enableCoordinatorBatchlog = Boolean.getBoolean("cassandra.mv_enable_coordinator_batchlog");
 
-    private final ConcurrentNavigableMap<UUID, ForStore> viewManagersByStore;
-    private final ConcurrentNavigableMap<String, View> viewsByName;
+    private final ConcurrentMap<String, View> viewsByName = new ConcurrentHashMap<>();
+    private final ConcurrentMap<UUID, TableViews> viewsByBaseTable = new ConcurrentHashMap<>();
     private final Keyspace keyspace;
 
     public ViewManager(Keyspace keyspace)
     {
-        this.viewManagersByStore = new ConcurrentSkipListMap<>();
-        this.viewsByName = new ConcurrentSkipListMap<>();
         this.keyspace = keyspace;
     }
 
-    /**
-     * Calculates and pushes updates to the views replicas. The replicas are determined by
-     * {@link ViewUtils#getViewNaturalEndpoint(String, Token, Token)}.
-     */
-    public void pushViewReplicaUpdates(PartitionUpdate update, boolean writeCommitLog, AtomicLong baseComplete)
-    {
-        List<Mutation> mutations = null;
-        TemporalRow.Set temporalRows = null;
-        for (Map.Entry<String, View> view : viewsByName.entrySet())
-        {
-            // Make sure that we only get mutations from views which are affected since the set includes all views for a
-            // keyspace. This will prevent calling getTemporalRowSet for the wrong base table.
-            if (view.getValue().updateAffectsView(update))
-            {
-                temporalRows = view.getValue().getTemporalRowSet(update, temporalRows, false);
-
-                Collection<Mutation> viewMutations = view.getValue().createMutations(update, temporalRows, false);
-                if (viewMutations != null && !viewMutations.isEmpty())
-                {
-                    if (mutations == null)
-                        mutations = Lists.newLinkedList();
-                    mutations.addAll(viewMutations);
-                }
-            }
-        }
-
-        if (mutations != null)
-            StorageProxy.mutateMV(update.partitionKey().getKey(), mutations, writeCommitLog, baseComplete);
-    }
-
     public boolean updatesAffectView(Collection<? extends IMutation> mutations, boolean coordinatorBatchlog)
     {
         if (coordinatorBatchlog && !enableCoordinatorBatchlog)
@@ -156,25 +74,22 @@ public class ViewManager
 
         for (IMutation mutation : mutations)
         {
-            for (PartitionUpdate cf : mutation.getPartitionUpdates())
+            for (PartitionUpdate update : mutation.getPartitionUpdates())
             {
-                assert keyspace.getName().equals(cf.metadata().ksName);
+                assert keyspace.getName().equals(update.metadata().ksName);
 
                 if (coordinatorBatchlog && keyspace.getReplicationStrategy().getReplicationFactor() == 1)
                     continue;
 
-                for (View view : allViews())
-                {
-                    if (view.updateAffectsView(cf))
-                        return true;
-                }
+                if (!forTable(update.metadata()).updatedViews(update).isEmpty())
+                    return true;
             }
         }
 
         return false;
     }
 
-    public Iterable<View> allViews()
+    private Iterable<View> allViews()
     {
         return viewsByName.values();
     }
@@ -222,7 +137,7 @@ public class ViewManager
     public void addView(ViewDefinition definition)
     {
         View view = new View(definition, keyspace.getColumnFamilyStore(definition.baseTableId));
-        forTable(view.getDefinition().baseTableId).addView(view);
+        forTable(view.getDefinition().baseTableMetadata()).add(view);
         viewsByName.put(definition.viewName, view);
     }
 
@@ -233,7 +148,7 @@ public class ViewManager
         if (view == null)
             return;
 
-        forTable(view.getDefinition().baseTableId).removeView(name);
+        forTable(view.getDefinition().baseTableMetadata()).removeByName(name);
         SystemKeyspace.setViewRemoved(keyspace.getName(), view.name);
     }
 
@@ -243,17 +158,18 @@ public class ViewManager
             view.build();
     }
 
-    public ForStore forTable(UUID baseId)
+    public TableViews forTable(CFMetaData metadata)
     {
-        ForStore forStore = viewManagersByStore.get(baseId);
-        if (forStore == null)
+        UUID baseId = metadata.cfId;
+        TableViews views = viewsByBaseTable.get(baseId);
+        if (views == null)
         {
-            forStore = new ForStore();
-            ForStore previous = viewManagersByStore.put(baseId, forStore);
+            views = new TableViews(metadata);
+            TableViews previous = viewsByBaseTable.putIfAbsent(baseId, views);
             if (previous != null)
-                forStore = previous;
+                views = previous;
         }
-        return forStore;
+        return views;
     }
 
     public static Lock acquireLockFor(ByteBuffer key)

http://git-wip-us.apache.org/repos/asf/cassandra/blob/86ba2274/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
new file mode 100644
index 0000000..af025cb
--- /dev/null
+++ b/src/java/org/apache/cassandra/db/view/ViewUpdateGenerator.java
@@ -0,0 +1,549 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.cassandra.db.view;
+
+import java.nio.ByteBuffer;
+import java.util.*;
+
+import com.google.common.collect.Iterators;
+import com.google.common.collect.PeekingIterator;
+
+import org.apache.cassandra.config.CFMetaData;
+import org.apache.cassandra.config.ColumnDefinition;
+import org.apache.cassandra.config.ViewDefinition;
+import org.apache.cassandra.db.*;
+import org.apache.cassandra.db.rows.*;
+import org.apache.cassandra.db.partitions.*;
+import org.apache.cassandra.db.marshal.AbstractType;
+import org.apache.cassandra.db.marshal.CompositeType;
+
+/**
+ * Creates the updates to apply to a view given the existing rows in the base
+ * table and the updates that we're applying to them (this handles updates
+ * on a single partition only).
+ *
+ * This class is used by passing the updates made to the base table to
+ * {@link #addBaseTableUpdate} and calling {@link #generateViewUpdates} once all updates have
+ * been handled to get the resulting view mutations.
+ */
+public class ViewUpdateGenerator
+{
+    private final View view;
+    private final int nowInSec;
+
+    private final CFMetaData baseMetadata;
+    private final DecoratedKey baseDecoratedKey;
+    private final ByteBuffer[] basePartitionKey;
+
+    private final CFMetaData viewMetadata;
+
+    private final Map<DecoratedKey, PartitionUpdate> updates = new HashMap<>();
+
+    // Reused internally to build a new entry
+    private final ByteBuffer[] currentViewEntryPartitionKey;
+    private final Row.Builder currentViewEntryBuilder;
+
+    /**
+     * The type of type update action to perform to the view for a given base table
+     * update.
+     */
+    private enum UpdateAction
+    {
+        NONE,            // There was no view entry and none should be added
+        NEW_ENTRY,       // There was no entry but there is one post-update
+        DELETE_OLD,      // There was an entry but there is nothing after update
+        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}.
+     *
+     * @param view the view for which this will be building updates for.
+     * @param basePartitionKey the partition key for the base table partition for which
+     * we'll handle updates for.
+     * @param nowInSec the current time in seconds. Used to decide if data are live or not
+     * and as base reference for new deletions.
+     */
+    public ViewUpdateGenerator(View view, DecoratedKey basePartitionKey, int nowInSec)
+    {
+        this.view = view;
+        this.nowInSec = nowInSec;
+
+        this.baseMetadata = view.getDefinition().baseTableMetadata();
+        this.baseDecoratedKey = basePartitionKey;
+        this.basePartitionKey = extractKeyComponents(basePartitionKey, baseMetadata.getKeyValidator());
+
+        this.viewMetadata = view.getDefinition().metadata;
+
+        this.currentViewEntryPartitionKey = new ByteBuffer[viewMetadata.partitionKeyColumns().size()];
+        this.currentViewEntryBuilder = BTreeRow.sortedBuilder();
+    }
+
+    private static ByteBuffer[] extractKeyComponents(DecoratedKey partitionKey, AbstractType<?> type)
+    {
+        return type instanceof CompositeType
+             ? ((CompositeType)type).split(partitionKey.getKey())
+             : new ByteBuffer[]{ partitionKey.getKey() };
+    }
+
+    /**
+     * Adds to this generator the updates to be made to the view given a base table row
+     * before and after an update.
+     *
+     * @param existingBaseRow the base table row as it is before an update.
+     * @param mergedBaseRow the base table row after the update is applied (note that
+     * this is not just the new update, but rather the resulting row).
+     */
+    public void addBaseTableUpdate(Row existingBaseRow, Row mergedBaseRow)
+    {
+        switch (updateAction(existingBaseRow, mergedBaseRow))
+        {
+            case NONE:
+                return;
+            case NEW_ENTRY:
+                createEntry(mergedBaseRow);
+                return;
+            case DELETE_OLD:
+                deleteOldEntry(existingBaseRow);
+                return;
+            case UPDATE_EXISTING:
+                updateEntry(existingBaseRow, mergedBaseRow);
+                return;
+            case SWITCH_ENTRY:
+                createEntry(mergedBaseRow);
+                deleteOldEntry(existingBaseRow);
+                return;
+        }
+    }
+
+    /**
+     * Returns the updates that needs to be done to the view given the base table updates
+     * passed to {@link #generateViewMutations}.
+     *
+     * @return the updates to do to the view.
+     */
+    public Collection<PartitionUpdate> generateViewUpdates()
+    {
+        return updates.values();
+    }
+
+    /**
+     * Compute which type of action needs to be performed to the view for a base table row
+     * before and after an update.
+     */
+    private UpdateAction updateAction(Row existingBaseRow, Row mergedBaseRow)
+    {
+        // Having existing empty is useful, it just means we'll insert a brand new entry for mergedBaseRow,
+        // but if we have no update at all, we shouldn't get there.
+        assert !mergedBaseRow.isEmpty();
+
+        // Note that none of the base PK columns will differ since we're intrinsically dealing
+        // with the same base row. So we have to check 3 things:
+        //   1) that the clustering doesn't have a null, which can happen for compact tables. If that's the case,
+        //      there is no corresponding entries.
+        //   2) if there is a column not part of the base PK in the view PK, whether it is changed by the update.
+        //   3) whether mergedBaseRow actually match the view SELECT filter
+
+        if (baseMetadata.isCompactTable())
+        {
+            Clustering clustering = mergedBaseRow.clustering();
+            for (int i = 0; i < clustering.size(); i++)
+            {
+                if (clustering.get(i) == null)
+                    return UpdateAction.NONE;
+            }
+        }
+
+        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.
+
+            // Note that we allow existingBaseRow to be null and treat it as empty (see MultiViewUpdateBuilder.generateViewsMutations).
+            boolean existingHasLiveData = existingBaseRow != null && existingBaseRow.hasLiveData(nowInSec);
+            boolean mergedHasLiveData = mergedBaseRow.hasLiveData(nowInSec);
+            return existingHasLiveData
+                 ? (mergedHasLiveData ? UpdateAction.UPDATE_EXISTING : UpdateAction.DELETE_OLD)
+                 : (mergedHasLiveData ? UpdateAction.NEW_ENTRY : UpdateAction.NONE);
+        }
+
+        ColumnDefinition baseColumn = view.baseNonPKColumnsInViewPK.get(0);
+        assert !baseColumn.isComplex() : "A complex column couldn't be part of the view PK";
+        Cell before = existingBaseRow == null ? null : existingBaseRow.getCell(baseColumn);
+        Cell after = mergedBaseRow.getCell(baseColumn);
+
+        // If the update didn't modified this column, the cells will be the same object so it's worth checking
+        if (before == after)
+            return before == null ? UpdateAction.NONE : UpdateAction.UPDATE_EXISTING;
+
+        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
+             : UpdateAction.SWITCH_ENTRY;
+    }
+
+    private boolean matchesViewFilter(Row baseRow)
+    {
+        return view.matchesViewFilter(baseDecoratedKey, baseRow, nowInSec);
+    }
+
+    private boolean isLive(Cell cell)
+    {
+        return cell != null && cell.isLive(nowInSec);
+    }
+
+    /**
+     * Creates a view entry corresponding to the provided base row.
+     * <p>
+     * This method checks that the base row does match the view filter before applying it.
+     */
+    private void createEntry(Row baseRow)
+    {
+        // Before create a new entry, make sure it matches the view filter
+        if (!matchesViewFilter(baseRow))
+            return;
+
+        startNewUpdate(baseRow);
+        currentViewEntryBuilder.addPrimaryKeyLivenessInfo(computeLivenessInfoForEntry(baseRow));
+        currentViewEntryBuilder.addRowDeletion(baseRow.deletion());
+
+        for (ColumnData data : baseRow)
+        {
+            ColumnDefinition viewColumn = view.getViewColumn(data.column());
+            // If that base table column is not denormalized in the view, we had nothing to do.
+            // Alose, if it's part of the view PK it's already been taken into account in the clustering.
+            if (viewColumn == null || viewColumn.isPrimaryKeyColumn())
+                continue;
+
+            addColumnData(viewColumn, data);
+        }
+
+        submitUpdate();
+    }
+
+    /**
+     * Creates the updates to apply to the existing view entry given the base table row before
+     * and after the update, assuming that the update hasn't changed to which view entry the
+     * row correspond (that is, we know the columns composing the view PK haven't changed).
+     * <p>
+     * This method checks that the base row (before and after) does match the view filter before
+     * applying anything.
+     */
+    private void updateEntry(Row existingBaseRow, Row mergedBaseRow)
+    {
+        // While we know existingBaseRow and mergedBaseRow are corresponding to the same view entry,
+        // they may not match the view filter.
+        if (!matchesViewFilter(existingBaseRow))
+        {
+            createEntry(mergedBaseRow);
+            return;
+        }
+        if (!matchesViewFilter(mergedBaseRow))
+        {
+            deleteOldEntryInternal(existingBaseRow);
+            return;
+        }
+
+        startNewUpdate(mergedBaseRow);
+
+        // In theory, it may be the PK liveness and row deletion hasn't been change by the update
+        // and we could condition the 2 additions below. In practice though, it's as fast (if not
+        // faster) to compute those info than to check if they have changed so we keep it simple.
+        currentViewEntryBuilder.addPrimaryKeyLivenessInfo(computeLivenessInfoForEntry(mergedBaseRow));
+        currentViewEntryBuilder.addRowDeletion(mergedBaseRow.deletion());
+
+        // 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
+        // mergedBaseRow (note that including more cells than we strictly should isn't a problem
+        // for correction, so even if the code change and pointer equality don't work anymore, it'll
+        // only a slightly inefficiency which we can fix then).
+        // Note: we could alternatively use Rows.diff() for this, but because it is a bit more generic
+        // than what we need here, it's also a bit less efficient (it allocates more in particular),
+        // and this might be called a lot of time for view updates. So, given that this is not a whole
+        // lot of code anyway, it's probably doing the diff manually.
+        PeekingIterator<ColumnData> existingIter = Iterators.peekingIterator(existingBaseRow.iterator());
+        for (ColumnData mergedData : mergedBaseRow)
+        {
+            ColumnDefinition baseColumn = mergedData.column();
+            ColumnDefinition viewColumn = view.getViewColumn(baseColumn);
+            // If that base table column is not denormalized in the view, we had nothing to do.
+            // Alose, if it's part of the view PK it's already been taken into account in the clustering.
+            if (viewColumn == null || viewColumn.isPrimaryKeyColumn())
+                continue;
+
+            ColumnData existingData = null;
+            // Find if there is data for that column in the existing row
+            while (existingIter.hasNext())
+            {
+                int cmp = baseColumn.compareTo(existingIter.peek().column());
+                if (cmp < 0)
+                    break;
+
+                ColumnData next = existingIter.next();
+                if (cmp == 0)
+                {
+                    existingData = next;
+                    break;
+                }
+            }
+
+            if (existingData == null)
+            {
+                addColumnData(viewColumn, mergedData);
+                continue;
+            }
+
+            if (mergedData == existingData)
+                continue;
+
+            if (baseColumn.isComplex())
+            {
+                ComplexColumnData mergedComplexData = (ComplexColumnData)mergedData;
+                ComplexColumnData existingComplexData = (ComplexColumnData)existingData;
+                if (mergedComplexData.complexDeletion().supersedes(existingComplexData.complexDeletion()))
+                    currentViewEntryBuilder.addComplexDeletion(viewColumn, mergedComplexData.complexDeletion());
+
+                PeekingIterator<Cell> existingCells = Iterators.peekingIterator(existingComplexData.iterator());
+                for (Cell mergedCell : mergedComplexData)
+                {
+                    Cell existingCell = null;
+                    // Find if there is corresponding cell in the existing row
+                    while (existingCells.hasNext())
+                    {
+                        int cmp = baseColumn.cellPathComparator().compare(mergedCell.path(), existingCells.peek().path());
+                        if (cmp > 0)
+                            break;
+
+                        Cell next = existingCells.next();
+                        if (cmp == 0)
+                        {
+                            existingCell = next;
+                            break;
+                        }
+                    }
+
+                    if (mergedCell != existingCell)
+                        addCell(viewColumn, mergedCell);
+                }
+            }
+            else
+            {
+                // Note that we've already eliminated the case where merged == existing
+                addCell(viewColumn, (Cell)mergedData);
+            }
+        }
+
+        submitUpdate();
+    }
+
+    /**
+     * Deletes the view entry corresponding to the provided base row.
+     * <p>
+     * This method checks that the base row does match the view filter before bothering.
+     */
+    private void deleteOldEntry(Row existingBaseRow)
+    {
+        // 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);
+    }
+
+    private void deleteOldEntryInternal(Row existingBaseRow)
+    {
+        startNewUpdate(existingBaseRow);
+        DeletionTime dt = new DeletionTime(computeTimestampForEntryDeletion(existingBaseRow), nowInSec);
+        currentViewEntryBuilder.addRowDeletion(Row.Deletion.shadowable(dt));
+        submitUpdate();
+    }
+
+    /**
+     * Computes the partition key and clustering for a new view entry, and setup the internal
+     * row builder for the new row.
+     *
+     * This assumes that there is corresponding entry, i.e. no values for the partition key and
+     * clustering are null (since we have eliminated that case through updateAction).
+     */
+    private void startNewUpdate(Row baseRow)
+    {
+        ByteBuffer[] clusteringValues = new ByteBuffer[viewMetadata.clusteringColumns().size()];
+        for (ColumnDefinition viewColumn : viewMetadata.primaryKeyColumns())
+        {
+            ColumnDefinition baseColumn = view.getBaseColumn(viewColumn);
+            ByteBuffer value = getValueForPK(baseColumn, baseRow);
+            if (viewColumn.isPartitionKey())
+                currentViewEntryPartitionKey[viewColumn.position()] = value;
+            else
+                clusteringValues[viewColumn.position()] = value;
+        }
+
+        currentViewEntryBuilder.newRow(new Clustering(clusteringValues));
+    }
+
+    private LivenessInfo computeLivenessInfoForEntry(Row baseRow)
+    {
+        /*
+         * We need to compute both the timestamp and expiration.
+         *
+         * 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())
+        {
+            int ttl = baseLiveness.ttl();
+            int expirationTime = baseLiveness.localExpirationTime();
+            for (Cell cell : baseRow.cells())
+            {
+                if (cell.ttl() > ttl)
+                {
+                    ttl = cell.ttl();
+                    expirationTime = cell.localDeletionTime();
+                }
+            }
+            return ttl == baseLiveness.ttl()
+                 ? baseLiveness
+                 : LivenessInfo.create(baseLiveness.timestamp(), ttl, expirationTime);
+        }
+
+        ColumnDefinition baseColumn = view.baseNonPKColumnsInViewPK.get(0);
+        Cell cell = baseRow.getCell(baseColumn);
+        assert isLive(cell) : "We shouldn't have got there is the base row had no associated entry";
+
+        long timestamp = Math.max(baseLiveness.timestamp(), cell.timestamp());
+        return LivenessInfo.create(timestamp, cell.ttl(), cell.localDeletionTime());
+    }
+
+    private long computeTimestampForEntryDeletion(Row baseRow)
+    {
+        // 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)
+        {
+            if (!view.getDefinition().includes(data.column().name))
+                continue;
+
+            timestamp = Math.max(timestamp, data.maxTimestamp());
+        }
+        return timestamp;
+    }
+
+    private void addColumnData(ColumnDefinition viewColumn, ColumnData baseTableData)
+    {
+        assert viewColumn.isComplex() == baseTableData.column().isComplex();
+        if (!viewColumn.isComplex())
+        {
+            addCell(viewColumn, (Cell)baseTableData);
+            return;
+        }
+
+        ComplexColumnData complexData = (ComplexColumnData)baseTableData;
+        currentViewEntryBuilder.addComplexDeletion(viewColumn, complexData.complexDeletion());
+        for (Cell cell : complexData)
+            addCell(viewColumn, cell);
+    }
+
+    private void addCell(ColumnDefinition viewColumn, Cell baseTableCell)
+    {
+        assert !viewColumn.isPrimaryKeyColumn();
+        currentViewEntryBuilder.addCell(baseTableCell.withUpdatedColumn(viewColumn));
+    }
+
+    /**
+     * Finish building the currently updated view entry and add it to the other built
+     * updates.
+     */
+    private void submitUpdate()
+    {
+        Row row = currentViewEntryBuilder.build();
+        // I'm not sure we can reach there is there is nothing is updated, but adding an empty row breaks things
+        // and it costs us nothing to be prudent here.
+        if (row.isEmpty())
+            return;
+
+        DecoratedKey partitionKey = makeCurrentPartitionKey();
+        PartitionUpdate update = updates.get(partitionKey);
+        if (update == null)
+        {
+            // We can't really know which columns of the view will be updated nor how many row will be updated for this key
+            // so we rely on hopefully sane defaults.
+            update = new PartitionUpdate(viewMetadata, partitionKey, viewMetadata.partitionColumns(), 4);
+            updates.put(partitionKey, update);
+        }
+        update.add(row);
+    }
+
+    private DecoratedKey makeCurrentPartitionKey()
+    {
+        ByteBuffer rawKey = viewMetadata.partitionKeyColumns().size() == 1
+                          ? currentViewEntryPartitionKey[0]
+                          : CompositeType.build(currentViewEntryPartitionKey);
+
+        return viewMetadata.decorateKey(rawKey);
+    }
+
+    private ByteBuffer getValueForPK(ColumnDefinition column, Row row)
+    {
+        switch (column.kind)
+        {
+            case PARTITION_KEY:
+                return basePartitionKey[column.position()];
+            case CLUSTERING:
+                return row.clustering().get(column.position());
+            default:
+                // This shouldn't NPE as we shouldn't get there if the value can be null (or there is a bug in updateAction())
+                return row.getCell(column).value();
+        }
+    }
+}

http://git-wip-us.apache.org/repos/asf/cassandra/blob/86ba2274/test/unit/org/apache/cassandra/cql3/ViewTest.java
----------------------------------------------------------------------
diff --git a/test/unit/org/apache/cassandra/cql3/ViewTest.java b/test/unit/org/apache/cassandra/cql3/ViewTest.java
index ac4becb..4a1dc07 100644
--- a/test/unit/org/apache/cassandra/cql3/ViewTest.java
+++ b/test/unit/org/apache/cassandra/cql3/ViewTest.java
@@ -98,7 +98,7 @@ public class ViewTest extends CQLTester
     @Test
     public void testPartitionTombstone() throws Throwable
     {
-        createTable("CREATE TABLE %s (k1 int, c1 int , val int, PRIMARY KEY (k1))");
+        createTable("CREATE TABLE %s (k1 int, c1 int , val int, PRIMARY KEY (k1, c1))");
 
         execute("USE " + keyspace());
         executeNet(protocolVersion, "USE " + keyspace());
@@ -108,8 +108,8 @@ public class ViewTest extends CQLTester
         updateView("INSERT INTO %s (k1, c1, val) VALUES (1, 2, 200)");
         updateView("INSERT INTO %s (k1, c1, val) VALUES (1, 3, 300)");
 
-        Assert.assertEquals(1, execute("select * from %s").size());
-        Assert.assertEquals(1, execute("select * from view1").size());
+        Assert.assertEquals(2, execute("select * from %s").size());
+        Assert.assertEquals(2, execute("select * from view1").size());
 
         updateView("DELETE FROM %s WHERE k1 = 1");
 
@@ -814,18 +814,58 @@ public class ViewTest extends CQLTester
 
         createView("mv", "CREATE MATERIALIZED VIEW %s AS SELECT * FROM %%s WHERE c IS NOT NULL AND a IS NOT NULL AND b IS NOT NULL PRIMARY KEY (c, a, b)");
 
-        updateView("INSERT INTO %s (a, b, c, d) VALUES (?, ?, ?, ?) USING TTL 5", 1, 1, 1, 1);
+        updateView("INSERT INTO %s (a, b, c, d) VALUES (?, ?, ?, ?) USING TTL 3", 1, 1, 1, 1);
 
-        Thread.sleep(TimeUnit.SECONDS.toMillis(3));
+        Thread.sleep(TimeUnit.SECONDS.toMillis(1));
         updateView("INSERT INTO %s (a, b, c) VALUES (?, ?, ?)", 1, 1, 2);
 
-        Thread.sleep(TimeUnit.SECONDS.toMillis(3));
+        Thread.sleep(TimeUnit.SECONDS.toMillis(5));
         List<Row> results = executeNet(protocolVersion, "SELECT d FROM mv WHERE c = 2 AND a = 1 AND b = 1").all();
         Assert.assertEquals(1, results.size());
         Assert.assertTrue("There should be a null result given back due to ttl expiry", results.get(0).isNull(0));
     }
 
     @Test
+    public void ttlExpirationTest() throws Throwable
+    {
+        createTable("CREATE TABLE %s (" +
+                    "a int," +
+                    "b int," +
+                    "c int," +
+                    "d int," +
+                    "PRIMARY KEY (a, b))");
+
+        executeNet(protocolVersion, "USE " + keyspace());
+
+        createView("mv", "CREATE MATERIALIZED VIEW %s AS SELECT * FROM %%s WHERE c IS NOT NULL AND a IS NOT NULL AND b IS NOT NULL PRIMARY KEY (c, a, b)");
+
+        updateView("INSERT INTO %s (a, b, c, d) VALUES (?, ?, ?, ?) USING TTL 3", 1, 1, 1, 1);
+
+        Thread.sleep(TimeUnit.SECONDS.toMillis(4));
+        Assert.assertEquals(0, executeNet(protocolVersion, "SELECT * FROM mv WHERE c = 1 AND a = 1 AND b = 1").all().size());
+    }
+
+    @Test
+    public void rowDeletionTest() throws Throwable
+    {
+        createTable("CREATE TABLE %s (" +
+                    "a int," +
+                    "b int," +
+                    "c int," +
+                    "d int," +
+                    "PRIMARY KEY (a, b))");
+
+        executeNet(protocolVersion, "USE " + keyspace());
+
+        createView("mv", "CREATE MATERIALIZED VIEW %s AS SELECT * FROM %%s WHERE c IS NOT NULL AND a IS NOT NULL AND b IS NOT NULL PRIMARY KEY (c, a, b)");
+
+        String table = keyspace() + "." + currentTable();
+        updateView("DELETE FROM " + table + " USING TIMESTAMP 6 WHERE a = 1 AND b = 1;");
+        updateView("INSERT INTO %s (a, b, c, d) VALUES (?, ?, ?, ?) USING TIMESTAMP 3", 1, 1, 1, 1);
+        Assert.assertEquals(0, executeNet(protocolVersion, "SELECT * FROM mv WHERE c = 1 AND a = 1 AND b = 1").all().size());
+    }
+
+    @Test
     public void conflictingTimestampTest() throws Throwable
     {
         createTable("CREATE TABLE %s (" +

http://git-wip-us.apache.org/repos/asf/cassandra/blob/86ba2274/test/unit/org/apache/cassandra/db/rows/RowsTest.java
----------------------------------------------------------------------
diff --git a/test/unit/org/apache/cassandra/db/rows/RowsTest.java b/test/unit/org/apache/cassandra/db/rows/RowsTest.java
index dede867..b47bea2 100644
--- a/test/unit/org/apache/cassandra/db/rows/RowsTest.java
+++ b/test/unit/org/apache/cassandra/db/rows/RowsTest.java
@@ -239,7 +239,8 @@ public class RowsTest
                                                       BufferCell.live(kcvm, m, secondToTs(now), BB1, CellPath.create(BB1)),
                                                       BufferCell.live(kcvm, m, secondToTs(now), BB2, CellPath.create(BB2)));
         expectedCells.forEach(originalBuilder::addCell);
-        Row.Deletion rowDeletion = new Row.Deletion(new DeletionTime(ts, now), false);
+        // We need to use ts-1 so the deletion doesn't shadow what we've created
+        Row.Deletion rowDeletion = new Row.Deletion(new DeletionTime(ts-1, now), false);
         originalBuilder.addRowDeletion(rowDeletion);
 
         RowBuilder builder = new RowBuilder();
@@ -267,7 +268,8 @@ public class RowsTest
                                                       BufferCell.live(kcvm, m, ts, BB1, CellPath.create(BB1)),
                                                       BufferCell.live(kcvm, m, ts, BB2, CellPath.create(BB2)));
         expectedCells.forEach(builder::addCell);
-        Row.Deletion rowDeletion = new Row.Deletion(new DeletionTime(ts, now), false);
+        // We need to use ts-1 so the deletion doesn't shadow what we've created
+        Row.Deletion rowDeletion = new Row.Deletion(new DeletionTime(ts-1, now), false);
         builder.addRowDeletion(rowDeletion);
 
         StatsCollector collector = new StatsCollector();


[5/9] cassandra git commit: Refactor MV code

Posted by sl...@apache.org.
http://git-wip-us.apache.org/repos/asf/cassandra/blob/86ba2274/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 1b823aa..845a6ab 100644
--- a/src/java/org/apache/cassandra/db/view/View.java
+++ b/src/java/org/apache/cassandra/db/view/View.java
@@ -32,17 +32,15 @@ import org.apache.cassandra.cql3.statements.SelectStatement;
 import org.apache.cassandra.db.*;
 import org.apache.cassandra.config.*;
 import org.apache.cassandra.cql3.ColumnIdentifier;
-import org.apache.cassandra.db.AbstractReadCommandBuilder.SinglePartitionSliceBuilder;
 import org.apache.cassandra.db.compaction.CompactionManager;
-import org.apache.cassandra.db.partitions.AbstractBTreePartition;
-import org.apache.cassandra.db.partitions.PartitionIterator;
-import org.apache.cassandra.db.partitions.PartitionUpdate;
+import org.apache.cassandra.db.partitions.*;
 import org.apache.cassandra.db.rows.*;
 import org.apache.cassandra.schema.KeyspaceMetadata;
 import org.apache.cassandra.service.ClientState;
 import org.apache.cassandra.service.pager.QueryPager;
 import org.apache.cassandra.transport.Server;
 import org.apache.cassandra.utils.FBUtilities;
+import org.apache.cassandra.utils.btree.BTreeSet;
 import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
 
@@ -50,46 +48,18 @@ import org.slf4j.LoggerFactory;
  * A View copies data from a base table into a view table which can be queried independently from the
  * base. Every update which targets the base table must be fed through the {@link ViewManager} to ensure
  * that if a view needs to be updated, the updates are properly created and fed into the view.
- *
- * This class does the job of translating the base row to the view row.
- *
- * It handles reading existing state and figuring out what tombstones need to be generated.
- *
- * {@link View#createMutations(AbstractBTreePartition, TemporalRow.Set, boolean)} is the "main method"
- *
  */
 public class View
 {
     private static final Logger logger = LoggerFactory.getLogger(View.class);
 
-    /**
-     * The columns should all be updated together, so we use this object as group.
-     */
-    private static class Columns
-    {
-        //These are the base column definitions in terms of the *views* partitioning.
-        //Meaning we can see (for example) the partition key of the view contains a clustering key
-        //from the base table.
-        public final List<ColumnDefinition> partitionDefs;
-        public final List<ColumnDefinition> primaryKeyDefs;
-        public final List<ColumnDefinition> baseComplexColumns;
-
-        private Columns(List<ColumnDefinition> partitionDefs, List<ColumnDefinition> primaryKeyDefs, List<ColumnDefinition> baseComplexColumns)
-        {
-            this.partitionDefs = partitionDefs;
-            this.primaryKeyDefs = primaryKeyDefs;
-            this.baseComplexColumns = baseComplexColumns;
-        }
-    }
-
     public final String name;
     private volatile ViewDefinition definition;
 
     private final ColumnFamilyStore baseCfs;
 
-    private Columns columns;
+    public volatile List<ColumnDefinition> baseNonPKColumnsInViewPK;
 
-    private final boolean viewPKIncludesOnlyBasePKColumns;
     private final boolean includeAllColumns;
     private ViewBuilder builder;
 
@@ -104,12 +74,11 @@ public class View
                 ColumnFamilyStore baseCfs)
     {
         this.baseCfs = baseCfs;
-
-        name = definition.viewName;
-        includeAllColumns = definition.includeAllColumns;
-
-        viewPKIncludesOnlyBasePKColumns = updateDefinition(definition);
+        this.name = definition.viewName;
+        this.includeAllColumns = definition.includeAllColumns;
         this.rawSelect = definition.select;
+
+        updateDefinition(definition);
     }
 
     public ViewDefinition getDefinition()
@@ -118,513 +87,100 @@ public class View
     }
 
     /**
-     * Lookup column definitions in the base table that correspond to the view columns (should be 1:1)
-     *
-     * Notify caller if all primary keys in the view are ALL primary keys in the base. We do this to simplify
-     * tombstone checks.
-     *
-     * @param columns a list of columns to lookup in the base table
-     * @param definitions lists to populate for the base table definitions
-     * @return true if all view PKs are also Base PKs
-     */
-    private boolean resolveAndAddColumns(Iterable<ColumnIdentifier> columns, List<ColumnDefinition>... definitions)
-    {
-        boolean allArePrimaryKeys = true;
-        for (ColumnIdentifier identifier : columns)
-        {
-            ColumnDefinition cdef = baseCfs.metadata.getColumnDefinition(identifier);
-            assert cdef != null : "Could not resolve column " + identifier.toString();
-
-            for (List<ColumnDefinition> list : definitions)
-            {
-                list.add(cdef);
-            }
-
-            allArePrimaryKeys = allArePrimaryKeys && cdef.isPrimaryKeyColumn();
-        }
-
-        return allArePrimaryKeys;
-    }
-
-    /**
      * This updates the columns stored which are dependent on the base CFMetaData.
      *
      * @return true if the view contains only columns which are part of the base's primary key; false if there is at
      *         least one column which is not.
      */
-    public boolean updateDefinition(ViewDefinition definition)
+    public void updateDefinition(ViewDefinition definition)
     {
         this.definition = definition;
 
         CFMetaData viewCfm = definition.metadata;
-        List<ColumnDefinition> partitionDefs = new ArrayList<>(viewCfm.partitionKeyColumns().size());
-        List<ColumnDefinition> primaryKeyDefs = new ArrayList<>(viewCfm.partitionKeyColumns().size()
-                                                                + viewCfm.clusteringColumns().size());
-        List<ColumnDefinition> baseComplexColumns = new ArrayList<>();
-
-        // We only add the partition columns to the partitions list, but both partition columns and clustering
-        // columns are added to the primary keys list
-        boolean partitionAllPrimaryKeyColumns = resolveAndAddColumns(Iterables.transform(viewCfm.partitionKeyColumns(), cd -> cd.name), primaryKeyDefs, partitionDefs);
-        boolean clusteringAllPrimaryKeyColumns = resolveAndAddColumns(Iterables.transform(viewCfm.clusteringColumns(), cd -> cd.name), primaryKeyDefs);
-
-        for (ColumnDefinition cdef : baseCfs.metadata.allColumns())
+        List<ColumnDefinition> nonPKDefPartOfViewPK = new ArrayList<>();
+        for (ColumnDefinition baseColumn : baseCfs.metadata.allColumns())
         {
-            if (cdef.isComplex() && definition.includes(cdef.name))
-            {
-                baseComplexColumns.add(cdef);
-            }
+            ColumnDefinition viewColumn = getViewColumn(baseColumn);
+            if (viewColumn != null && !baseColumn.isPrimaryKeyColumn() && viewColumn.isPrimaryKeyColumn())
+                nonPKDefPartOfViewPK.add(baseColumn);
         }
-
-        this.columns = new Columns(partitionDefs, primaryKeyDefs, baseComplexColumns);
-
-        return partitionAllPrimaryKeyColumns && clusteringAllPrimaryKeyColumns;
+        this.baseNonPKColumnsInViewPK = nonPKDefPartOfViewPK;
     }
 
     /**
-     * Check to see if the update could possibly modify a view. Cases where the view may be updated are:
-     * <ul>
-     *     <li>View selects all columns</li>
-     *     <li>Update contains any range tombstones</li>
-     *     <li>Update touches one of the columns included in the view</li>
-     * </ul>
-     *
-     * If the update contains any range tombstones, there is a possibility that it will not touch a range that is
-     * currently included in the view.
-     *
-     * @return true if {@param partition} modifies a column included in the view
+     * The view column corresponding to the provided base column. This <b>can</b>
+     * return {@code null} if the column is denormalized in the view.
      */
-    public boolean updateAffectsView(AbstractBTreePartition partition)
+    public ColumnDefinition getViewColumn(ColumnDefinition baseColumn)
     {
-        ReadQuery selectQuery = getReadQuery();
-
-        if (!partition.metadata().cfId.equals(definition.baseTableId))
-            return false;
-
-        if (!selectQuery.selectsKey(partition.partitionKey()))
-            return false;
-
-        // If there are range tombstones, tombstones will also need to be generated for the view
-        // This requires a query of the base rows and generating tombstones for all of those values
-        if (!partition.deletionInfo().isLive())
-            return true;
-
-        // Check each row for deletion or update
-        for (Row row : partition)
-        {
-            if (!selectQuery.selectsClustering(partition.partitionKey(), row.clustering()))
-                continue;
-
-            if (includeAllColumns || !row.deletion().isLive())
-                return true;
-
-            if (row.primaryKeyLivenessInfo().isLive(FBUtilities.nowInSeconds()))
-                return true;
-
-            for (ColumnData data : row)
-            {
-                if (definition.metadata.getColumnDefinition(data.column().name) != null)
-                    return true;
-            }
-        }
-
-        return false;
-    }
-
-    /**
-     * Creates the clustering columns for the view based on the specified row and resolver policy
-     *
-     * @param temporalRow The current row
-     * @param resolver The policy to use when selecting versions of cells use
-     * @return The clustering object to use for the view
-     */
-    private Clustering viewClustering(TemporalRow temporalRow, TemporalRow.Resolver resolver)
-    {
-        CFMetaData viewCfm = definition.metadata;
-        int numViewClustering = viewCfm.clusteringColumns().size();
-        CBuilder clustering = CBuilder.create(viewCfm.comparator);
-        for (int i = 0; i < numViewClustering; i++)
-        {
-            ColumnDefinition definition = viewCfm.clusteringColumns().get(i);
-            clustering.add(temporalRow.clusteringValue(definition, resolver));
-        }
-
-        return clustering.build();
-    }
-
-    /**
-     * @return Mutation containing a range tombstone for a base partition key and TemporalRow.
-     */
-    private PartitionUpdate createTombstone(TemporalRow temporalRow,
-                                            DecoratedKey partitionKey,
-                                            Row.Deletion deletion,
-                                            TemporalRow.Resolver resolver,
-                                            int nowInSec)
-    {
-        CFMetaData viewCfm = definition.metadata;
-        Row.Builder builder = BTreeRow.unsortedBuilder(nowInSec);
-        builder.newRow(viewClustering(temporalRow, resolver));
-        builder.addRowDeletion(deletion);
-        return PartitionUpdate.singleRowUpdate(viewCfm, partitionKey, builder.build());
-    }
-
-    /**
-     * @return PartitionUpdate containing a complex tombstone for a TemporalRow, and the collection's column identifier.
-     */
-    private PartitionUpdate createComplexTombstone(TemporalRow temporalRow,
-                                                   DecoratedKey partitionKey,
-                                                   ColumnDefinition deletedColumn,
-                                                   DeletionTime deletionTime,
-                                                   TemporalRow.Resolver resolver,
-                                                   int nowInSec)
-    {
-        CFMetaData viewCfm = definition.metadata;
-        Row.Builder builder = BTreeRow.unsortedBuilder(nowInSec);
-        builder.newRow(viewClustering(temporalRow, resolver));
-        builder.addComplexDeletion(deletedColumn, deletionTime);
-        return PartitionUpdate.singleRowUpdate(viewCfm, partitionKey, builder.build());
+        return definition.metadata.getColumnDefinition(baseColumn.name);
     }
 
     /**
-     * @return View's DecoratedKey or null, if one of the view's primary key components has an invalid resolution from
-     *         the TemporalRow and its Resolver
+     * The base column corresponding to the provided view column. This should
+     * never return {@code null} since a view can't have its "own" columns.
      */
-    private DecoratedKey viewPartitionKey(TemporalRow temporalRow, TemporalRow.Resolver resolver)
+    public ColumnDefinition getBaseColumn(ColumnDefinition viewColumn)
     {
-        List<ColumnDefinition> partitionDefs = this.columns.partitionDefs;
-        Object[] partitionKey = new Object[partitionDefs.size()];
-
-        for (int i = 0; i < partitionKey.length; i++)
-        {
-            ByteBuffer value = temporalRow.clusteringValue(partitionDefs.get(i), resolver);
-
-            if (value == null)
-                return null;
-
-            partitionKey[i] = value;
-        }
-
-        CFMetaData metadata = definition.metadata;
-        return metadata.decorateKey(CFMetaData.serializePartitionKey(metadata
-                                                                     .getKeyValidatorAsClusteringComparator()
-                                                                     .make(partitionKey)));
+        ColumnDefinition baseColumn = baseCfs.metadata.getColumnDefinition(viewColumn.name);
+        assert baseColumn != null;
+        return baseColumn;
     }
 
     /**
-     * @return mutation which contains the tombstone for the referenced TemporalRow, or null if not necessary.
-     * TemporalRow's can reference at most one view row; there will be at most one row to be tombstoned, so only one
-     * mutation is necessary
-     */
-    private PartitionUpdate createRangeTombstoneForRow(TemporalRow temporalRow)
-    {
-        // Primary Key and Clustering columns do not generate tombstones
-        if (viewPKIncludesOnlyBasePKColumns)
-            return null;
-
-        boolean hasUpdate = false;
-        List<ColumnDefinition> primaryKeyDefs = this.columns.primaryKeyDefs;
-        for (ColumnDefinition viewPartitionKeys : primaryKeyDefs)
-        {
-            if (!viewPartitionKeys.isPrimaryKeyColumn() && temporalRow.clusteringValue(viewPartitionKeys, TemporalRow.oldValueIfUpdated) != null)
-                hasUpdate = true;
-        }
-
-        if (!hasUpdate)
-            return null;
-
-        TemporalRow.Resolver resolver = TemporalRow.earliest;
-        return createTombstone(temporalRow,
-                               viewPartitionKey(temporalRow, resolver),
-                               Row.Deletion.shadowable(new DeletionTime(temporalRow.viewClusteringTimestamp(), temporalRow.nowInSec)),
-                               resolver,
-                               temporalRow.nowInSec);
-    }
-
-    /**
-     * @return Mutation which is the transformed base table mutation for the view.
-     */
-    private PartitionUpdate createUpdatesForInserts(TemporalRow temporalRow)
-    {
-        TemporalRow.Resolver resolver = TemporalRow.latest;
-
-        DecoratedKey partitionKey = viewPartitionKey(temporalRow, resolver);
-        CFMetaData viewCfm = definition.metadata;
-
-        if (partitionKey == null)
-        {
-            // Not having a partition key means we aren't updating anything
-            return null;
-        }
-
-        Row.Builder regularBuilder = BTreeRow.unsortedBuilder(temporalRow.nowInSec);
-
-        CBuilder clustering = CBuilder.create(viewCfm.comparator);
-        for (int i = 0; i < viewCfm.clusteringColumns().size(); i++)
-        {
-            ColumnDefinition column = viewCfm.clusteringColumns().get(i);
-            ByteBuffer value = temporalRow.clusteringValue(column, resolver);
-
-            // handle single-column deletions correctly to avoid nulls in the view primary key
-            if (value == null)
-                return null;
-
-            clustering.add(value);
-        }
-        regularBuilder.newRow(clustering.build());
-        regularBuilder.addPrimaryKeyLivenessInfo(LivenessInfo.create(viewCfm,
-                                                                     temporalRow.viewClusteringTimestamp(),
-                                                                     temporalRow.viewClusteringTtl(),
-                                                                     temporalRow.viewClusteringLocalDeletionTime()));
-
-        for (ColumnDefinition columnDefinition : viewCfm.allColumns())
-        {
-            if (columnDefinition.isPrimaryKeyColumn())
-                continue;
-
-            for (Cell cell : temporalRow.values(columnDefinition, resolver))
-            {
-                regularBuilder.addCell(cell);
-            }
-        }
-
-        Row row = regularBuilder.build();
-
-        // although we check for empty rows in updateAppliesToView(), if there are any good rows in the PartitionUpdate,
-        // all rows in the partition will be processed, and we need to handle empty/non-live rows here (CASSANDRA-10614)
-        if (row.isEmpty())
-            return null;
-
-        return PartitionUpdate.singleRowUpdate(viewCfm, partitionKey, row);
-    }
-
-    /**
-     * @param partition Update which possibly contains deletion info for which to generate view tombstones.
-     * @return    View Tombstones which delete all of the rows which have been removed from the base table with
-     *            {@param partition}
-     */
-    private Collection<Mutation> createForDeletionInfo(TemporalRow.Set rowSet, AbstractBTreePartition partition)
-    {
-        final TemporalRow.Resolver resolver = TemporalRow.earliest;
-
-        DeletionInfo deletionInfo = partition.deletionInfo();
-
-        List<Mutation> mutations = new ArrayList<>();
-
-        // Check the complex columns to see if there are any which may have tombstones we need to create for the view
-        if (!columns.baseComplexColumns.isEmpty())
-        {
-            for (Row row : partition)
-            {
-                if (!row.hasComplexDeletion())
-                    continue;
-
-                TemporalRow temporalRow = rowSet.getClustering(row.clustering());
-
-                assert temporalRow != null;
-
-                for (ColumnDefinition definition : columns.baseComplexColumns)
-                {
-                    ComplexColumnData columnData = row.getComplexColumnData(definition);
-
-                    if (columnData != null)
-                    {
-                        DeletionTime time = columnData.complexDeletion();
-                        if (!time.isLive())
-                        {
-                            DecoratedKey targetKey = viewPartitionKey(temporalRow, resolver);
-                            if (targetKey != null)
-                                mutations.add(new Mutation(createComplexTombstone(temporalRow, targetKey, definition, time, resolver, temporalRow.nowInSec)));
-                        }
-                    }
-                }
-            }
-        }
-
-        ReadCommand command = null;
-
-        if (!deletionInfo.isLive())
-        {
-            // We have to generate tombstones for all of the affected rows, but we don't have the information in order
-            // to create them. This requires that we perform a read for the entire range that is being tombstoned, and
-            // generate a tombstone for each. This may be slow, because a single range tombstone can cover up to an
-            // entire partition of data which is not distributed on a single partition node.
-            DecoratedKey dk = rowSet.dk;
-
-            if (!deletionInfo.getPartitionDeletion().isLive())
-            {
-                command = getSelectStatement().internalReadForView(dk, rowSet.nowInSec);
-            }
-            else
-            {
-                SinglePartitionSliceBuilder builder = new SinglePartitionSliceBuilder(baseCfs, dk);
-                Iterator<RangeTombstone> tombstones = deletionInfo.rangeIterator(false);
-                while (tombstones.hasNext())
-                {
-                    RangeTombstone tombstone = tombstones.next();
-
-                    builder.addSlice(tombstone.deletedSlice());
-                }
-
-                command = builder.build();
-            }
-        }
-
-        if (command == null)
-        {
-            ReadQuery selectQuery = getReadQuery();
-            SinglePartitionSliceBuilder builder = null;
-            for (Row row : partition)
-            {
-                if (!row.deletion().isLive())
-                {
-                    if (!selectQuery.selectsClustering(rowSet.dk, row.clustering()))
-                        continue;
-
-                    if (builder == null)
-                        builder = new SinglePartitionSliceBuilder(baseCfs, rowSet.dk);
-                    builder.addSlice(Slice.make(row.clustering()));
-                }
-            }
-
-            if (builder != null)
-                command = builder.build();
-        }
-
-        if (command != null)
-        {
-            ReadQuery selectQuery = getReadQuery();
-            assert selectQuery.selectsKey(rowSet.dk);
-
-            // We may have already done this work for another MV update so check
-            if (!rowSet.hasTombstonedExisting())
-            {
-                QueryPager pager = command.getPager(null, Server.CURRENT_VERSION);
-
-                // Add all of the rows which were recovered from the query to the row set
-                while (!pager.isExhausted())
-                {
-                    try (ReadOrderGroup orderGroup = pager.startOrderGroup();
-                         PartitionIterator iter = pager.fetchPageInternal(128, orderGroup))
-                    {
-                        if (!iter.hasNext())
-                            break;
-
-                        try (RowIterator rowIterator = iter.next())
-                        {
-                            while (rowIterator.hasNext())
-                            {
-                                Row row = rowIterator.next();
-                                if (selectQuery.selectsClustering(rowSet.dk, row.clustering()))
-                                    rowSet.addRow(row, false);
-                            }
-                        }
-                    }
-                }
-
-                //Incase we fetched nothing, avoid re checking on another MV update
-                rowSet.setTombstonedExisting();
-            }
-
-            // If the temporal row has been deleted by the deletion info, we generate the corresponding range tombstone
-            // for the view.
-            for (TemporalRow temporalRow : rowSet)
-            {
-                DeletionTime deletionTime = temporalRow.deletionTime(partition);
-                if (!deletionTime.isLive())
-                {
-                    DecoratedKey value = viewPartitionKey(temporalRow, resolver);
-                    if (value != null)
-                    {
-                        PartitionUpdate update = createTombstone(temporalRow, value, Row.Deletion.regular(deletionTime), resolver, temporalRow.nowInSec);
-                        if (update != null)
-                            mutations.add(new Mutation(update));
-                    }
-                }
-            }
-        }
-
-        return !mutations.isEmpty() ? mutations : null;
-    }
-
-    /**
-     * Read and update temporal rows in the set which have corresponding values stored on the local node
+     * Whether the view might be affected by the provided update.
+     * <p>
+     * Note that having this method return {@code true} is not an absolute guarantee that the view will be
+     * updated, just that it most likely will, but a {@code false} return guarantees it won't be affected).
+     *
+     * @param partitionKey the partition key that is updated.
+     * @param update the update being applied.
+     * @return {@code false} if we can guarantee that inserting {@code update} for key {@code partitionKey}
+     * won't affect the view in any way, {@code true} otherwise.
      */
-    private void readLocalRows(TemporalRow.Set rowSet)
+    public boolean mayBeAffectedBy(DecoratedKey partitionKey, Row update)
     {
-        long start = System.currentTimeMillis();
-        SinglePartitionSliceBuilder builder = new SinglePartitionSliceBuilder(baseCfs, rowSet.dk);
-
-        for (TemporalRow temporalRow : rowSet)
-            builder.addSlice(temporalRow.baseSlice());
+        // We can guarantee that the view won't be affected if:
+        //  - the clustering is excluded by the view filter (note that this isn't true of the filter on regular columns:
+        //    even if an update don't match a view condition on a regular column, that update can still invalidate an pre-existing
+        //    entry).
+        //  - or the update don't modify any of the columns impacting the view (where "impacting" the view means that column is
+        //    neither included in the view, nor used by the view filter).
+        if (!getReadQuery().selectsClustering(partitionKey, update.clustering()))
+            return false;
 
-        QueryPager pager = builder.build().getPager(null, Server.CURRENT_VERSION);
+        // 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;
 
-        while (!pager.isExhausted())
+        for (ColumnData data : update)
         {
-            try (ReadOrderGroup orderGroup = pager.startOrderGroup();
-                 PartitionIterator iter = pager.fetchPageInternal(128, orderGroup))
-            {
-                while (iter.hasNext())
-                {
-                    try (RowIterator rows = iter.next())
-                    {
-                        while (rows.hasNext())
-                        {
-                            rowSet.addRow(rows.next(), false);
-                        }
-                    }
-                }
-            }
+            if (definition.metadata.getColumnDefinition(data.column().name) != null)
+                return true;
         }
-        baseCfs.metric.viewReadTime.update(System.currentTimeMillis() - start, TimeUnit.MILLISECONDS);
-    }
-
-    /**
-     * @return Set of rows which are contained in the partition update {@param partition}
-     */
-    private TemporalRow.Set separateRows(AbstractBTreePartition partition, Set<ColumnIdentifier> viewPrimaryKeyCols)
-    {
-
-        TemporalRow.Set rowSet = new TemporalRow.Set(baseCfs, viewPrimaryKeyCols, partition.partitionKey().getKey());
-
-        for (Row row : partition)
-            rowSet.addRow(row, true);
-
-        return rowSet;
+        return false;
     }
 
     /**
-     * Splits the partition update up and adds the existing state to each row.
-     * This data can be reused for multiple MV updates on the same base table
+     * Whether a given base row matches the view filter (and thus if is should have a corresponding entry).
+     * <p>
+     * Note that this differs from {@link #mayBeAffectedBy} in that the provide row <b>must</b> be the current
+     * state of the base row, not just some updates to it. This method also has no false positive: a base
+     * row either do or don't match the view filter.
      *
-     * @param partition the mutation
-     * @param isBuilding If the view is currently being built, we do not query the values which are already stored,
-     *                   since all of the update will already be present in the base table.
-     * @return The set of temoral rows contained in this update
+     * @param partitionKey the partition key that is updated.
+     * @param baseRow the current state of a particular base row.
+     * @param nowInSec the current time in seconds (to decide what is live and what isn't).
+     * @return {@code true} if {@code baseRow} matches the view filters, {@code false} otherwise.
      */
-    public TemporalRow.Set getTemporalRowSet(AbstractBTreePartition partition, TemporalRow.Set existing, boolean isBuilding)
+    public boolean matchesViewFilter(DecoratedKey partitionKey, Row baseRow, int nowInSec)
     {
-        if (!updateAffectsView(partition))
-            return existing;
-
-        Set<ColumnIdentifier> columns = new HashSet<>(this.columns.primaryKeyDefs.size());
-        for (ColumnDefinition def : this.columns.primaryKeyDefs)
-            columns.add(def.name);
-
-        TemporalRow.Set rowSet;
-        if (existing == null)
-        {
-            rowSet = separateRows(partition, columns);
-
-            // If we are building the view, we do not want to add old values; they will always be the same
-            if (!isBuilding)
-                readLocalRows(rowSet);
-        }
-        else
-        {
-            rowSet = existing.withNewViewPrimaryKey(columns);
-        }
-
-        return rowSet;
+        return getReadQuery().selectsClustering(partitionKey, baseRow.clustering())
+            && getSelectStatement().rowFilterForInternalCalls().isSatisfiedBy(baseCfs.metadata, partitionKey, baseRow, nowInSec);
     }
 
     /**
@@ -656,61 +212,6 @@ public class View
         return query;
     }
 
-    /**
-     * @param isBuilding If the view is currently being built, we do not query the values which are already stored,
-     *                   since all of the update will already be present in the base table.
-     * @return View mutations which represent the changes necessary as long as previously created mutations for the view
-     *         have been applied successfully. This is based solely on the changes that are necessary given the current
-     *         state of the base table and the newly applying partition data.
-     */
-    public Collection<Mutation> createMutations(AbstractBTreePartition partition, TemporalRow.Set rowSet, boolean isBuilding)
-    {
-        if (!updateAffectsView(partition))
-            return null;
-
-        ReadQuery selectQuery = getReadQuery();
-        Collection<Mutation> mutations = null;
-        for (TemporalRow temporalRow : rowSet)
-        {
-            // In updateAffectsView, we check the partition to see if there is at least one row that matches the
-            // filters and is live.  If there is more than one row in the partition, we need to re-check each one
-            // individually.
-            if (partition.rowCount() != 1 && !selectQuery.selectsClustering(partition.partitionKey(), temporalRow.baseClustering()))
-                continue;
-
-            // If we are building, there is no need to check for partition tombstones; those values will not be present
-            // in the partition data
-            if (!isBuilding)
-            {
-                PartitionUpdate partitionTombstone = createRangeTombstoneForRow(temporalRow);
-                if (partitionTombstone != null)
-                {
-                    if (mutations == null) mutations = new LinkedList<>();
-                    mutations.add(new Mutation(partitionTombstone));
-                }
-            }
-
-            PartitionUpdate insert = createUpdatesForInserts(temporalRow);
-            if (insert != null)
-            {
-                if (mutations == null) mutations = new LinkedList<>();
-                mutations.add(new Mutation(insert));
-            }
-        }
-
-        if (!isBuilding)
-        {
-            Collection<Mutation> deletion = createForDeletionInfo(rowSet, partition);
-            if (deletion != null && !deletion.isEmpty())
-            {
-                if (mutations == null) mutations = new LinkedList<>();
-                mutations.addAll(deletion);
-            }
-        }
-
-        return mutations;
-    }
-
     public synchronized void build()
     {
         if (this.builder != null)

http://git-wip-us.apache.org/repos/asf/cassandra/blob/86ba2274/src/java/org/apache/cassandra/db/view/ViewBuilder.java
----------------------------------------------------------------------
diff --git a/src/java/org/apache/cassandra/db/view/ViewBuilder.java b/src/java/org/apache/cassandra/db/view/ViewBuilder.java
index 35b023b..b2b409b 100644
--- a/src/java/org/apache/cassandra/db/view/ViewBuilder.java
+++ b/src/java/org/apache/cassandra/db/view/ViewBuilder.java
@@ -19,6 +19,7 @@
 package org.apache.cassandra.db.view;
 
 import java.util.Collection;
+import java.util.Collections;
 import java.util.UUID;
 import java.util.concurrent.TimeUnit;
 import java.util.concurrent.atomic.AtomicLong;
@@ -35,9 +36,8 @@ import org.apache.cassandra.db.compaction.CompactionInfo;
 import org.apache.cassandra.db.compaction.CompactionManager;
 import org.apache.cassandra.db.compaction.OperationType;
 import org.apache.cassandra.db.lifecycle.SSTableSet;
-import org.apache.cassandra.db.partitions.FilteredPartition;
-import org.apache.cassandra.db.partitions.PartitionIterator;
-import org.apache.cassandra.db.rows.RowIterator;
+import org.apache.cassandra.db.partitions.*;
+import org.apache.cassandra.db.rows.*;
 import org.apache.cassandra.dht.Range;
 import org.apache.cassandra.dht.Token;
 import org.apache.cassandra.io.sstable.ReducingKeyIterator;
@@ -76,28 +76,22 @@ public class ViewBuilder extends CompactionInfo.Holder
         if (!selectQuery.selectsKey(key))
             return;
 
-        QueryPager pager = view.getSelectStatement().internalReadForView(key, FBUtilities.nowInSeconds()).getPager(null, Server.CURRENT_VERSION);
+        int nowInSec = FBUtilities.nowInSeconds();
+        SinglePartitionReadCommand command = view.getSelectStatement().internalReadForView(key, nowInSec);
 
-        while (!pager.isExhausted())
+        // We're rebuilding everything from what's on disk, so we read everything, consider that as new updates
+        // and pretend that there is nothing pre-existing.
+        UnfilteredRowIterator empty = UnfilteredRowIterators.noRowsIterator(baseCfs.metadata, key, Rows.EMPTY_STATIC_ROW, DeletionTime.LIVE, false);
+
+        Collection<Mutation> mutations;
+        try (ReadOrderGroup orderGroup = command.startOrderGroup();
+             UnfilteredRowIterator data = UnfilteredPartitionIterators.getOnlyElement(command.executeLocally(orderGroup), command))
         {
-           try (ReadOrderGroup orderGroup = pager.startOrderGroup();
-                PartitionIterator partitionIterator = pager.fetchPageInternal(128, orderGroup))
-           {
-               if (!partitionIterator.hasNext())
-                   return;
-
-               try (RowIterator rowIterator = partitionIterator.next())
-               {
-                   FilteredPartition partition = FilteredPartition.create(rowIterator);
-                   TemporalRow.Set temporalRows = view.getTemporalRowSet(partition, null, true);
-
-                   Collection<Mutation> mutations = view.createMutations(partition, temporalRows, true);
-
-                   if (mutations != null)
-                       StorageProxy.mutateMV(key.getKey(), mutations, true, noBase);
-               }
-           }
+            mutations = baseCfs.keyspace.viewManager.forTable(baseCfs.metadata).generateViewUpdates(Collections.singleton(view), data, empty, nowInSec);
         }
+
+        if (!mutations.isEmpty())
+            StorageProxy.mutateMV(key.getKey(), mutations, true, noBase);
     }
 
     public void run()

http://git-wip-us.apache.org/repos/asf/cassandra/blob/86ba2274/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 9fe0544..fd04b97 100644
--- a/src/java/org/apache/cassandra/db/view/ViewManager.java
+++ b/src/java/org/apache/cassandra/db/view/ViewManager.java
@@ -19,23 +19,21 @@ package org.apache.cassandra.db.view;
 
 import java.nio.ByteBuffer;
 import java.util.*;
-import java.util.concurrent.ConcurrentNavigableMap;
-import java.util.concurrent.ConcurrentSkipListMap;
-import java.util.concurrent.atomic.AtomicLong;
+import java.util.concurrent.ConcurrentMap;
+import java.util.concurrent.ConcurrentHashMap;
 import java.util.concurrent.locks.Lock;
 
-import com.google.common.collect.Lists;
 import com.google.common.util.concurrent.Striped;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
 
+import org.apache.cassandra.config.CFMetaData;
 import org.apache.cassandra.config.DatabaseDescriptor;
 import org.apache.cassandra.config.ViewDefinition;
 import org.apache.cassandra.db.*;
-import org.apache.cassandra.db.commitlog.ReplayPosition;
-import org.apache.cassandra.db.partitions.PartitionUpdate;
-import org.apache.cassandra.dht.Token;
-import org.apache.cassandra.service.StorageProxy;
-import org.slf4j.Logger;
-import org.slf4j.LoggerFactory;
+import org.apache.cassandra.db.rows.*;
+import org.apache.cassandra.db.partitions.*;
+
 
 /**
  * Manages {@link View}'s for a single {@link ColumnFamilyStore}. All of the views for that table are created when this
@@ -45,110 +43,30 @@ import org.slf4j.LoggerFactory;
  * any views {@link ViewManager#updatesAffectView(Collection, boolean)}, provide locks to prevent multiple
  * updates from creating incoherent updates in the view {@link ViewManager#acquireLockFor(ByteBuffer)}, and
  * to affect change on the view.
+ *
+ * TODO: I think we can get rid of that class. For addition/removal of view by names, we could move it Keyspace. And we
+ * not sure it's even worth keeping viewsByName as none of the related operation are performance sensitive so we could
+ * find the view by iterating over the CFStore.viewManager directly.
+ * For the lock, it could move to Keyspace too, but I don't remmenber why it has to be at the keyspace level and if it
+ * can be at the table level, maybe that's where it should be.
  */
 public class ViewManager
 {
     private static final Logger logger = LoggerFactory.getLogger(ViewManager.class);
 
-    public class ForStore
-    {
-        private final ConcurrentNavigableMap<String, View> viewsByName;
-
-        public ForStore()
-        {
-            this.viewsByName = new ConcurrentSkipListMap<>();
-        }
-
-        public Iterable<View> allViews()
-        {
-            return viewsByName.values();
-        }
-
-        public Iterable<ColumnFamilyStore> allViewsCfs()
-        {
-            List<ColumnFamilyStore> viewColumnFamilies = new ArrayList<>();
-            for (View view : allViews())
-                viewColumnFamilies.add(keyspace.getColumnFamilyStore(view.getDefinition().viewName));
-            return viewColumnFamilies;
-        }
-
-        public void forceBlockingFlush()
-        {
-            for (ColumnFamilyStore viewCfs : allViewsCfs())
-                viewCfs.forceBlockingFlush();
-        }
-
-        public void dumpMemtables()
-        {
-            for (ColumnFamilyStore viewCfs : allViewsCfs())
-                viewCfs.dumpMemtable();
-        }
-
-        public void truncateBlocking(long truncatedAt)
-        {
-            for (ColumnFamilyStore viewCfs : allViewsCfs())
-            {
-                ReplayPosition replayAfter = viewCfs.discardSSTables(truncatedAt);
-                SystemKeyspace.saveTruncationRecord(viewCfs, truncatedAt, replayAfter);
-            }
-        }
-
-        public void addView(View view)
-        {
-            viewsByName.put(view.name, view);
-        }
-
-        public void removeView(String name)
-        {
-            viewsByName.remove(name);
-        }
-    }
-
     private static final Striped<Lock> LOCKS = Striped.lazyWeakLock(DatabaseDescriptor.getConcurrentViewWriters() * 1024);
 
     private static final boolean enableCoordinatorBatchlog = Boolean.getBoolean("cassandra.mv_enable_coordinator_batchlog");
 
-    private final ConcurrentNavigableMap<UUID, ForStore> viewManagersByStore;
-    private final ConcurrentNavigableMap<String, View> viewsByName;
+    private final ConcurrentMap<String, View> viewsByName = new ConcurrentHashMap<>();
+    private final ConcurrentMap<UUID, TableViews> viewsByBaseTable = new ConcurrentHashMap<>();
     private final Keyspace keyspace;
 
     public ViewManager(Keyspace keyspace)
     {
-        this.viewManagersByStore = new ConcurrentSkipListMap<>();
-        this.viewsByName = new ConcurrentSkipListMap<>();
         this.keyspace = keyspace;
     }
 
-    /**
-     * Calculates and pushes updates to the views replicas. The replicas are determined by
-     * {@link ViewUtils#getViewNaturalEndpoint(String, Token, Token)}.
-     */
-    public void pushViewReplicaUpdates(PartitionUpdate update, boolean writeCommitLog, AtomicLong baseComplete)
-    {
-        List<Mutation> mutations = null;
-        TemporalRow.Set temporalRows = null;
-        for (Map.Entry<String, View> view : viewsByName.entrySet())
-        {
-            // Make sure that we only get mutations from views which are affected since the set includes all views for a
-            // keyspace. This will prevent calling getTemporalRowSet for the wrong base table.
-            if (view.getValue().updateAffectsView(update))
-            {
-                temporalRows = view.getValue().getTemporalRowSet(update, temporalRows, false);
-
-                Collection<Mutation> viewMutations = view.getValue().createMutations(update, temporalRows, false);
-                if (viewMutations != null && !viewMutations.isEmpty())
-                {
-                    if (mutations == null)
-                        mutations = Lists.newLinkedList();
-                    mutations.addAll(viewMutations);
-                }
-            }
-        }
-
-        if (mutations != null)
-            StorageProxy.mutateMV(update.partitionKey().getKey(), mutations, writeCommitLog, baseComplete);
-    }
-
     public boolean updatesAffectView(Collection<? extends IMutation> mutations, boolean coordinatorBatchlog)
     {
         if (coordinatorBatchlog && !enableCoordinatorBatchlog)
@@ -156,25 +74,22 @@ public class ViewManager
 
         for (IMutation mutation : mutations)
         {
-            for (PartitionUpdate cf : mutation.getPartitionUpdates())
+            for (PartitionUpdate update : mutation.getPartitionUpdates())
             {
-                assert keyspace.getName().equals(cf.metadata().ksName);
+                assert keyspace.getName().equals(update.metadata().ksName);
 
                 if (coordinatorBatchlog && keyspace.getReplicationStrategy().getReplicationFactor() == 1)
                     continue;
 
-                for (View view : allViews())
-                {
-                    if (view.updateAffectsView(cf))
-                        return true;
-                }
+                if (!forTable(update.metadata()).updatedViews(update).isEmpty())
+                    return true;
             }
         }
 
         return false;
     }
 
-    public Iterable<View> allViews()
+    private Iterable<View> allViews()
     {
         return viewsByName.values();
     }
@@ -222,7 +137,7 @@ public class ViewManager
     public void addView(ViewDefinition definition)
     {
         View view = new View(definition, keyspace.getColumnFamilyStore(definition.baseTableId));
-        forTable(view.getDefinition().baseTableId).addView(view);
+        forTable(view.getDefinition().baseTableMetadata()).add(view);
         viewsByName.put(definition.viewName, view);
     }
 
@@ -233,7 +148,7 @@ public class ViewManager
         if (view == null)
             return;
 
-        forTable(view.getDefinition().baseTableId).removeView(name);
+        forTable(view.getDefinition().baseTableMetadata()).removeByName(name);
         SystemKeyspace.setViewRemoved(keyspace.getName(), view.name);
     }
 
@@ -243,17 +158,18 @@ public class ViewManager
             view.build();
     }
 
-    public ForStore forTable(UUID baseId)
+    public TableViews forTable(CFMetaData metadata)
     {
-        ForStore forStore = viewManagersByStore.get(baseId);
-        if (forStore == null)
+        UUID baseId = metadata.cfId;
+        TableViews views = viewsByBaseTable.get(baseId);
+        if (views == null)
         {
-            forStore = new ForStore();
-            ForStore previous = viewManagersByStore.put(baseId, forStore);
+            views = new TableViews(metadata);
+            TableViews previous = viewsByBaseTable.putIfAbsent(baseId, views);
             if (previous != null)
-                forStore = previous;
+                views = previous;
         }
-        return forStore;
+        return views;
     }
 
     public static Lock acquireLockFor(ByteBuffer key)

http://git-wip-us.apache.org/repos/asf/cassandra/blob/86ba2274/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
new file mode 100644
index 0000000..af025cb
--- /dev/null
+++ b/src/java/org/apache/cassandra/db/view/ViewUpdateGenerator.java
@@ -0,0 +1,549 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.cassandra.db.view;
+
+import java.nio.ByteBuffer;
+import java.util.*;
+
+import com.google.common.collect.Iterators;
+import com.google.common.collect.PeekingIterator;
+
+import org.apache.cassandra.config.CFMetaData;
+import org.apache.cassandra.config.ColumnDefinition;
+import org.apache.cassandra.config.ViewDefinition;
+import org.apache.cassandra.db.*;
+import org.apache.cassandra.db.rows.*;
+import org.apache.cassandra.db.partitions.*;
+import org.apache.cassandra.db.marshal.AbstractType;
+import org.apache.cassandra.db.marshal.CompositeType;
+
+/**
+ * Creates the updates to apply to a view given the existing rows in the base
+ * table and the updates that we're applying to them (this handles updates
+ * on a single partition only).
+ *
+ * This class is used by passing the updates made to the base table to
+ * {@link #addBaseTableUpdate} and calling {@link #generateViewUpdates} once all updates have
+ * been handled to get the resulting view mutations.
+ */
+public class ViewUpdateGenerator
+{
+    private final View view;
+    private final int nowInSec;
+
+    private final CFMetaData baseMetadata;
+    private final DecoratedKey baseDecoratedKey;
+    private final ByteBuffer[] basePartitionKey;
+
+    private final CFMetaData viewMetadata;
+
+    private final Map<DecoratedKey, PartitionUpdate> updates = new HashMap<>();
+
+    // Reused internally to build a new entry
+    private final ByteBuffer[] currentViewEntryPartitionKey;
+    private final Row.Builder currentViewEntryBuilder;
+
+    /**
+     * The type of type update action to perform to the view for a given base table
+     * update.
+     */
+    private enum UpdateAction
+    {
+        NONE,            // There was no view entry and none should be added
+        NEW_ENTRY,       // There was no entry but there is one post-update
+        DELETE_OLD,      // There was an entry but there is nothing after update
+        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}.
+     *
+     * @param view the view for which this will be building updates for.
+     * @param basePartitionKey the partition key for the base table partition for which
+     * we'll handle updates for.
+     * @param nowInSec the current time in seconds. Used to decide if data are live or not
+     * and as base reference for new deletions.
+     */
+    public ViewUpdateGenerator(View view, DecoratedKey basePartitionKey, int nowInSec)
+    {
+        this.view = view;
+        this.nowInSec = nowInSec;
+
+        this.baseMetadata = view.getDefinition().baseTableMetadata();
+        this.baseDecoratedKey = basePartitionKey;
+        this.basePartitionKey = extractKeyComponents(basePartitionKey, baseMetadata.getKeyValidator());
+
+        this.viewMetadata = view.getDefinition().metadata;
+
+        this.currentViewEntryPartitionKey = new ByteBuffer[viewMetadata.partitionKeyColumns().size()];
+        this.currentViewEntryBuilder = BTreeRow.sortedBuilder();
+    }
+
+    private static ByteBuffer[] extractKeyComponents(DecoratedKey partitionKey, AbstractType<?> type)
+    {
+        return type instanceof CompositeType
+             ? ((CompositeType)type).split(partitionKey.getKey())
+             : new ByteBuffer[]{ partitionKey.getKey() };
+    }
+
+    /**
+     * Adds to this generator the updates to be made to the view given a base table row
+     * before and after an update.
+     *
+     * @param existingBaseRow the base table row as it is before an update.
+     * @param mergedBaseRow the base table row after the update is applied (note that
+     * this is not just the new update, but rather the resulting row).
+     */
+    public void addBaseTableUpdate(Row existingBaseRow, Row mergedBaseRow)
+    {
+        switch (updateAction(existingBaseRow, mergedBaseRow))
+        {
+            case NONE:
+                return;
+            case NEW_ENTRY:
+                createEntry(mergedBaseRow);
+                return;
+            case DELETE_OLD:
+                deleteOldEntry(existingBaseRow);
+                return;
+            case UPDATE_EXISTING:
+                updateEntry(existingBaseRow, mergedBaseRow);
+                return;
+            case SWITCH_ENTRY:
+                createEntry(mergedBaseRow);
+                deleteOldEntry(existingBaseRow);
+                return;
+        }
+    }
+
+    /**
+     * Returns the updates that needs to be done to the view given the base table updates
+     * passed to {@link #generateViewMutations}.
+     *
+     * @return the updates to do to the view.
+     */
+    public Collection<PartitionUpdate> generateViewUpdates()
+    {
+        return updates.values();
+    }
+
+    /**
+     * Compute which type of action needs to be performed to the view for a base table row
+     * before and after an update.
+     */
+    private UpdateAction updateAction(Row existingBaseRow, Row mergedBaseRow)
+    {
+        // Having existing empty is useful, it just means we'll insert a brand new entry for mergedBaseRow,
+        // but if we have no update at all, we shouldn't get there.
+        assert !mergedBaseRow.isEmpty();
+
+        // Note that none of the base PK columns will differ since we're intrinsically dealing
+        // with the same base row. So we have to check 3 things:
+        //   1) that the clustering doesn't have a null, which can happen for compact tables. If that's the case,
+        //      there is no corresponding entries.
+        //   2) if there is a column not part of the base PK in the view PK, whether it is changed by the update.
+        //   3) whether mergedBaseRow actually match the view SELECT filter
+
+        if (baseMetadata.isCompactTable())
+        {
+            Clustering clustering = mergedBaseRow.clustering();
+            for (int i = 0; i < clustering.size(); i++)
+            {
+                if (clustering.get(i) == null)
+                    return UpdateAction.NONE;
+            }
+        }
+
+        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.
+
+            // Note that we allow existingBaseRow to be null and treat it as empty (see MultiViewUpdateBuilder.generateViewsMutations).
+            boolean existingHasLiveData = existingBaseRow != null && existingBaseRow.hasLiveData(nowInSec);
+            boolean mergedHasLiveData = mergedBaseRow.hasLiveData(nowInSec);
+            return existingHasLiveData
+                 ? (mergedHasLiveData ? UpdateAction.UPDATE_EXISTING : UpdateAction.DELETE_OLD)
+                 : (mergedHasLiveData ? UpdateAction.NEW_ENTRY : UpdateAction.NONE);
+        }
+
+        ColumnDefinition baseColumn = view.baseNonPKColumnsInViewPK.get(0);
+        assert !baseColumn.isComplex() : "A complex column couldn't be part of the view PK";
+        Cell before = existingBaseRow == null ? null : existingBaseRow.getCell(baseColumn);
+        Cell after = mergedBaseRow.getCell(baseColumn);
+
+        // If the update didn't modified this column, the cells will be the same object so it's worth checking
+        if (before == after)
+            return before == null ? UpdateAction.NONE : UpdateAction.UPDATE_EXISTING;
+
+        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
+             : UpdateAction.SWITCH_ENTRY;
+    }
+
+    private boolean matchesViewFilter(Row baseRow)
+    {
+        return view.matchesViewFilter(baseDecoratedKey, baseRow, nowInSec);
+    }
+
+    private boolean isLive(Cell cell)
+    {
+        return cell != null && cell.isLive(nowInSec);
+    }
+
+    /**
+     * Creates a view entry corresponding to the provided base row.
+     * <p>
+     * This method checks that the base row does match the view filter before applying it.
+     */
+    private void createEntry(Row baseRow)
+    {
+        // Before create a new entry, make sure it matches the view filter
+        if (!matchesViewFilter(baseRow))
+            return;
+
+        startNewUpdate(baseRow);
+        currentViewEntryBuilder.addPrimaryKeyLivenessInfo(computeLivenessInfoForEntry(baseRow));
+        currentViewEntryBuilder.addRowDeletion(baseRow.deletion());
+
+        for (ColumnData data : baseRow)
+        {
+            ColumnDefinition viewColumn = view.getViewColumn(data.column());
+            // If that base table column is not denormalized in the view, we had nothing to do.
+            // Alose, if it's part of the view PK it's already been taken into account in the clustering.
+            if (viewColumn == null || viewColumn.isPrimaryKeyColumn())
+                continue;
+
+            addColumnData(viewColumn, data);
+        }
+
+        submitUpdate();
+    }
+
+    /**
+     * Creates the updates to apply to the existing view entry given the base table row before
+     * and after the update, assuming that the update hasn't changed to which view entry the
+     * row correspond (that is, we know the columns composing the view PK haven't changed).
+     * <p>
+     * This method checks that the base row (before and after) does match the view filter before
+     * applying anything.
+     */
+    private void updateEntry(Row existingBaseRow, Row mergedBaseRow)
+    {
+        // While we know existingBaseRow and mergedBaseRow are corresponding to the same view entry,
+        // they may not match the view filter.
+        if (!matchesViewFilter(existingBaseRow))
+        {
+            createEntry(mergedBaseRow);
+            return;
+        }
+        if (!matchesViewFilter(mergedBaseRow))
+        {
+            deleteOldEntryInternal(existingBaseRow);
+            return;
+        }
+
+        startNewUpdate(mergedBaseRow);
+
+        // In theory, it may be the PK liveness and row deletion hasn't been change by the update
+        // and we could condition the 2 additions below. In practice though, it's as fast (if not
+        // faster) to compute those info than to check if they have changed so we keep it simple.
+        currentViewEntryBuilder.addPrimaryKeyLivenessInfo(computeLivenessInfoForEntry(mergedBaseRow));
+        currentViewEntryBuilder.addRowDeletion(mergedBaseRow.deletion());
+
+        // 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
+        // mergedBaseRow (note that including more cells than we strictly should isn't a problem
+        // for correction, so even if the code change and pointer equality don't work anymore, it'll
+        // only a slightly inefficiency which we can fix then).
+        // Note: we could alternatively use Rows.diff() for this, but because it is a bit more generic
+        // than what we need here, it's also a bit less efficient (it allocates more in particular),
+        // and this might be called a lot of time for view updates. So, given that this is not a whole
+        // lot of code anyway, it's probably doing the diff manually.
+        PeekingIterator<ColumnData> existingIter = Iterators.peekingIterator(existingBaseRow.iterator());
+        for (ColumnData mergedData : mergedBaseRow)
+        {
+            ColumnDefinition baseColumn = mergedData.column();
+            ColumnDefinition viewColumn = view.getViewColumn(baseColumn);
+            // If that base table column is not denormalized in the view, we had nothing to do.
+            // Alose, if it's part of the view PK it's already been taken into account in the clustering.
+            if (viewColumn == null || viewColumn.isPrimaryKeyColumn())
+                continue;
+
+            ColumnData existingData = null;
+            // Find if there is data for that column in the existing row
+            while (existingIter.hasNext())
+            {
+                int cmp = baseColumn.compareTo(existingIter.peek().column());
+                if (cmp < 0)
+                    break;
+
+                ColumnData next = existingIter.next();
+                if (cmp == 0)
+                {
+                    existingData = next;
+                    break;
+                }
+            }
+
+            if (existingData == null)
+            {
+                addColumnData(viewColumn, mergedData);
+                continue;
+            }
+
+            if (mergedData == existingData)
+                continue;
+
+            if (baseColumn.isComplex())
+            {
+                ComplexColumnData mergedComplexData = (ComplexColumnData)mergedData;
+                ComplexColumnData existingComplexData = (ComplexColumnData)existingData;
+                if (mergedComplexData.complexDeletion().supersedes(existingComplexData.complexDeletion()))
+                    currentViewEntryBuilder.addComplexDeletion(viewColumn, mergedComplexData.complexDeletion());
+
+                PeekingIterator<Cell> existingCells = Iterators.peekingIterator(existingComplexData.iterator());
+                for (Cell mergedCell : mergedComplexData)
+                {
+                    Cell existingCell = null;
+                    // Find if there is corresponding cell in the existing row
+                    while (existingCells.hasNext())
+                    {
+                        int cmp = baseColumn.cellPathComparator().compare(mergedCell.path(), existingCells.peek().path());
+                        if (cmp > 0)
+                            break;
+
+                        Cell next = existingCells.next();
+                        if (cmp == 0)
+                        {
+                            existingCell = next;
+                            break;
+                        }
+                    }
+
+                    if (mergedCell != existingCell)
+                        addCell(viewColumn, mergedCell);
+                }
+            }
+            else
+            {
+                // Note that we've already eliminated the case where merged == existing
+                addCell(viewColumn, (Cell)mergedData);
+            }
+        }
+
+        submitUpdate();
+    }
+
+    /**
+     * Deletes the view entry corresponding to the provided base row.
+     * <p>
+     * This method checks that the base row does match the view filter before bothering.
+     */
+    private void deleteOldEntry(Row existingBaseRow)
+    {
+        // 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);
+    }
+
+    private void deleteOldEntryInternal(Row existingBaseRow)
+    {
+        startNewUpdate(existingBaseRow);
+        DeletionTime dt = new DeletionTime(computeTimestampForEntryDeletion(existingBaseRow), nowInSec);
+        currentViewEntryBuilder.addRowDeletion(Row.Deletion.shadowable(dt));
+        submitUpdate();
+    }
+
+    /**
+     * Computes the partition key and clustering for a new view entry, and setup the internal
+     * row builder for the new row.
+     *
+     * This assumes that there is corresponding entry, i.e. no values for the partition key and
+     * clustering are null (since we have eliminated that case through updateAction).
+     */
+    private void startNewUpdate(Row baseRow)
+    {
+        ByteBuffer[] clusteringValues = new ByteBuffer[viewMetadata.clusteringColumns().size()];
+        for (ColumnDefinition viewColumn : viewMetadata.primaryKeyColumns())
+        {
+            ColumnDefinition baseColumn = view.getBaseColumn(viewColumn);
+            ByteBuffer value = getValueForPK(baseColumn, baseRow);
+            if (viewColumn.isPartitionKey())
+                currentViewEntryPartitionKey[viewColumn.position()] = value;
+            else
+                clusteringValues[viewColumn.position()] = value;
+        }
+
+        currentViewEntryBuilder.newRow(new Clustering(clusteringValues));
+    }
+
+    private LivenessInfo computeLivenessInfoForEntry(Row baseRow)
+    {
+        /*
+         * We need to compute both the timestamp and expiration.
+         *
+         * 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())
+        {
+            int ttl = baseLiveness.ttl();
+            int expirationTime = baseLiveness.localExpirationTime();
+            for (Cell cell : baseRow.cells())
+            {
+                if (cell.ttl() > ttl)
+                {
+                    ttl = cell.ttl();
+                    expirationTime = cell.localDeletionTime();
+                }
+            }
+            return ttl == baseLiveness.ttl()
+                 ? baseLiveness
+                 : LivenessInfo.create(baseLiveness.timestamp(), ttl, expirationTime);
+        }
+
+        ColumnDefinition baseColumn = view.baseNonPKColumnsInViewPK.get(0);
+        Cell cell = baseRow.getCell(baseColumn);
+        assert isLive(cell) : "We shouldn't have got there is the base row had no associated entry";
+
+        long timestamp = Math.max(baseLiveness.timestamp(), cell.timestamp());
+        return LivenessInfo.create(timestamp, cell.ttl(), cell.localDeletionTime());
+    }
+
+    private long computeTimestampForEntryDeletion(Row baseRow)
+    {
+        // 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)
+        {
+            if (!view.getDefinition().includes(data.column().name))
+                continue;
+
+            timestamp = Math.max(timestamp, data.maxTimestamp());
+        }
+        return timestamp;
+    }
+
+    private void addColumnData(ColumnDefinition viewColumn, ColumnData baseTableData)
+    {
+        assert viewColumn.isComplex() == baseTableData.column().isComplex();
+        if (!viewColumn.isComplex())
+        {
+            addCell(viewColumn, (Cell)baseTableData);
+            return;
+        }
+
+        ComplexColumnData complexData = (ComplexColumnData)baseTableData;
+        currentViewEntryBuilder.addComplexDeletion(viewColumn, complexData.complexDeletion());
+        for (Cell cell : complexData)
+            addCell(viewColumn, cell);
+    }
+
+    private void addCell(ColumnDefinition viewColumn, Cell baseTableCell)
+    {
+        assert !viewColumn.isPrimaryKeyColumn();
+        currentViewEntryBuilder.addCell(baseTableCell.withUpdatedColumn(viewColumn));
+    }
+
+    /**
+     * Finish building the currently updated view entry and add it to the other built
+     * updates.
+     */
+    private void submitUpdate()
+    {
+        Row row = currentViewEntryBuilder.build();
+        // I'm not sure we can reach there is there is nothing is updated, but adding an empty row breaks things
+        // and it costs us nothing to be prudent here.
+        if (row.isEmpty())
+            return;
+
+        DecoratedKey partitionKey = makeCurrentPartitionKey();
+        PartitionUpdate update = updates.get(partitionKey);
+        if (update == null)
+        {
+            // We can't really know which columns of the view will be updated nor how many row will be updated for this key
+            // so we rely on hopefully sane defaults.
+            update = new PartitionUpdate(viewMetadata, partitionKey, viewMetadata.partitionColumns(), 4);
+            updates.put(partitionKey, update);
+        }
+        update.add(row);
+    }
+
+    private DecoratedKey makeCurrentPartitionKey()
+    {
+        ByteBuffer rawKey = viewMetadata.partitionKeyColumns().size() == 1
+                          ? currentViewEntryPartitionKey[0]
+                          : CompositeType.build(currentViewEntryPartitionKey);
+
+        return viewMetadata.decorateKey(rawKey);
+    }
+
+    private ByteBuffer getValueForPK(ColumnDefinition column, Row row)
+    {
+        switch (column.kind)
+        {
+            case PARTITION_KEY:
+                return basePartitionKey[column.position()];
+            case CLUSTERING:
+                return row.clustering().get(column.position());
+            default:
+                // This shouldn't NPE as we shouldn't get there if the value can be null (or there is a bug in updateAction())
+                return row.getCell(column).value();
+        }
+    }
+}

http://git-wip-us.apache.org/repos/asf/cassandra/blob/86ba2274/test/unit/org/apache/cassandra/cql3/ViewTest.java
----------------------------------------------------------------------
diff --git a/test/unit/org/apache/cassandra/cql3/ViewTest.java b/test/unit/org/apache/cassandra/cql3/ViewTest.java
index ac4becb..4a1dc07 100644
--- a/test/unit/org/apache/cassandra/cql3/ViewTest.java
+++ b/test/unit/org/apache/cassandra/cql3/ViewTest.java
@@ -98,7 +98,7 @@ public class ViewTest extends CQLTester
     @Test
     public void testPartitionTombstone() throws Throwable
     {
-        createTable("CREATE TABLE %s (k1 int, c1 int , val int, PRIMARY KEY (k1))");
+        createTable("CREATE TABLE %s (k1 int, c1 int , val int, PRIMARY KEY (k1, c1))");
 
         execute("USE " + keyspace());
         executeNet(protocolVersion, "USE " + keyspace());
@@ -108,8 +108,8 @@ public class ViewTest extends CQLTester
         updateView("INSERT INTO %s (k1, c1, val) VALUES (1, 2, 200)");
         updateView("INSERT INTO %s (k1, c1, val) VALUES (1, 3, 300)");
 
-        Assert.assertEquals(1, execute("select * from %s").size());
-        Assert.assertEquals(1, execute("select * from view1").size());
+        Assert.assertEquals(2, execute("select * from %s").size());
+        Assert.assertEquals(2, execute("select * from view1").size());
 
         updateView("DELETE FROM %s WHERE k1 = 1");
 
@@ -814,18 +814,58 @@ public class ViewTest extends CQLTester
 
         createView("mv", "CREATE MATERIALIZED VIEW %s AS SELECT * FROM %%s WHERE c IS NOT NULL AND a IS NOT NULL AND b IS NOT NULL PRIMARY KEY (c, a, b)");
 
-        updateView("INSERT INTO %s (a, b, c, d) VALUES (?, ?, ?, ?) USING TTL 5", 1, 1, 1, 1);
+        updateView("INSERT INTO %s (a, b, c, d) VALUES (?, ?, ?, ?) USING TTL 3", 1, 1, 1, 1);
 
-        Thread.sleep(TimeUnit.SECONDS.toMillis(3));
+        Thread.sleep(TimeUnit.SECONDS.toMillis(1));
         updateView("INSERT INTO %s (a, b, c) VALUES (?, ?, ?)", 1, 1, 2);
 
-        Thread.sleep(TimeUnit.SECONDS.toMillis(3));
+        Thread.sleep(TimeUnit.SECONDS.toMillis(5));
         List<Row> results = executeNet(protocolVersion, "SELECT d FROM mv WHERE c = 2 AND a = 1 AND b = 1").all();
         Assert.assertEquals(1, results.size());
         Assert.assertTrue("There should be a null result given back due to ttl expiry", results.get(0).isNull(0));
     }
 
     @Test
+    public void ttlExpirationTest() throws Throwable
+    {
+        createTable("CREATE TABLE %s (" +
+                    "a int," +
+                    "b int," +
+                    "c int," +
+                    "d int," +
+                    "PRIMARY KEY (a, b))");
+
+        executeNet(protocolVersion, "USE " + keyspace());
+
+        createView("mv", "CREATE MATERIALIZED VIEW %s AS SELECT * FROM %%s WHERE c IS NOT NULL AND a IS NOT NULL AND b IS NOT NULL PRIMARY KEY (c, a, b)");
+
+        updateView("INSERT INTO %s (a, b, c, d) VALUES (?, ?, ?, ?) USING TTL 3", 1, 1, 1, 1);
+
+        Thread.sleep(TimeUnit.SECONDS.toMillis(4));
+        Assert.assertEquals(0, executeNet(protocolVersion, "SELECT * FROM mv WHERE c = 1 AND a = 1 AND b = 1").all().size());
+    }
+
+    @Test
+    public void rowDeletionTest() throws Throwable
+    {
+        createTable("CREATE TABLE %s (" +
+                    "a int," +
+                    "b int," +
+                    "c int," +
+                    "d int," +
+                    "PRIMARY KEY (a, b))");
+
+        executeNet(protocolVersion, "USE " + keyspace());
+
+        createView("mv", "CREATE MATERIALIZED VIEW %s AS SELECT * FROM %%s WHERE c IS NOT NULL AND a IS NOT NULL AND b IS NOT NULL PRIMARY KEY (c, a, b)");
+
+        String table = keyspace() + "." + currentTable();
+        updateView("DELETE FROM " + table + " USING TIMESTAMP 6 WHERE a = 1 AND b = 1;");
+        updateView("INSERT INTO %s (a, b, c, d) VALUES (?, ?, ?, ?) USING TIMESTAMP 3", 1, 1, 1, 1);
+        Assert.assertEquals(0, executeNet(protocolVersion, "SELECT * FROM mv WHERE c = 1 AND a = 1 AND b = 1").all().size());
+    }
+
+    @Test
     public void conflictingTimestampTest() throws Throwable
     {
         createTable("CREATE TABLE %s (" +

http://git-wip-us.apache.org/repos/asf/cassandra/blob/86ba2274/test/unit/org/apache/cassandra/db/rows/RowsTest.java
----------------------------------------------------------------------
diff --git a/test/unit/org/apache/cassandra/db/rows/RowsTest.java b/test/unit/org/apache/cassandra/db/rows/RowsTest.java
index dede867..b47bea2 100644
--- a/test/unit/org/apache/cassandra/db/rows/RowsTest.java
+++ b/test/unit/org/apache/cassandra/db/rows/RowsTest.java
@@ -239,7 +239,8 @@ public class RowsTest
                                                       BufferCell.live(kcvm, m, secondToTs(now), BB1, CellPath.create(BB1)),
                                                       BufferCell.live(kcvm, m, secondToTs(now), BB2, CellPath.create(BB2)));
         expectedCells.forEach(originalBuilder::addCell);
-        Row.Deletion rowDeletion = new Row.Deletion(new DeletionTime(ts, now), false);
+        // We need to use ts-1 so the deletion doesn't shadow what we've created
+        Row.Deletion rowDeletion = new Row.Deletion(new DeletionTime(ts-1, now), false);
         originalBuilder.addRowDeletion(rowDeletion);
 
         RowBuilder builder = new RowBuilder();
@@ -267,7 +268,8 @@ public class RowsTest
                                                       BufferCell.live(kcvm, m, ts, BB1, CellPath.create(BB1)),
                                                       BufferCell.live(kcvm, m, ts, BB2, CellPath.create(BB2)));
         expectedCells.forEach(builder::addCell);
-        Row.Deletion rowDeletion = new Row.Deletion(new DeletionTime(ts, now), false);
+        // We need to use ts-1 so the deletion doesn't shadow what we've created
+        Row.Deletion rowDeletion = new Row.Deletion(new DeletionTime(ts-1, now), false);
         builder.addRowDeletion(rowDeletion);
 
         StatsCollector collector = new StatsCollector();


[9/9] cassandra git commit: Merge branch 'cassandra-3.7' into trunk

Posted by sl...@apache.org.
Merge branch 'cassandra-3.7' into trunk


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

Branch: refs/heads/trunk
Commit: 89a645ac4ca63114d74dedc2e94a869f769b15a2
Parents: 1dd33ec a87fd71
Author: Sylvain Lebresne <sy...@datastax.com>
Authored: Fri May 6 13:47:20 2016 +0200
Committer: Sylvain Lebresne <sy...@datastax.com>
Committed: Fri May 6 13:47:20 2016 +0200

----------------------------------------------------------------------
 .../org/apache/cassandra/config/CFMetaData.java |   6 +
 .../apache/cassandra/config/ViewDefinition.java |   1 -
 .../cql3/statements/CreateViewStatement.java    |   4 +-
 .../cql3/statements/SelectStatement.java        |  41 +-
 .../apache/cassandra/db/ColumnFamilyStore.java  |   6 +-
 src/java/org/apache/cassandra/db/Keyspace.java  |   2 +-
 .../db/SinglePartitionReadCommand.java          |  33 +
 src/java/org/apache/cassandra/db/Slices.java    |   7 +
 .../apache/cassandra/db/filter/RowFilter.java   |  24 +
 .../SingletonUnfilteredPartitionIterator.java   |   3 +-
 .../apache/cassandra/db/rows/AbstractCell.java  |   5 +
 .../org/apache/cassandra/db/rows/BTreeRow.java  |  35 +-
 .../apache/cassandra/db/rows/BufferCell.java    |   5 +
 src/java/org/apache/cassandra/db/rows/Cell.java |   2 +
 .../apache/cassandra/db/rows/ColumnData.java    |   2 +
 .../cassandra/db/rows/ComplexColumnData.java    |   8 +
 .../apache/cassandra/db/rows/NativeCell.java    |   5 +
 src/java/org/apache/cassandra/db/rows/Row.java  |  35 +-
 .../cassandra/db/rows/RowDiffListener.java      |   2 +-
 .../db/rows/UnfilteredRowIterators.java         |   2 +-
 .../apache/cassandra/db/view/TableViews.java    | 481 ++++++++++++++
 .../apache/cassandra/db/view/TemporalRow.java   | 601 ------------------
 src/java/org/apache/cassandra/db/view/View.java | 629 ++-----------------
 .../apache/cassandra/db/view/ViewBuilder.java   |  38 +-
 .../apache/cassandra/db/view/ViewManager.java   | 146 +----
 .../cassandra/db/view/ViewUpdateGenerator.java  | 549 ++++++++++++++++
 .../org/apache/cassandra/cql3/ViewTest.java     |  52 +-
 .../org/apache/cassandra/db/rows/RowsTest.java  |   6 +-
 28 files changed, 1401 insertions(+), 1329 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/cassandra/blob/89a645ac/src/java/org/apache/cassandra/cql3/statements/SelectStatement.java
----------------------------------------------------------------------

http://git-wip-us.apache.org/repos/asf/cassandra/blob/89a645ac/src/java/org/apache/cassandra/db/Slices.java
----------------------------------------------------------------------

http://git-wip-us.apache.org/repos/asf/cassandra/blob/89a645ac/src/java/org/apache/cassandra/db/rows/BTreeRow.java
----------------------------------------------------------------------
diff --cc src/java/org/apache/cassandra/db/rows/BTreeRow.java
index 63aa157,0eed9e1..c699634
--- a/src/java/org/apache/cassandra/db/rows/BTreeRow.java
+++ b/src/java/org/apache/cassandra/db/rows/BTreeRow.java
@@@ -686,7 -704,11 +714,12 @@@ public class BTreeRow extends AbstractR
          public void addCell(Cell cell)
          {
              assert cell.column().isStatic() == (clustering == Clustering.STATIC_CLUSTERING) : "Column is " + cell.column() + ", clustering = " + clustering;
++
+             // In practice, only unsorted builder have to deal with shadowed cells, but it doesn't cost us much to deal with it unconditionally in this case
+             if (deletion.deletes(cell))
+                 return;
+ 
 -            cells.add(cell);
 +            getCells().add(cell);
              hasComplex |= cell.column.isComplex();
          }
  

http://git-wip-us.apache.org/repos/asf/cassandra/blob/89a645ac/src/java/org/apache/cassandra/db/rows/ComplexColumnData.java
----------------------------------------------------------------------


[2/9] cassandra git commit: Refactor MV code

Posted by sl...@apache.org.
Refactor MV code

patch by slebresne; reviewed by carlyeks for CASSANDRA-11475


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

Branch: refs/heads/cassandra-3.0
Commit: 86ba227477b9f8595eb610ecaf950cfbc29dd36b
Parents: c19066e
Author: Sylvain Lebresne <sy...@datastax.com>
Authored: Fri Mar 11 14:19:38 2016 +0100
Committer: Sylvain Lebresne <sy...@datastax.com>
Committed: Fri May 6 13:41:41 2016 +0200

----------------------------------------------------------------------
 CHANGES.txt                                     |   3 +
 .../org/apache/cassandra/config/CFMetaData.java |   6 +
 .../apache/cassandra/config/ViewDefinition.java |   1 -
 .../cql3/statements/CreateViewStatement.java    |   4 +-
 .../cql3/statements/SelectStatement.java        |  41 +-
 .../apache/cassandra/db/ColumnFamilyStore.java  |   6 +-
 src/java/org/apache/cassandra/db/Keyspace.java  |   2 +-
 .../db/SinglePartitionReadCommand.java          |  33 +
 src/java/org/apache/cassandra/db/Slices.java    |   7 +
 .../apache/cassandra/db/filter/RowFilter.java   |  24 +
 .../SingletonUnfilteredPartitionIterator.java   |   3 +-
 .../apache/cassandra/db/rows/AbstractCell.java  |   5 +
 .../org/apache/cassandra/db/rows/BTreeRow.java  |  34 +-
 .../apache/cassandra/db/rows/BufferCell.java    |   5 +
 src/java/org/apache/cassandra/db/rows/Cell.java |   2 +
 .../apache/cassandra/db/rows/ColumnData.java    |   2 +
 .../cassandra/db/rows/ComplexColumnData.java    |   8 +
 src/java/org/apache/cassandra/db/rows/Row.java  |  35 +-
 .../cassandra/db/rows/RowDiffListener.java      |   2 +-
 .../db/rows/UnfilteredRowIterators.java         |   2 +-
 .../apache/cassandra/db/view/TableViews.java    | 481 ++++++++++++++
 .../apache/cassandra/db/view/TemporalRow.java   | 610 ------------------
 src/java/org/apache/cassandra/db/view/View.java | 629 ++-----------------
 .../apache/cassandra/db/view/ViewBuilder.java   |  38 +-
 .../apache/cassandra/db/view/ViewManager.java   | 146 +----
 .../cassandra/db/view/ViewUpdateGenerator.java  | 549 ++++++++++++++++
 .../org/apache/cassandra/cql3/ViewTest.java     |  52 +-
 .../org/apache/cassandra/db/rows/RowsTest.java  |   6 +-
 28 files changed, 1399 insertions(+), 1337 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/cassandra/blob/86ba2274/CHANGES.txt
----------------------------------------------------------------------
diff --git a/CHANGES.txt b/CHANGES.txt
index 0679e11..3a49f6a 100644
--- a/CHANGES.txt
+++ b/CHANGES.txt
@@ -1,3 +1,6 @@
+3.0.7
+ * Refactor Materialized View code (CASSANDRA-11475)
+
 3.0.6
  * Disallow creating view with a static column (CASSANDRA-11602)
  * Reduce the amount of object allocations caused by the getFunctions methods (CASSANDRA-11593)

http://git-wip-us.apache.org/repos/asf/cassandra/blob/86ba2274/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 79cd779..e263697 100644
--- a/src/java/org/apache/cassandra/config/CFMetaData.java
+++ b/src/java/org/apache/cassandra/config/CFMetaData.java
@@ -31,6 +31,7 @@ import com.google.common.annotations.VisibleForTesting;
 import com.google.common.base.MoreObjects;
 import com.google.common.base.Objects;
 import com.google.common.collect.ImmutableSet;
+import com.google.common.collect.Iterables;
 import com.google.common.collect.Sets;
 import org.apache.commons.lang3.ArrayUtils;
 import org.apache.commons.lang3.builder.HashCodeBuilder;
@@ -612,6 +613,11 @@ public final class CFMetaData
         };
     }
 
+    public Iterable<ColumnDefinition> primaryKeyColumns()
+    {
+        return Iterables.concat(partitionKeyColumns, clusteringColumns);
+    }
+
     public List<ColumnDefinition> partitionKeyColumns()
     {
         return partitionKeyColumns;

http://git-wip-us.apache.org/repos/asf/cassandra/blob/86ba2274/src/java/org/apache/cassandra/config/ViewDefinition.java
----------------------------------------------------------------------
diff --git a/src/java/org/apache/cassandra/config/ViewDefinition.java b/src/java/org/apache/cassandra/config/ViewDefinition.java
index b29a8f9..5300f56 100644
--- a/src/java/org/apache/cassandra/config/ViewDefinition.java
+++ b/src/java/org/apache/cassandra/config/ViewDefinition.java
@@ -37,7 +37,6 @@ public class ViewDefinition
     public final UUID baseTableId;
     public final String baseTableName;
     public final boolean includeAllColumns;
-    // The order of partititon columns and clustering columns is important, so we cannot switch these two to sets
     public final CFMetaData metadata;
 
     public SelectStatement.RawStatement select;

http://git-wip-us.apache.org/repos/asf/cassandra/blob/86ba2274/src/java/org/apache/cassandra/cql3/statements/CreateViewStatement.java
----------------------------------------------------------------------
diff --git a/src/java/org/apache/cassandra/cql3/statements/CreateViewStatement.java b/src/java/org/apache/cassandra/cql3/statements/CreateViewStatement.java
index 45231b7..6446602 100644
--- a/src/java/org/apache/cassandra/cql3/statements/CreateViewStatement.java
+++ b/src/java/org/apache/cassandra/cql3/statements/CreateViewStatement.java
@@ -227,10 +227,10 @@ public class CreateViewStatement extends SchemaAlteringStatement
         // This is only used as an intermediate state; this is to catch whether multiple non-PK columns are used
         boolean hasNonPKColumn = false;
         for (ColumnIdentifier.Raw raw : partitionKeys)
-            hasNonPKColumn = getColumnIdentifier(cfm, basePrimaryKeyCols, hasNonPKColumn, raw, targetPartitionKeys, restrictions);
+            hasNonPKColumn |= getColumnIdentifier(cfm, basePrimaryKeyCols, hasNonPKColumn, raw, targetPartitionKeys, restrictions);
 
         for (ColumnIdentifier.Raw raw : clusteringKeys)
-            hasNonPKColumn = getColumnIdentifier(cfm, basePrimaryKeyCols, hasNonPKColumn, raw, targetClusteringColumns, restrictions);
+            hasNonPKColumn |= getColumnIdentifier(cfm, basePrimaryKeyCols, hasNonPKColumn, raw, targetClusteringColumns, restrictions);
 
         // We need to include all of the primary key columns from the base table in order to make sure that we do not
         // overwrite values in the view. We cannot support "collapsing" the base table into a smaller number of rows in

http://git-wip-us.apache.org/repos/asf/cassandra/blob/86ba2274/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 c3e13f4..0e33475 100644
--- a/src/java/org/apache/cassandra/cql3/statements/SelectStatement.java
+++ b/src/java/org/apache/cassandra/cql3/statements/SelectStatement.java
@@ -151,6 +151,15 @@ public class SelectStatement implements CQLStatement
         return builder.build();
     }
 
+    /**
+     * The columns to fetch internally for this SELECT statement (which can be more than the one selected by the
+     * user as it also include any restricted column in particular).
+     */
+    public ColumnFilter queriedColumns()
+    {
+        return queriedColumns;
+    }
+
     // Creates a simple select based on the given selection.
     // Note that the results select statement should not be used for actual queries, but only for processing already
     // queried data through processColumnFamily.
@@ -473,7 +482,29 @@ public class SelectStatement implements CQLStatement
     }
 
     /**
-     * Returns a read command that can be used internally to filter individual rows for materialized views.
+     * Returns the slices fetched by this SELECT, assuming an internal call (no bound values in particular).
+     * <p>
+     * Note that if the SELECT intrinsically selects rows by names, we convert them into equivalent slices for
+     * the purpose of this method. This is used for MVs to restrict what needs to be read when we want to read
+     * everything that could be affected by a given view (and so, if the view SELECT statement has restrictions
+     * on the clustering columns, we can restrict what we read).
+     */
+    public Slices clusteringIndexFilterAsSlices()
+    {
+        QueryOptions options = QueryOptions.forInternalCalls(Collections.emptyList());
+        ClusteringIndexFilter filter = makeClusteringIndexFilter(options);
+        if (filter instanceof ClusteringIndexSliceFilter)
+            return ((ClusteringIndexSliceFilter)filter).requestedSlices();
+
+        Slices.Builder builder = new Slices.Builder(cfm.comparator);
+        for (Clustering clustering: ((ClusteringIndexNamesFilter)filter).requestedRows())
+            builder.add(Slice.make(clustering));
+        return builder.build();
+    }
+
+    /**
+     * Returns a read command that can be used internally to query all the rows queried by this SELECT for a
+     * give key (used for materialized views).
      */
     public SinglePartitionReadCommand internalReadForView(DecoratedKey key, int nowInSec)
     {
@@ -483,6 +514,14 @@ public class SelectStatement implements CQLStatement
         return SinglePartitionReadCommand.create(cfm, nowInSec, queriedColumns, rowFilter, DataLimits.NONE, key, filter);
     }
 
+    /**
+     * The {@code RowFilter} for this SELECT, assuming an internal call (no bound values in particular).
+     */
+    public RowFilter rowFilterForInternalCalls()
+    {
+        return getRowFilter(QueryOptions.forInternalCalls(Collections.emptyList()));
+    }
+
     private ReadQuery getRangeCommand(QueryOptions options, DataLimits limit, int nowInSec) throws RequestValidationException
     {
         ClusteringIndexFilter clusteringIndexFilter = makeClusteringIndexFilter(options);

http://git-wip-us.apache.org/repos/asf/cassandra/blob/86ba2274/src/java/org/apache/cassandra/db/ColumnFamilyStore.java
----------------------------------------------------------------------
diff --git a/src/java/org/apache/cassandra/db/ColumnFamilyStore.java b/src/java/org/apache/cassandra/db/ColumnFamilyStore.java
index 919fed6..5b8de8f 100644
--- a/src/java/org/apache/cassandra/db/ColumnFamilyStore.java
+++ b/src/java/org/apache/cassandra/db/ColumnFamilyStore.java
@@ -48,7 +48,7 @@ import org.apache.cassandra.db.commitlog.ReplayPosition;
 import org.apache.cassandra.db.compaction.*;
 import org.apache.cassandra.db.filter.ClusteringIndexFilter;
 import org.apache.cassandra.db.filter.DataLimits;
-import org.apache.cassandra.db.view.ViewManager;
+import org.apache.cassandra.db.view.TableViews;
 import org.apache.cassandra.db.lifecycle.*;
 import org.apache.cassandra.db.partitions.CachedPartition;
 import org.apache.cassandra.db.partitions.PartitionUpdate;
@@ -198,7 +198,7 @@ public class ColumnFamilyStore implements ColumnFamilyStoreMBean
     private final AtomicInteger fileIndexGenerator = new AtomicInteger(0);
 
     public final SecondaryIndexManager indexManager;
-    public final ViewManager.ForStore viewManager;
+    public final TableViews viewManager;
 
     /* These are locally held copies to be changed from the config during runtime */
     private volatile DefaultValue<Integer> minCompactionThreshold;
@@ -373,7 +373,7 @@ public class ColumnFamilyStore implements ColumnFamilyStoreMBean
         maxCompactionThreshold = new DefaultValue<>(metadata.params.compaction.maxCompactionThreshold());
         crcCheckChance = new DefaultValue<>(metadata.params.crcCheckChance);
         indexManager = new SecondaryIndexManager(this);
-        viewManager = keyspace.viewManager.forTable(metadata.cfId);
+        viewManager = keyspace.viewManager.forTable(metadata);
         metric = new TableMetrics(this);
         fileIndexGenerator.set(generation);
         sampleLatencyNanos = DatabaseDescriptor.getReadRpcTimeout() / 2;

http://git-wip-us.apache.org/repos/asf/cassandra/blob/86ba2274/src/java/org/apache/cassandra/db/Keyspace.java
----------------------------------------------------------------------
diff --git a/src/java/org/apache/cassandra/db/Keyspace.java b/src/java/org/apache/cassandra/db/Keyspace.java
index 5783b41..273946e 100644
--- a/src/java/org/apache/cassandra/db/Keyspace.java
+++ b/src/java/org/apache/cassandra/db/Keyspace.java
@@ -484,7 +484,7 @@ public class Keyspace
                     try
                     {
                         Tracing.trace("Creating materialized view mutations from base table replica");
-                        viewManager.pushViewReplicaUpdates(upd, !isClReplay, baseComplete);
+                        viewManager.forTable(upd.metadata()).pushViewReplicaUpdates(upd, !isClReplay, baseComplete);
                     }
                     catch (Throwable t)
                     {

http://git-wip-us.apache.org/repos/asf/cassandra/blob/86ba2274/src/java/org/apache/cassandra/db/SinglePartitionReadCommand.java
----------------------------------------------------------------------
diff --git a/src/java/org/apache/cassandra/db/SinglePartitionReadCommand.java b/src/java/org/apache/cassandra/db/SinglePartitionReadCommand.java
index 14923b9..d5f2dc4 100644
--- a/src/java/org/apache/cassandra/db/SinglePartitionReadCommand.java
+++ b/src/java/org/apache/cassandra/db/SinglePartitionReadCommand.java
@@ -229,6 +229,39 @@ public class SinglePartitionReadCommand extends ReadCommand
         return create(metadata, nowInSec, metadata.decorateKey(key), slices);
     }
 
+    /**
+     * Creates a new single partition name command for the provided rows.
+     *
+     * @param metadata the table to query.
+     * @param nowInSec the time in seconds to use are "now" for this query.
+     * @param key the partition key for the partition to query.
+     * @param names the clustering for the rows to query.
+     *
+     * @return a newly created read command that queries the {@code names} in {@code key}. The returned query will
+     * query every columns (without limit or row filtering) and be in forward order.
+     */
+    public static SinglePartitionReadCommand create(CFMetaData metadata, int nowInSec, DecoratedKey key, NavigableSet<Clustering> names)
+    {
+        ClusteringIndexNamesFilter filter = new ClusteringIndexNamesFilter(names, false);
+        return SinglePartitionReadCommand.create(metadata, nowInSec, ColumnFilter.all(metadata), RowFilter.NONE, DataLimits.NONE, key, filter);
+    }
+
+    /**
+     * Creates a new single partition name command for the provided row.
+     *
+     * @param metadata the table to query.
+     * @param nowInSec the time in seconds to use are "now" for this query.
+     * @param key the partition key for the partition to query.
+     * @param name the clustering for the row to query.
+     *
+     * @return a newly created read command that queries {@code name} in {@code key}. The returned query will
+     * query every columns (without limit or row filtering).
+     */
+    public static SinglePartitionReadCommand create(CFMetaData metadata, int nowInSec, DecoratedKey key, Clustering name)
+    {
+        return create(metadata, nowInSec, key, FBUtilities.singleton(name, metadata.comparator));
+    }
+
     public SinglePartitionReadCommand copy()
     {
         return new SinglePartitionReadCommand(isDigestQuery(), digestVersion(), isForThrift(), metadata(), nowInSec(), columnFilter(), rowFilter(), limits(), partitionKey(), clusteringIndexFilter());

http://git-wip-us.apache.org/repos/asf/cassandra/blob/86ba2274/src/java/org/apache/cassandra/db/Slices.java
----------------------------------------------------------------------
diff --git a/src/java/org/apache/cassandra/db/Slices.java b/src/java/org/apache/cassandra/db/Slices.java
index 8fa9337..bb354a1 100644
--- a/src/java/org/apache/cassandra/db/Slices.java
+++ b/src/java/org/apache/cassandra/db/Slices.java
@@ -210,6 +210,13 @@ public abstract class Slices implements Iterable<Slice>
             return this;
         }
 
+        public Builder addAll(Slices slices)
+        {
+            for (Slice slice : slices)
+                add(slice);
+            return this;
+        }
+
         public int size()
         {
             return slices.size();

http://git-wip-us.apache.org/repos/asf/cassandra/blob/86ba2274/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 8060f23..11cfb87 100644
--- a/src/java/org/apache/cassandra/db/filter/RowFilter.java
+++ b/src/java/org/apache/cassandra/db/filter/RowFilter.java
@@ -122,6 +122,30 @@ public abstract class RowFilter implements Iterable<RowFilter.Expression>
     public abstract UnfilteredPartitionIterator filter(UnfilteredPartitionIterator iter, int nowInSec);
 
     /**
+     * Whether the provided row in the provided partition satisfies this filter.
+     *
+     * @param metadata the table metadata.
+     * @param partitionKey the partition key for partition to test.
+     * @param row the row to test.
+     * @param nowInSec the current time in seconds (to know what is live and what isn't).
+     * @return {@code true} if {@code row} in partition {@code partitionKey} satisfies this row filter.
+     */
+    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);
+        if (purged == null)
+            return expressions.isEmpty();
+
+        for (Expression e : expressions)
+        {
+            if (!e.isSatisfiedBy(metadata, partitionKey, purged))
+                return false;
+        }
+        return true;
+    }
+
+    /**
      * Returns true if all of the expressions within this filter that apply to the partition key are satisfied by
      * the given key, false otherwise.
      */

http://git-wip-us.apache.org/repos/asf/cassandra/blob/86ba2274/src/java/org/apache/cassandra/db/partitions/SingletonUnfilteredPartitionIterator.java
----------------------------------------------------------------------
diff --git a/src/java/org/apache/cassandra/db/partitions/SingletonUnfilteredPartitionIterator.java b/src/java/org/apache/cassandra/db/partitions/SingletonUnfilteredPartitionIterator.java
index bfa6690..1f966db 100644
--- a/src/java/org/apache/cassandra/db/partitions/SingletonUnfilteredPartitionIterator.java
+++ b/src/java/org/apache/cassandra/db/partitions/SingletonUnfilteredPartitionIterator.java
@@ -65,6 +65,7 @@ public class SingletonUnfilteredPartitionIterator implements UnfilteredPartition
 
     public void close()
     {
-        iter.close();
+        if (!returned)
+            iter.close();
     }
 }

http://git-wip-us.apache.org/repos/asf/cassandra/blob/86ba2274/src/java/org/apache/cassandra/db/rows/AbstractCell.java
----------------------------------------------------------------------
diff --git a/src/java/org/apache/cassandra/db/rows/AbstractCell.java b/src/java/org/apache/cassandra/db/rows/AbstractCell.java
index 00fc286..7e93c2e 100644
--- a/src/java/org/apache/cassandra/db/rows/AbstractCell.java
+++ b/src/java/org/apache/cassandra/db/rows/AbstractCell.java
@@ -74,6 +74,11 @@ public abstract class AbstractCell extends Cell
             column().validateCellPath(path());
     }
 
+    public long maxTimestamp()
+    {
+        return timestamp();
+    }
+
     @Override
     public boolean equals(Object other)
     {

http://git-wip-us.apache.org/repos/asf/cassandra/blob/86ba2274/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 e8667e0..ea1d9e0 100644
--- a/src/java/org/apache/cassandra/db/rows/BTreeRow.java
+++ b/src/java/org/apache/cassandra/db/rows/BTreeRow.java
@@ -322,6 +322,18 @@ public class BTreeRow extends AbstractRow
         return transformAndFilter(newInfo, newDeletion, (cd) -> cd.updateAllTimestamp(newTimestamp));
     }
 
+    public Row withRowDeletion(DeletionTime newDeletion)
+    {
+        // Note that:
+        //  - it is a contract with the caller that the new deletion shouldn't shadow anything in
+        //    the row, and so in particular it can't shadow the row deletion. So if there is a
+        //    already a row deletion we have nothing to do.
+        //  - we set the minLocalDeletionTime to MIN_VALUE because we know the deletion is live
+        return newDeletion.isLive() || !deletion.isLive()
+             ? this
+             : new BTreeRow(clustering, primaryKeyLivenessInfo, Deletion.regular(newDeletion), btree, Integer.MIN_VALUE);
+    }
+
     public Row purge(DeletionPurger purger, int nowInSec)
     {
         if (!hasDeletion(nowInSec))
@@ -566,6 +578,17 @@ public class BTreeRow extends AbstractRow
                 }
 
                 List<Object> buildFrom = Arrays.asList(cells).subList(lb, ub);
+                if (deletion != DeletionTime.LIVE)
+                {
+                    // Make sure we don't include any shadowed cells
+                    List<Object> filtered = new ArrayList<>(buildFrom.size());
+                    for (Object c : buildFrom)
+                    {
+                        if (((Cell)c).timestamp() >= deletion.markedForDeleteAt())
+                            filtered.add(c);
+                    }
+                    buildFrom = filtered;
+                }
                 Object[] btree = BTree.build(buildFrom, UpdateFunction.noOp());
                 return new ComplexColumnData(column, btree, deletion);
             }
@@ -621,17 +644,26 @@ public class BTreeRow extends AbstractRow
 
         public void addPrimaryKeyLivenessInfo(LivenessInfo info)
         {
-            this.primaryKeyLivenessInfo = info;
+            // The check is only required for unsorted builders, but it's worth the extra safety to have it unconditional
+            if (!deletion.deletes(info))
+                this.primaryKeyLivenessInfo = info;
         }
 
         public void addRowDeletion(Deletion deletion)
         {
             this.deletion = deletion;
+            // The check is only required for unsorted builders, but it's worth the extra safety to have it unconditional
+            if (deletion.deletes(primaryKeyLivenessInfo))
+                this.primaryKeyLivenessInfo = LivenessInfo.EMPTY;
         }
 
         public void addCell(Cell cell)
         {
             assert cell.column().isStatic() == (clustering == Clustering.STATIC_CLUSTERING) : "Column is " + cell.column() + ", clustering = " + clustering;
+            // In practice, only unsorted builder have to deal with shadowed cells, but it doesn't cost us much to deal with it unconditionally in this case
+            if (deletion.deletes(cell))
+                return;
+
             cells.add(cell);
             hasComplex |= cell.column.isComplex();
         }

http://git-wip-us.apache.org/repos/asf/cassandra/blob/86ba2274/src/java/org/apache/cassandra/db/rows/BufferCell.java
----------------------------------------------------------------------
diff --git a/src/java/org/apache/cassandra/db/rows/BufferCell.java b/src/java/org/apache/cassandra/db/rows/BufferCell.java
index 8912f59..0a2c528 100644
--- a/src/java/org/apache/cassandra/db/rows/BufferCell.java
+++ b/src/java/org/apache/cassandra/db/rows/BufferCell.java
@@ -133,6 +133,11 @@ public class BufferCell extends AbstractCell
         return path;
     }
 
+    public Cell withUpdatedColumn(ColumnDefinition newColumn)
+    {
+        return new BufferCell(newColumn, timestamp, ttl, localDeletionTime, value, path);
+    }
+
     public Cell withUpdatedValue(ByteBuffer newValue)
     {
         return new BufferCell(column, timestamp, ttl, localDeletionTime, newValue, path);

http://git-wip-us.apache.org/repos/asf/cassandra/blob/86ba2274/src/java/org/apache/cassandra/db/rows/Cell.java
----------------------------------------------------------------------
diff --git a/src/java/org/apache/cassandra/db/rows/Cell.java b/src/java/org/apache/cassandra/db/rows/Cell.java
index 73d9e44..b10ce06 100644
--- a/src/java/org/apache/cassandra/db/rows/Cell.java
+++ b/src/java/org/apache/cassandra/db/rows/Cell.java
@@ -129,6 +129,8 @@ public abstract class Cell extends ColumnData
      */
     public abstract CellPath path();
 
+    public abstract Cell withUpdatedColumn(ColumnDefinition newColumn);
+
     public abstract Cell withUpdatedValue(ByteBuffer newValue);
 
     public abstract Cell copy(AbstractAllocator allocator);

http://git-wip-us.apache.org/repos/asf/cassandra/blob/86ba2274/src/java/org/apache/cassandra/db/rows/ColumnData.java
----------------------------------------------------------------------
diff --git a/src/java/org/apache/cassandra/db/rows/ColumnData.java b/src/java/org/apache/cassandra/db/rows/ColumnData.java
index 84763e5..933da6a 100644
--- a/src/java/org/apache/cassandra/db/rows/ColumnData.java
+++ b/src/java/org/apache/cassandra/db/rows/ColumnData.java
@@ -82,4 +82,6 @@ public abstract class ColumnData
     public abstract ColumnData markCounterLocalToBeCleared();
 
     public abstract ColumnData purge(DeletionPurger purger, int nowInSec);
+
+    public abstract long maxTimestamp();
 }

http://git-wip-us.apache.org/repos/asf/cassandra/blob/86ba2274/src/java/org/apache/cassandra/db/rows/ComplexColumnData.java
----------------------------------------------------------------------
diff --git a/src/java/org/apache/cassandra/db/rows/ComplexColumnData.java b/src/java/org/apache/cassandra/db/rows/ComplexColumnData.java
index fab529b..d67d079 100644
--- a/src/java/org/apache/cassandra/db/rows/ComplexColumnData.java
+++ b/src/java/org/apache/cassandra/db/rows/ComplexColumnData.java
@@ -184,6 +184,14 @@ public class ComplexColumnData extends ColumnData implements Iterable<Cell>
         return transformAndFilter(newDeletion, (cell) -> (Cell) cell.updateAllTimestamp(newTimestamp));
     }
 
+    public long maxTimestamp()
+    {
+        long timestamp = complexDeletion.markedForDeleteAt();
+        for (Cell cell : this)
+            timestamp = Math.max(timestamp, cell.timestamp());
+        return timestamp;
+    }
+
     // This is the partner in crime of ArrayBackedRow.setValue. The exact warning apply. The short
     // version is: "don't use that method".
     void setValue(CellPath path, ByteBuffer value)

http://git-wip-us.apache.org/repos/asf/cassandra/blob/86ba2274/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 5f79a66..82c07a7 100644
--- a/src/java/org/apache/cassandra/db/rows/Row.java
+++ b/src/java/org/apache/cassandra/db/rows/Row.java
@@ -215,6 +215,18 @@ public interface Row extends Unfiltered, Collection<ColumnData>
      */
     public Row updateAllTimestamp(long newTimestamp);
 
+    /**
+     * Returns a copy of this row with the new deletion as row deletion if it is more recent
+     * than the current row deletion.
+     * <p>
+     * WARNING: this method <b>does not</b> check that nothing in the row is shadowed by the provided
+     * deletion and if that is the case, the created row will be <b>invalid</b>. It is thus up to the
+     * caller to verify that this is not the case and the only reasonable use case of this is probably
+     * when the row and the deletion comes from the same {@code UnfilteredRowIterator} since that gives
+     * use this guarantee.
+     */
+    public Row withRowDeletion(DeletionTime deletion);
+
     public int dataSize();
 
     public long unsharedHeapSizeExcludingData();
@@ -227,12 +239,15 @@ public interface Row extends Unfiltered, Collection<ColumnData>
      * A row deletion mostly consists of the time of said deletion, but there is 2 variants: shadowable
      * and regular row deletion.
      * <p>
-     * A shadowable row deletion only exists if the row timestamp ({@code primaryKeyLivenessInfo().timestamp()})
-     * is lower than the deletion timestamp. That is, if a row has a shadowable deletion with timestamp A and an update is made
-     * to that row with a timestamp B such that B > A, then the shadowable deletion is 'shadowed' by that update. A concrete
-     * consequence is that if said update has cells with timestamp lower than A, then those cells are preserved
-     * (since the deletion is removed), and this contrarily to a normal (regular) deletion where the deletion is preserved
-     * and such cells are removed.
+     * A shadowable row deletion only exists if the row has no timestamp. In other words, the deletion is only
+     * valid as long as no newer insert is done (thus setting a row timestap; note that if the row timestamp set
+     * is lower than the deletion, it is shadowed (and thus ignored) as usual).
+     * <p>
+     * That is, if a row has a shadowable deletion with timestamp A and an update is madeto that row with a
+     * timestamp B such that B > A (and that update sets the row timestamp), then the shadowable deletion is 'shadowed'
+     * by that update. A concrete consequence is that if said update has cells with timestamp lower than A, then those
+     * cells are preserved(since the deletion is removed), and this contrarily to a normal (regular) deletion where the
+     * deletion is preserved and such cells are removed.
      * <p>
      * Currently, the only use of shadowable row deletions is Materialized Views, see CASSANDRA-10261.
      */
@@ -312,6 +327,11 @@ public interface Row extends Unfiltered, Collection<ColumnData>
             return time.deletes(info);
         }
 
+        public boolean deletes(Cell cell)
+        {
+            return time.deletes(cell);
+        }
+
         public void digest(MessageDigest digest)
         {
             time.digest(digest);
@@ -361,6 +381,9 @@ public interface Row extends Unfiltered, Collection<ColumnData>
      *      any column before {@code c} and before any call for any column after {@code c}.
      *   5) Calls to {@link #addCell} are further done in strictly increasing cell order (the one defined by
      *      {@link Cell#comparator}. That is, for a give column, cells are passed in {@code CellPath} order.
+     *   6) No shadowed data should be added. Concretely, this means that if a a row deletion is added, it doesn't
+     *      deletes the row timestamp or any cell added later, and similarly no cell added is deleted by the complex
+     *      deletion of the column this is a cell of.
      *
      * An unsorted builder will not expect those last rules however: {@link #addCell} and {@link #addComplexDeletion}
      * can be done in any order. And in particular unsorted builder allows multiple calls for the same column/cell. In

http://git-wip-us.apache.org/repos/asf/cassandra/blob/86ba2274/src/java/org/apache/cassandra/db/rows/RowDiffListener.java
----------------------------------------------------------------------
diff --git a/src/java/org/apache/cassandra/db/rows/RowDiffListener.java b/src/java/org/apache/cassandra/db/rows/RowDiffListener.java
index f209bfc..ec848a0 100644
--- a/src/java/org/apache/cassandra/db/rows/RowDiffListener.java
+++ b/src/java/org/apache/cassandra/db/rows/RowDiffListener.java
@@ -71,7 +71,7 @@ public interface RowDiffListener
      * @param i the input row from which {@code original} is from.
      * @param clustering the clustering for the row that is merged.
      * @param merged the cell of the merged row. Will be {@code null} if input {@code i} had a cell but that cell is no present
-     * in the mergd result (it has been deleted/shadowed).
+     * in the merged result (it has been deleted/shadowed).
      * @param original the cell of input {@code i}. May be {@code null} if input {@code i} had cell corresponding to {@code merged}.
      */
     public void onCell(int i, Clustering clustering, Cell merged, Cell original);

http://git-wip-us.apache.org/repos/asf/cassandra/blob/86ba2274/src/java/org/apache/cassandra/db/rows/UnfilteredRowIterators.java
----------------------------------------------------------------------
diff --git a/src/java/org/apache/cassandra/db/rows/UnfilteredRowIterators.java b/src/java/org/apache/cassandra/db/rows/UnfilteredRowIterators.java
index 9416896..ce5fffe 100644
--- a/src/java/org/apache/cassandra/db/rows/UnfilteredRowIterators.java
+++ b/src/java/org/apache/cassandra/db/rows/UnfilteredRowIterators.java
@@ -92,7 +92,7 @@ public abstract class UnfilteredRowIterators
     }
 
     /**
-     * Returns an empty atom iterator for a given partition.
+     * Returns an empty unfiltered iterator for a given partition.
      */
     public static UnfilteredRowIterator noRowsIterator(final CFMetaData cfm, final DecoratedKey partitionKey, final Row staticRow, final DeletionTime partitionDeletion, final boolean isReverseOrder)
     {

http://git-wip-us.apache.org/repos/asf/cassandra/blob/86ba2274/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
new file mode 100644
index 0000000..893bdd5
--- /dev/null
+++ b/src/java/org/apache/cassandra/db/view/TableViews.java
@@ -0,0 +1,481 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.cassandra.db.view;
+
+import java.util.*;
+import java.util.concurrent.CopyOnWriteArrayList;
+import java.util.concurrent.TimeUnit;
+import java.util.concurrent.atomic.AtomicLong;
+
+import com.google.common.collect.Iterables;
+import com.google.common.collect.Iterators;
+import com.google.common.collect.PeekingIterator;
+
+import org.apache.cassandra.config.CFMetaData;
+import org.apache.cassandra.db.*;
+import org.apache.cassandra.db.commitlog.ReplayPosition;
+import org.apache.cassandra.db.filter.*;
+import org.apache.cassandra.db.rows.*;
+import org.apache.cassandra.db.partitions.*;
+import org.apache.cassandra.dht.Token;
+import org.apache.cassandra.service.StorageProxy;
+import org.apache.cassandra.utils.FBUtilities;
+import org.apache.cassandra.utils.btree.BTreeSet;
+
+
+/**
+ * Groups all the views for a given table.
+ */
+public class TableViews extends AbstractCollection<View>
+{
+    private final CFMetaData baseTableMetadata;
+
+    // We need this to be thread-safe, but the number of times this is changed (when a view is created in the keyspace)
+    // massively exceeds the number of time it's read (for every mutation on the keyspace), so a copy-on-write list is the best option.
+    private final List<View> views = new CopyOnWriteArrayList();
+
+    public TableViews(CFMetaData baseTableMetadata)
+    {
+        this.baseTableMetadata = baseTableMetadata;
+    }
+
+    public int size()
+    {
+        return views.size();
+    }
+
+    public Iterator<View> iterator()
+    {
+        return views.iterator();
+    }
+
+    public boolean contains(String viewName)
+    {
+        return Iterables.any(views, view -> view.name.equals(viewName));
+    }
+
+    public boolean add(View view)
+    {
+        // We should have validated that there is no existing view with this name at this point
+        assert !contains(view.name);
+        return views.add(view);
+    }
+
+    public Iterable<ColumnFamilyStore> allViewsCfs()
+    {
+        Keyspace keyspace = Keyspace.open(baseTableMetadata.ksName);
+        return Iterables.transform(views, view -> keyspace.getColumnFamilyStore(view.getDefinition().viewName));
+    }
+
+    public void forceBlockingFlush()
+    {
+        for (ColumnFamilyStore viewCfs : allViewsCfs())
+            viewCfs.forceBlockingFlush();
+    }
+
+    public void dumpMemtables()
+    {
+        for (ColumnFamilyStore viewCfs : allViewsCfs())
+            viewCfs.dumpMemtable();
+    }
+
+    public void truncateBlocking(long truncatedAt)
+    {
+        for (ColumnFamilyStore viewCfs : allViewsCfs())
+        {
+            ReplayPosition replayAfter = viewCfs.discardSSTables(truncatedAt);
+            SystemKeyspace.saveTruncationRecord(viewCfs, truncatedAt, replayAfter);
+        }
+    }
+
+    public void removeByName(String viewName)
+    {
+        views.removeIf(v -> v.name.equals(viewName));
+    }
+
+    /**
+     * Calculates and pushes updates to the views replicas. The replicas are determined by
+     * {@link ViewUtils#getViewNaturalEndpoint(String, Token, Token)}.
+     *
+     * @param update an update on the base table represented by this object.
+     * @param writeCommitLog whether we should write the commit log for the view updates.
+     * @param baseComplete time from epoch in ms that the local base mutation was (or will be) completed
+     */
+    public void pushViewReplicaUpdates(PartitionUpdate update, boolean writeCommitLog, AtomicLong baseComplete)
+    {
+        assert update.metadata().cfId.equals(baseTableMetadata.cfId);
+
+        Collection<View> views = updatedViews(update);
+        if (views.isEmpty())
+            return;
+
+        // Read modified rows
+        int nowInSec = FBUtilities.nowInSeconds();
+        SinglePartitionReadCommand command = readExistingRowsCommand(update, views, nowInSec);
+        if (command == null)
+            return;
+
+        ColumnFamilyStore cfs = Keyspace.openAndGetStore(update.metadata());
+        long start = System.nanoTime();
+        Collection<Mutation> mutations;
+        try (ReadOrderGroup orderGroup = command.startOrderGroup();
+             UnfilteredRowIterator existings = UnfilteredPartitionIterators.getOnlyElement(command.executeLocally(orderGroup), command);
+             UnfilteredRowIterator updates = update.unfilteredIterator())
+        {
+            mutations = generateViewUpdates(views, updates, existings, nowInSec);
+        }
+        Keyspace.openAndGetStore(update.metadata()).metric.viewReadTime.update(System.nanoTime() - start, TimeUnit.NANOSECONDS);
+
+        if (!mutations.isEmpty())
+            StorageProxy.mutateMV(update.partitionKey().getKey(), mutations, writeCommitLog, baseComplete);
+    }
+
+    /**
+     * Given some updates on the base table of this object and the existing values for the rows affected by that update, generates the
+     * mutation to be applied to the provided views.
+     *
+     * @param views the views potentially affected by {@code updates}.
+     * @param updates the base table updates being applied.
+     * @param existings the existing values for the rows affected by {@code updates}. This is used to decide if a view is
+     * obsoleted by the update and should be removed, gather the values for columns that may not be part of the update if
+     * a new view entry needs to be created, and compute the minimal updates to be applied if the view entry isn't changed
+     * 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.
+     * @return the mutations to apply to the {@code views}. This can be empty.
+     */
+    public Collection<Mutation> generateViewUpdates(Collection<View> views, UnfilteredRowIterator updates, UnfilteredRowIterator existings, int nowInSec)
+    {
+        assert updates.metadata().cfId.equals(baseTableMetadata.cfId);
+
+        List<ViewUpdateGenerator> generators = new ArrayList<>(views.size());
+        for (View view : views)
+            generators.add(new ViewUpdateGenerator(view, updates.partitionKey(), nowInSec));
+
+        DeletionTracker existingsDeletion = new DeletionTracker(existings.partitionLevelDeletion());
+        DeletionTracker updatesDeletion = new DeletionTracker(updates.partitionLevelDeletion());
+
+        /*
+         * We iterate through the updates and the existing rows in parallel. This allows us to know the consequence
+         * on the view of each update.
+         */
+        PeekingIterator<Unfiltered> existingsIter = Iterators.peekingIterator(existings);
+        PeekingIterator<Unfiltered> updatesIter = Iterators.peekingIterator(updates);
+
+        while (existingsIter.hasNext() && updatesIter.hasNext())
+        {
+            Unfiltered existing = existingsIter.peek();
+            Unfiltered update = updatesIter.peek();
+
+            Row existingRow;
+            Row updateRow;
+            int cmp = baseTableMetadata.comparator.compare(update, existing);
+            if (cmp < 0)
+            {
+                // We have an update where there was nothing before
+                if (update.isRangeTombstoneMarker())
+                {
+                    updatesDeletion.update(updatesIter.next());
+                    continue;
+                }
+
+                updateRow = ((Row)updatesIter.next()).withRowDeletion(updatesDeletion.currentDeletion());
+                existingRow = emptyRow(updateRow.clustering(), existingsDeletion.currentDeletion());
+            }
+            else if (cmp > 0)
+            {
+                // We have something existing but no update (which will happen either because it's a range tombstone marker in
+                // existing, or because we've fetched the existing row due to some partition/range deletion in the updates)
+                if (existing.isRangeTombstoneMarker())
+                {
+                    existingsDeletion.update(existingsIter.next());
+                    continue;
+                }
+
+                existingRow = ((Row)existingsIter.next()).withRowDeletion(existingsDeletion.currentDeletion());
+                updateRow = emptyRow(existingRow.clustering(), updatesDeletion.currentDeletion());
+
+                // The way we build the read command used for existing rows, we should always have updatesDeletion.currentDeletion()
+                // that is not live, since we wouldn't have read the existing row otherwise. And we could assert that, but if we ever
+                // change the read method so that it can slightly over-read in some case, that would be an easily avoiding bug lurking,
+                // so we just handle the case.
+                if (updateRow == null)
+                    continue;
+            }
+            else
+            {
+                // We're updating a row that had pre-existing data
+                if (update.isRangeTombstoneMarker())
+                {
+                    assert existing.isRangeTombstoneMarker();
+                    updatesDeletion.update(updatesIter.next());
+                    existingsDeletion.update(existingsIter.next());
+                    continue;
+                }
+
+                assert !existing.isRangeTombstoneMarker();
+                existingRow = ((Row)existingsIter.next()).withRowDeletion(existingsDeletion.currentDeletion());
+                updateRow = ((Row)updatesIter.next()).withRowDeletion(updatesDeletion.currentDeletion());
+            }
+
+            addToViewUpdateGenerators(existingRow, updateRow, generators, nowInSec);
+        }
+
+        // We only care about more existing rows if the update deletion isn't live, i.e. if we had a partition deletion
+        if (!updatesDeletion.currentDeletion().isLive())
+        {
+            while (existingsIter.hasNext())
+            {
+                Unfiltered existing = existingsIter.next();
+                // If it's a range tombstone, we don't care, we're only looking for existing entry that gets deleted by
+                // the new partition deletion
+                if (existing.isRangeTombstoneMarker())
+                    continue;
+
+                Row existingRow = (Row)existing;
+                addToViewUpdateGenerators(existingRow, emptyRow(existingRow.clustering(), updatesDeletion.currentDeletion()), generators, nowInSec);
+            }
+        }
+        while (updatesIter.hasNext())
+        {
+            Unfiltered update = updatesIter.next();
+            // If it's a range tombstone, it removes nothing pre-exisiting, so we can ignore it for view updates
+            if (update.isRangeTombstoneMarker())
+                continue;
+
+            Row updateRow = (Row)update;
+            addToViewUpdateGenerators(emptyRow(updateRow.clustering(), DeletionTime.LIVE), updateRow, generators, nowInSec);
+        }
+
+        return buildMutations(baseTableMetadata, generators);
+    }
+
+    /**
+     * Return the views that are potentially updated by the provided updates.
+     *
+     * @param updates the updates applied to the base table.
+     * @return the views affected by {@code updates}.
+     */
+    public Collection<View> updatedViews(PartitionUpdate updates)
+    {
+        List<View> matchingViews = new ArrayList<>(views.size());
+
+        for (View view : views)
+        {
+            ReadQuery selectQuery = view.getReadQuery();
+            if (!selectQuery.selectsKey(updates.partitionKey()))
+                continue;
+
+            matchingViews.add(view);
+        }
+        return matchingViews;
+    }
+
+    /**
+     * Returns the command to use to read the existing rows required to generate view updates for the provided base
+     * base updates.
+     *
+     * @param updates the base table updates being applied.
+     * @param views the views potentially affected by {@code updates}.
+     * @param nowInSec the current time in seconds.
+     * @return the command to use to read the base table rows required to generate view updates for {@code updates}.
+     */
+    private SinglePartitionReadCommand readExistingRowsCommand(PartitionUpdate updates, Collection<View> views, int nowInSec)
+    {
+        Slices.Builder sliceBuilder = null;
+        DeletionInfo deletionInfo = updates.deletionInfo();
+        CFMetaData metadata = updates.metadata();
+        DecoratedKey key = updates.partitionKey();
+        // TODO: This is subtle: we need to gather all the slices that we have to fetch between partition del, range tombstones and rows.
+        if (!deletionInfo.isLive())
+        {
+            sliceBuilder = new Slices.Builder(metadata.comparator);
+            // Everything covered by a deletion might invalidate an existing view entry, which means we must read it to know. In practice
+            // though, the views involved might filter some base table clustering columns, in which case we can restrict what we read
+            // using those restrictions.
+            // If there is a partition deletion, then we can simply take each slices from each view select filter. They may overlap but
+            // the Slices.Builder handles that for us. Note that in many case this will just involve reading everything (as soon as any
+            // view involved has no clustering restrictions for instance).
+            // For range tombstone, we should theoretically take the difference between the range tombstoned and the slices selected
+            // by every views, but as we don't an easy way to compute that right now, we keep it simple and just use the tombstoned
+            // range.
+            // TODO: we should improve that latter part.
+            if (!deletionInfo.getPartitionDeletion().isLive())
+            {
+                for (View view : views)
+                    sliceBuilder.addAll(view.getSelectStatement().clusteringIndexFilterAsSlices());
+            }
+            else
+            {
+                assert deletionInfo.hasRanges();
+                Iterator<RangeTombstone> iter = deletionInfo.rangeIterator(false);
+                while (iter.hasNext())
+                    sliceBuilder.add(iter.next().deletedSlice());
+            }
+        }
+
+        // We need to read every row that is updated, unless we can prove that it has no impact on any view entries.
+
+        // If we had some slices from the deletions above, we'll continue using that. Otherwise, it's more efficient to build
+        // a names query.
+        BTreeSet.Builder<Clustering> namesBuilder = sliceBuilder == null ? BTreeSet.builder(metadata.comparator) : null;
+        for (Row row : updates)
+        {
+            // Don't read the existing state if we can prove the update won't affect any views
+            if (!affectsAnyViews(key, row, views))
+                continue;
+
+            if (namesBuilder == null)
+                sliceBuilder.add(Slice.make(row.clustering()));
+            else
+                namesBuilder.add(row.clustering());
+        }
+
+        NavigableSet<Clustering> names = namesBuilder == null ? null : namesBuilder.build();
+        // If we have a slice builder, it means we had some deletions and we have to read. But if we had
+        // only row updates, it's possible none of them affected the views, in which case we have nothing
+        // to do.
+        if (names != null && names.isEmpty())
+            return null;
+
+        ClusteringIndexFilter clusteringFilter = names == null
+                                               ? new ClusteringIndexSliceFilter(sliceBuilder.build(), false)
+                                               : new ClusteringIndexNamesFilter(names, false);
+        // 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);
+        // 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.
+        // TODO: we could still make sense to special case for when there is a single view and a small number of updates (and
+        // no deletions). Indeed, in that case we could check whether any of the update modify any of the restricted regular
+        // column, and if that's not the case we could use view filter. We keep it simple for now though.
+        RowFilter rowFilter = RowFilter.NONE;
+        return SinglePartitionReadCommand.create(metadata, nowInSec, queriedColumns, rowFilter, DataLimits.NONE, key, clusteringFilter);
+    }
+
+    private boolean affectsAnyViews(DecoratedKey partitionKey, Row update, Collection<View> views)
+    {
+        for (View view : views)
+        {
+            if (view.mayBeAffectedBy(partitionKey, update))
+                return true;
+        }
+        return false;
+    }
+
+    /**
+     * Given an existing base row and the update that we're going to apply to this row, generate the modifications
+     * to apply to MVs using the provided {@code ViewUpdateGenerator}s.
+     *
+     * @param existingBaseRow the base table row as it is before an update.
+     * @param updateBaseRow the newly updates made to {@code existingBaseRow}.
+     * @param generators the view update generators to add the new changes to.
+     * @param nowInSec the current time in seconds. Used to decide if data is live or not.
+     */
+    private static void addToViewUpdateGenerators(Row existingBaseRow, Row updateBaseRow, Collection<ViewUpdateGenerator> generators, int nowInSec)
+    {
+        // Having existing empty is useful, it just means we'll insert a brand new entry for updateBaseRow,
+        // but if we have no update at all, we shouldn't get there.
+        assert !updateBaseRow.isEmpty();
+
+        // We allow existingBaseRow to be null, which we treat the same as being empty as an small optimization
+        // to avoid allocating empty row objects when we know there was nothing existing.
+        Row mergedBaseRow = existingBaseRow == null ? updateBaseRow : Rows.merge(existingBaseRow, updateBaseRow, nowInSec);
+        for (ViewUpdateGenerator generator : generators)
+            generator.addBaseTableUpdate(existingBaseRow, mergedBaseRow);
+    }
+
+    private static Row emptyRow(Clustering clustering, DeletionTime deletion)
+    {
+        // Returning null for an empty row is slightly ugly, but the case where there is no pre-existing row is fairly common
+        // (especially when building the view), so we want to avoid a dummy allocation of an empty row every time.
+        // And MultiViewUpdateBuilder knows how to deal with that.
+        return deletion.isLive() ? null : BTreeRow.emptyDeletedRow(clustering, Row.Deletion.regular(deletion));
+    }
+
+    /**
+     * Extracts (and potentially groups) the mutations generated by the provided view update generator.
+     * Returns the mutation that needs to be done to the views given the base table updates
+     * passed to {@link #addBaseTableUpdate}.
+     *
+     * @param baseTableMetadata the metadata for the base table being updated.
+     * @param generators the generators from which to extract the view mutations from.
+     * @return the mutations created by all the generators in {@code generators}.
+     */
+    private Collection<Mutation> buildMutations(CFMetaData baseTableMetadata, List<ViewUpdateGenerator> generators)
+    {
+        // One view is probably common enough and we can optimize a bit easily
+        if (generators.size() == 1)
+        {
+            Collection<PartitionUpdate> updates = generators.get(0).generateViewUpdates();
+            List<Mutation> mutations = new ArrayList<>(updates.size());
+            for (PartitionUpdate update : updates)
+                mutations.add(new Mutation(update));
+            return mutations;
+        }
+
+        Map<DecoratedKey, Mutation> mutations = new HashMap<>();
+        for (ViewUpdateGenerator generator : generators)
+        {
+            for (PartitionUpdate update : generator.generateViewUpdates())
+            {
+                DecoratedKey key = update.partitionKey();
+                Mutation mutation = mutations.get(key);
+                if (mutation == null)
+                {
+                    mutation = new Mutation(baseTableMetadata.ksName, key);
+                    mutations.put(key, mutation);
+                }
+                mutation.add(update);
+            }
+        }
+        return mutations.values();
+    }
+
+    /**
+     * A simple helper that tracks for a given {@code UnfilteredRowIterator} what is the current deletion at any time of the
+     * iteration. It will be the currently open range tombstone deletion if there is one and the partition deletion otherwise.
+     */
+    private static class DeletionTracker
+    {
+        private final DeletionTime partitionDeletion;
+        private DeletionTime deletion;
+
+        public DeletionTracker(DeletionTime partitionDeletion)
+        {
+            this.partitionDeletion = partitionDeletion;
+        }
+
+        public void update(Unfiltered marker)
+        {
+            assert marker instanceof RangeTombstoneMarker;
+            RangeTombstoneMarker rtm = (RangeTombstoneMarker)marker;
+            this.deletion = rtm.isOpen(false)
+                          ? rtm.openDeletionTime(false)
+                          : null;
+        }
+
+        public DeletionTime currentDeletion()
+        {
+            return deletion == null ? partitionDeletion : deletion;
+        }
+    }
+}

http://git-wip-us.apache.org/repos/asf/cassandra/blob/86ba2274/src/java/org/apache/cassandra/db/view/TemporalRow.java
----------------------------------------------------------------------
diff --git a/src/java/org/apache/cassandra/db/view/TemporalRow.java b/src/java/org/apache/cassandra/db/view/TemporalRow.java
deleted file mode 100644
index 23705b9..0000000
--- a/src/java/org/apache/cassandra/db/view/TemporalRow.java
+++ /dev/null
@@ -1,610 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements.  See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership.  The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License.  You may obtain a copy of the License at
- *
- *     http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-
-package org.apache.cassandra.db.view;
-
-import static java.util.Comparator.naturalOrder;
-import static java.util.Comparator.reverseOrder;
-
-import java.nio.ByteBuffer;
-import java.util.ArrayList;
-import java.util.Collection;
-import java.util.Collections;
-import java.util.Comparator;
-import java.util.HashMap;
-import java.util.Iterator;
-import java.util.List;
-import java.util.Map;
-
-import com.google.common.base.MoreObjects;
-import com.google.common.collect.Iterables;
-
-import org.apache.cassandra.config.CFMetaData;
-import org.apache.cassandra.config.ColumnDefinition;
-import org.apache.cassandra.cql3.ColumnIdentifier;
-import org.apache.cassandra.db.CBuilder;
-import org.apache.cassandra.db.Clustering;
-import org.apache.cassandra.db.ColumnFamilyStore;
-import org.apache.cassandra.db.Conflicts;
-import org.apache.cassandra.db.DecoratedKey;
-import org.apache.cassandra.db.DeletionInfo;
-import org.apache.cassandra.db.DeletionTime;
-import org.apache.cassandra.db.LivenessInfo;
-import org.apache.cassandra.db.RangeTombstone;
-import org.apache.cassandra.db.Slice;
-import org.apache.cassandra.db.marshal.CompositeType;
-import org.apache.cassandra.db.partitions.AbstractBTreePartition;
-import org.apache.cassandra.db.rows.BufferCell;
-import org.apache.cassandra.db.rows.Cell;
-import org.apache.cassandra.db.rows.CellPath;
-import org.apache.cassandra.db.rows.Row;
-import org.apache.cassandra.utils.ByteBufferUtil;
-import org.apache.cassandra.utils.FBUtilities;
-
-/**
- * Represents a single CQL Row in a base table, with both the currently persisted value and the update's value. The
- * values are stored in timestamp order, but also indicate whether they are from the currently persisted, allowing a
- * {@link TemporalRow.Resolver} to resolve if the value is an old value that has been updated; if it sorts after the
- * update's value, then it does not qualify.
- */
-public class TemporalRow
-{
-    private static final int NO_TTL = LivenessInfo.NO_TTL;
-    private static final long NO_TIMESTAMP = LivenessInfo.NO_TIMESTAMP;
-    private static final int NO_DELETION_TIME = DeletionTime.LIVE.localDeletionTime();
-
-    public interface Resolver
-    {
-        /**
-         * @param cellVersions  all cells for a certain TemporalRow's Cell
-         * @return      A single TemporalCell from the iterable which satisfies the resolution criteria, or null if
-         *              there is no cell which qualifies
-         */
-        TemporalCell resolve(TemporalCell.Versions cellVersions);
-    }
-
-    public static final Resolver oldValueIfUpdated = TemporalCell.Versions::getOldCellIfUpdated;
-    public static final Resolver earliest = TemporalCell.Versions::getEarliestCell;
-    public static final Resolver latest = TemporalCell.Versions::getLatestCell;
-
-    private static class TemporalCell
-    {
-        public final ByteBuffer value;
-        public final long timestamp;
-        public final int ttl;
-        public final int localDeletionTime;
-        public final boolean isNew;
-
-        private TemporalCell(ByteBuffer value, long timestamp, int ttl, int localDeletionTime, boolean isNew)
-        {
-            this.value = value;
-            this.timestamp = timestamp;
-            this.ttl = ttl;
-            this.localDeletionTime = localDeletionTime;
-            this.isNew = isNew;
-        }
-
-        @Override
-        public String toString()
-        {
-            return MoreObjects.toStringHelper(this)
-                    .add("value", value == null ? "null" : ByteBufferUtil.bytesToHex(value))
-                    .add("timestamp", timestamp)
-                    .add("ttl", ttl)
-                    .add("localDeletionTime", localDeletionTime)
-                    .add("isNew", isNew)
-                    .toString();
-        }
-
-        public TemporalCell reconcile(TemporalCell that)
-        {
-            int now = FBUtilities.nowInSeconds();
-            Conflicts.Resolution resolution = Conflicts.resolveRegular(that.timestamp,
-                                                                       that.isLive(now),
-                                                                       that.localDeletionTime,
-                                                                       that.value,
-                                                                       this.timestamp,
-                                                                       this.isLive(now),
-                                                                       this.localDeletionTime,
-                                                                       this.value);
-            assert resolution != Conflicts.Resolution.MERGE;
-            if (resolution == Conflicts.Resolution.LEFT_WINS)
-                return that;
-            return this;
-        }
-
-        private boolean isLive(int now)
-        {
-            return localDeletionTime == NO_DELETION_TIME || (ttl != NO_TTL && now < localDeletionTime);
-        }
-
-        public Cell cell(ColumnDefinition definition, CellPath cellPath)
-        {
-            return new BufferCell(definition, timestamp, ttl, localDeletionTime, value, cellPath);
-        }
-
-        public boolean equals(Object o)
-        {
-            if (this == o) return true;
-            if (o == null || getClass() != o.getClass()) return false;
-
-            TemporalCell that = (TemporalCell) o;
-
-            if (timestamp != that.timestamp) return false;
-            if (ttl != that.ttl) return false;
-            if (localDeletionTime != that.localDeletionTime) return false;
-            if (isNew != that.isNew) return false;
-            return !(value != null ? !value.equals(that.value) : that.value != null);
-        }
-
-        public int hashCode()
-        {
-            int result = value != null ? value.hashCode() : 0;
-            result = 31 * result + (int) (timestamp ^ (timestamp >>> 32));
-            result = 31 * result + ttl;
-            result = 31 * result + localDeletionTime;
-            result = 31 * result + (isNew ? 1 : 0);
-            return result;
-        }
-
-        /**
-         * Tracks the versions of a cell for a given TemporalRow.
-         * There are only two possible versions, existing and new.
-         *
-         */
-        static class Versions
-        {
-            private TemporalCell existingCell = null;
-            private TemporalCell newCell = null;
-            private int numSet = 0;
-
-
-            /**
-             * @return the cell that is earliest
-             * Or would be overwritten in the case of a timestamp conflict
-             */
-            public TemporalCell getEarliestCell()
-            {
-                assert numSet > 0;
-
-                if (numSet == 1)
-                    return existingCell == null ? newCell : existingCell;
-
-                TemporalCell latest = existingCell.reconcile(newCell);
-
-                return latest == newCell ? existingCell : newCell;
-            }
-
-            /**
-             * @return the cell that is latest
-             * Or would be the winner in the case of a timestamp conflict
-             */
-            public TemporalCell getLatestCell()
-            {
-                assert numSet > 0;
-
-                if (numSet == 1)
-                    return existingCell == null ? newCell : existingCell;
-
-                return existingCell.reconcile(newCell);
-            }
-
-            /**
-             * @return the new cell if it updates the existing cell
-             */
-            public TemporalCell getOldCellIfUpdated()
-            {
-                assert numSet > 0;
-
-                if (numSet == 1)
-                   return null;
-
-                TemporalCell value = existingCell.reconcile(newCell);
-
-                return ByteBufferUtil.compareUnsigned(existingCell.value, value.value) != 0 ? existingCell : null;
-            }
-
-            void setVersion(TemporalCell cell)
-            {
-                assert cell != null;
-
-                if (cell.isNew)
-                {
-                    assert newCell == null || newCell.equals(cell) : "Only one cell version can be marked New; newCell: " + newCell + ", cell: " + cell;
-                    newCell = cell;
-                    numSet = existingCell == null ? 1 : 2;
-                }
-                else
-                {
-                    assert existingCell == null || existingCell.equals(cell) : "Only one cell version can be marked Existing; existingCell: " + existingCell + ", cell: " + cell;
-                    existingCell = cell;
-                    numSet = newCell == null ? 1 : 2;
-                }
-            }
-
-            public void addToRow(TemporalRow row, ColumnIdentifier column, CellPath path)
-            {
-                if (existingCell != null)
-                    row.addColumnValue(column, path, existingCell.timestamp, existingCell.ttl,
-                                       existingCell.localDeletionTime, existingCell.value, existingCell.isNew);
-
-                if (newCell != null)
-                    row.addColumnValue(column, path, newCell.timestamp, newCell.ttl,
-                                       newCell.localDeletionTime, newCell.value, newCell.isNew);
-            }
-
-            @Override
-            public String toString()
-            {
-                return MoreObjects.toStringHelper(this)
-                        .add("numSet", numSet)
-                        .add("existingCell", existingCell)
-                        .add("newCell", newCell)
-                        .toString();
-            }
-        }
-    }
-
-    private final ColumnFamilyStore baseCfs;
-    private final java.util.Set<ColumnIdentifier> viewPrimaryKey;
-    private final ByteBuffer basePartitionKey;
-    private final Map<ColumnIdentifier, ByteBuffer> clusteringColumns;
-    private final Row startRow;
-    private final boolean startIsNew;
-
-    public final int nowInSec;
-    private final Map<ColumnIdentifier, Map<CellPath, TemporalCell.Versions>> columnValues = new HashMap<>();
-    private int viewClusteringTtl = NO_TTL;
-    private long viewClusteringTimestamp = NO_TIMESTAMP;
-    private int viewClusteringLocalDeletionTime = NO_DELETION_TIME;
-
-    TemporalRow(ColumnFamilyStore baseCfs, java.util.Set<ColumnIdentifier> viewPrimaryKey, ByteBuffer key, Row row, int nowInSec, boolean isNew)
-    {
-        this.baseCfs = baseCfs;
-        this.viewPrimaryKey = viewPrimaryKey;
-        this.basePartitionKey = key;
-        this.startRow = row;
-        this.startIsNew = isNew;
-        this.nowInSec = nowInSec;
-
-        LivenessInfo liveness = row.primaryKeyLivenessInfo();
-        updateLiveness(liveness.ttl(), liveness.timestamp(), row.deletion().time().localDeletionTime());
-
-        List<ColumnDefinition> clusteringDefs = baseCfs.metadata.clusteringColumns();
-        clusteringColumns = new HashMap<>();
-
-        for (int i = 0; i < clusteringDefs.size(); i++)
-        {
-            ColumnDefinition cdef = clusteringDefs.get(i);
-            clusteringColumns.put(cdef.name, row.clustering().get(i));
-
-            addColumnValue(cdef.name, null, NO_TIMESTAMP, NO_TTL, NO_DELETION_TIME, row.clustering().get(i), isNew);
-        }
-    }
-
-    /*
-     * PK ts:5, ttl:1, deletion: 2
-     * Col ts:4, ttl:2, deletion: 3
-     *
-     * TTL use min, since it expires at the lowest time which we are expiring. If we have the above values, we
-     * would want to return 1, since the base row expires in 1 second.
-     *
-     * Timestamp uses max, as this is the time that the row has been written to the view. See CASSANDRA-10910.
-     *
-     * Local Deletion Time should use max, as this deletion will cover all previous values written.
-     */
-    private void updateLiveness(int ttl, long timestamp, int localDeletionTime)
-    {
-        // We are returning whichever is higher from valueIfSet
-        // Natural order will return the max: 1.compareTo(2) < 0, so 2 is returned
-        // Reverse order will return the min: 1.compareTo(2) > 0, so 1 is returned
-        this.viewClusteringTtl = valueIfSet(viewClusteringTtl, ttl, NO_TTL, reverseOrder());
-        this.viewClusteringTimestamp = valueIfSet(viewClusteringTimestamp, timestamp, NO_TIMESTAMP, naturalOrder());
-        this.viewClusteringLocalDeletionTime = valueIfSet(viewClusteringLocalDeletionTime, localDeletionTime, NO_DELETION_TIME, naturalOrder());
-    }
-
-    @Override
-    public String toString()
-    {
-        return MoreObjects.toStringHelper(this)
-                .add("table", baseCfs.keyspace.getName() + "." + baseCfs.getTableName())
-                .add("basePartitionKey", ByteBufferUtil.bytesToHex(basePartitionKey))
-                .add("startRow", startRow.toString(baseCfs.metadata))
-                .add("startIsNew", startIsNew)
-                .add("nowInSec", nowInSec)
-                .add("viewClusteringTtl", viewClusteringTtl)
-                .add("viewClusteringTimestamp", viewClusteringTimestamp)
-                .add("viewClusteringLocalDeletionTime", viewClusteringLocalDeletionTime)
-                .add("columnValues", columnValues)
-                .toString();
-    }
-
-    @Override
-    public boolean equals(Object o)
-    {
-        if (this == o) return true;
-        if (o == null || getClass() != o.getClass()) return false;
-
-        TemporalRow that = (TemporalRow) o;
-
-        if (!clusteringColumns.equals(that.clusteringColumns)) return false;
-        if (!basePartitionKey.equals(that.basePartitionKey)) return false;
-
-        return true;
-    }
-
-    @Override
-    public int hashCode()
-    {
-        int result = basePartitionKey.hashCode();
-        result = 31 * result + clusteringColumns.hashCode();
-        return result;
-    }
-
-    public void addColumnValue(ColumnIdentifier identifier,
-                               CellPath cellPath,
-                               long timestamp,
-                               int ttl,
-                               int localDeletionTime,
-                               ByteBuffer value,  boolean isNew)
-    {
-        if (!columnValues.containsKey(identifier))
-            columnValues.put(identifier, new HashMap<>());
-
-        Map<CellPath, TemporalCell.Versions> innerMap = columnValues.get(identifier);
-
-        if (!innerMap.containsKey(cellPath))
-            innerMap.put(cellPath, new TemporalCell.Versions());
-
-        // If this column is part of the view's primary keys
-        if (viewPrimaryKey.contains(identifier))
-        {
-            updateLiveness(ttl, timestamp, localDeletionTime);
-        }
-
-        innerMap.get(cellPath).setVersion(new TemporalCell(value, timestamp, ttl, localDeletionTime, isNew));
-    }
-
-    /**
-     * @return
-     * <ul>
-     *     <li>
-     *         If both existing and update are defaultValue, return defaultValue
-     *     </li>
-     *     <li>
-     *         If only one of existing or existing are defaultValue, return the one which is not
-     *     </li>
-     *     <li>
-     *         If both existing and update are not defaultValue, compare using comparator and return the higher one.
-     *     </li>
-     * </ul>
-     */
-    private static <T> T valueIfSet(T existing, T update, T defaultValue, Comparator<T> comparator)
-    {
-        if (existing.equals(defaultValue))
-            return update;
-        if (update.equals(defaultValue))
-            return existing;
-        return comparator.compare(existing, update) > 0 ? existing : update;
-    }
-
-    public int viewClusteringTtl()
-    {
-        return viewClusteringTtl;
-    }
-
-    public long viewClusteringTimestamp()
-    {
-        return viewClusteringTimestamp;
-    }
-
-    public int viewClusteringLocalDeletionTime()
-    {
-        return viewClusteringLocalDeletionTime;
-    }
-
-    public void addCell(Cell cell, boolean isNew)
-    {
-        addColumnValue(cell.column().name, cell.path(), cell.timestamp(), cell.ttl(), cell.localDeletionTime(), cell.value(), isNew);
-    }
-
-    // The Definition here is actually the *base table* definition
-    public ByteBuffer clusteringValue(ColumnDefinition definition, Resolver resolver)
-    {
-        ColumnDefinition baseDefinition = definition.cfName.equals(baseCfs.name)
-                                          ? definition
-                                          : baseCfs.metadata.getColumnDefinition(definition.name);
-
-        if (baseDefinition.isPartitionKey())
-        {
-            if (baseCfs.metadata.getKeyValidator() instanceof CompositeType)
-            {
-                CompositeType keyComparator = (CompositeType) baseCfs.metadata.getKeyValidator();
-                ByteBuffer[] components = keyComparator.split(basePartitionKey);
-                return components[baseDefinition.position()];
-            }
-            else
-            {
-                return basePartitionKey;
-            }
-        }
-        else
-        {
-            ColumnIdentifier columnIdentifier = baseDefinition.name;
-
-            if (clusteringColumns.containsKey(columnIdentifier))
-                return clusteringColumns.get(columnIdentifier);
-
-            Collection<org.apache.cassandra.db.rows.Cell> val = values(definition, resolver);
-            if (val != null && val.size() == 1)
-            {
-                org.apache.cassandra.db.rows.Cell cell = Iterables.getOnlyElement(val);
-                // handle single-column deletions correctly
-                return cell.isTombstone() ? null : cell.value();
-            }
-        }
-        return null;
-    }
-
-    public DeletionTime deletionTime(AbstractBTreePartition partition)
-    {
-        DeletionInfo deletionInfo = partition.deletionInfo();
-        if (!deletionInfo.getPartitionDeletion().isLive())
-            return deletionInfo.getPartitionDeletion();
-
-        Clustering baseClustering = baseClusteringBuilder().build();
-        RangeTombstone clusterTombstone = deletionInfo.rangeCovering(baseClustering);
-        if (clusterTombstone != null)
-            return clusterTombstone.deletionTime();
-
-        Row row = partition.getRow(baseClustering);
-        return row == null || row.deletion().isLive() ? DeletionTime.LIVE : row.deletion().time();
-    }
-
-    public Collection<org.apache.cassandra.db.rows.Cell> values(ColumnDefinition definition, Resolver resolver)
-    {
-        Map<CellPath, TemporalCell.Versions> innerMap = columnValues.get(definition.name);
-        if (innerMap == null)
-        {
-            return Collections.emptyList();
-        }
-
-        Collection<org.apache.cassandra.db.rows.Cell> value = new ArrayList<>();
-        for (Map.Entry<CellPath, TemporalCell.Versions> pathAndCells : innerMap.entrySet())
-        {
-            TemporalCell cell = resolver.resolve(pathAndCells.getValue());
-
-            if (cell != null)
-                value.add(cell.cell(definition, pathAndCells.getKey()));
-        }
-        return value;
-    }
-
-    public Slice baseSlice()
-    {
-        return baseClusteringBuilder().buildSlice();
-    }
-
-    private CBuilder baseClusteringBuilder()
-    {
-        CFMetaData metadata = baseCfs.metadata;
-        CBuilder builder = CBuilder.create(metadata.comparator);
-
-        ByteBuffer[] buffers = new ByteBuffer[clusteringColumns.size()];
-        for (Map.Entry<ColumnIdentifier, ByteBuffer> buffer : clusteringColumns.entrySet())
-            buffers[metadata.getColumnDefinition(buffer.getKey()).position()] = buffer.getValue();
-
-        for (ByteBuffer byteBuffer : buffers)
-            builder = builder.add(byteBuffer);
-
-        return builder;
-    }
-
-    public Clustering baseClustering()
-    {
-        return startRow.clustering();
-    }
-
-    static class Set implements Iterable<TemporalRow>
-    {
-        private final ColumnFamilyStore baseCfs;
-        private final java.util.Set<ColumnIdentifier> viewPrimaryKey;
-        private final ByteBuffer key;
-        public final DecoratedKey dk;
-        private final Map<Clustering, TemporalRow> clusteringToRow;
-        final int nowInSec = FBUtilities.nowInSeconds();
-        private boolean hasTombstonedExisting = false;
-
-        Set(ColumnFamilyStore baseCfs, java.util.Set<ColumnIdentifier> viewPrimaryKey, ByteBuffer key)
-        {
-            this.baseCfs = baseCfs;
-            this.viewPrimaryKey = viewPrimaryKey;
-            this.key = key;
-            this.dk = baseCfs.decorateKey(key);
-            this.clusteringToRow = new HashMap<>();
-        }
-
-        public Iterator<TemporalRow> iterator()
-        {
-            return clusteringToRow.values().iterator();
-        }
-
-        public TemporalRow getClustering(Clustering clustering)
-        {
-            return clusteringToRow.get(clustering);
-        }
-
-        public void addRow(Row row, boolean isNew)
-        {
-            TemporalRow temporalRow = clusteringToRow.get(row.clustering());
-            if (temporalRow == null)
-            {
-                temporalRow = new TemporalRow(baseCfs, viewPrimaryKey, key, row, nowInSec, isNew);
-                clusteringToRow.put(row.clustering(), temporalRow);
-            }
-
-            for (Cell cell : row.cells())
-            {
-                temporalRow.addCell(cell, isNew);
-            }
-        }
-
-        private void addRow(TemporalRow row)
-        {
-            TemporalRow newRow = new TemporalRow(baseCfs, viewPrimaryKey, key, row.startRow, nowInSec, row.startIsNew);
-
-            TemporalRow existing = clusteringToRow.put(row.startRow.clustering(), newRow);
-            assert existing == null;
-
-            for (Map.Entry<ColumnIdentifier, Map<CellPath, TemporalCell.Versions>> entry : row.columnValues.entrySet())
-            {
-                for (Map.Entry<CellPath, TemporalCell.Versions> cellPathEntry : entry.getValue().entrySet())
-                {
-                    TemporalCell.Versions cellVersions = cellPathEntry.getValue();
-
-                    cellVersions.addToRow(newRow, entry.getKey(), cellPathEntry.getKey());
-                }
-            }
-        }
-
-        public TemporalRow.Set withNewViewPrimaryKey(java.util.Set<ColumnIdentifier> viewPrimaryKey)
-        {
-            TemporalRow.Set newSet = new Set(baseCfs, viewPrimaryKey, key);
-
-            for (TemporalRow row : this)
-                newSet.addRow(row);
-
-            return newSet;
-        }
-
-        public boolean hasTombstonedExisting()
-        {
-            return hasTombstonedExisting;
-        }
-
-        public void setTombstonedExisting()
-        {
-            hasTombstonedExisting = true;
-        }
-
-        public int size()
-        {
-            return clusteringToRow.size();
-        }
-    }
-}


[7/9] cassandra git commit: Merge commit '86ba227477b9f8595eb610ecaf950cfbc29dd36b' into cassandra-3.7

Posted by sl...@apache.org.
Merge commit '86ba227477b9f8595eb610ecaf950cfbc29dd36b' into cassandra-3.7


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

Branch: refs/heads/trunk
Commit: a87fd715d6b26128603a404074ec3df42a595b2e
Parents: 4e364d7 86ba227
Author: Sylvain Lebresne <sy...@datastax.com>
Authored: Fri May 6 13:43:44 2016 +0200
Committer: Sylvain Lebresne <sy...@datastax.com>
Committed: Fri May 6 13:44:12 2016 +0200

----------------------------------------------------------------------
 .../org/apache/cassandra/config/CFMetaData.java |   6 +
 .../apache/cassandra/config/ViewDefinition.java |   1 -
 .../cql3/statements/CreateViewStatement.java    |   4 +-
 .../cql3/statements/SelectStatement.java        |  41 +-
 .../apache/cassandra/db/ColumnFamilyStore.java  |   6 +-
 src/java/org/apache/cassandra/db/Keyspace.java  |   2 +-
 .../db/SinglePartitionReadCommand.java          |  33 +
 src/java/org/apache/cassandra/db/Slices.java    |   7 +
 .../apache/cassandra/db/filter/RowFilter.java   |  24 +
 .../SingletonUnfilteredPartitionIterator.java   |   3 +-
 .../apache/cassandra/db/rows/AbstractCell.java  |   5 +
 .../org/apache/cassandra/db/rows/BTreeRow.java  |  34 +-
 .../apache/cassandra/db/rows/BufferCell.java    |   5 +
 src/java/org/apache/cassandra/db/rows/Cell.java |   2 +
 .../apache/cassandra/db/rows/ColumnData.java    |   2 +
 .../cassandra/db/rows/ComplexColumnData.java    |   8 +
 .../apache/cassandra/db/rows/NativeCell.java    |   5 +
 src/java/org/apache/cassandra/db/rows/Row.java  |  35 +-
 .../cassandra/db/rows/RowDiffListener.java      |   2 +-
 .../db/rows/UnfilteredRowIterators.java         |   2 +-
 .../apache/cassandra/db/view/TableViews.java    | 481 ++++++++++++++
 .../apache/cassandra/db/view/TemporalRow.java   | 601 ------------------
 src/java/org/apache/cassandra/db/view/View.java | 629 ++-----------------
 .../apache/cassandra/db/view/ViewBuilder.java   |  38 +-
 .../apache/cassandra/db/view/ViewManager.java   | 146 +----
 .../cassandra/db/view/ViewUpdateGenerator.java  | 549 ++++++++++++++++
 .../org/apache/cassandra/cql3/ViewTest.java     |  52 +-
 .../org/apache/cassandra/db/rows/RowsTest.java  |   6 +-
 28 files changed, 1400 insertions(+), 1329 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/cassandra/blob/a87fd715/src/java/org/apache/cassandra/config/CFMetaData.java
----------------------------------------------------------------------

http://git-wip-us.apache.org/repos/asf/cassandra/blob/a87fd715/src/java/org/apache/cassandra/config/ViewDefinition.java
----------------------------------------------------------------------

http://git-wip-us.apache.org/repos/asf/cassandra/blob/a87fd715/src/java/org/apache/cassandra/cql3/statements/CreateViewStatement.java
----------------------------------------------------------------------

http://git-wip-us.apache.org/repos/asf/cassandra/blob/a87fd715/src/java/org/apache/cassandra/cql3/statements/SelectStatement.java
----------------------------------------------------------------------

http://git-wip-us.apache.org/repos/asf/cassandra/blob/a87fd715/src/java/org/apache/cassandra/db/ColumnFamilyStore.java
----------------------------------------------------------------------

http://git-wip-us.apache.org/repos/asf/cassandra/blob/a87fd715/src/java/org/apache/cassandra/db/Keyspace.java
----------------------------------------------------------------------

http://git-wip-us.apache.org/repos/asf/cassandra/blob/a87fd715/src/java/org/apache/cassandra/db/SinglePartitionReadCommand.java
----------------------------------------------------------------------

http://git-wip-us.apache.org/repos/asf/cassandra/blob/a87fd715/src/java/org/apache/cassandra/db/Slices.java
----------------------------------------------------------------------

http://git-wip-us.apache.org/repos/asf/cassandra/blob/a87fd715/src/java/org/apache/cassandra/db/filter/RowFilter.java
----------------------------------------------------------------------

http://git-wip-us.apache.org/repos/asf/cassandra/blob/a87fd715/src/java/org/apache/cassandra/db/rows/AbstractCell.java
----------------------------------------------------------------------

http://git-wip-us.apache.org/repos/asf/cassandra/blob/a87fd715/src/java/org/apache/cassandra/db/rows/BTreeRow.java
----------------------------------------------------------------------

http://git-wip-us.apache.org/repos/asf/cassandra/blob/a87fd715/src/java/org/apache/cassandra/db/rows/BufferCell.java
----------------------------------------------------------------------

http://git-wip-us.apache.org/repos/asf/cassandra/blob/a87fd715/src/java/org/apache/cassandra/db/rows/Cell.java
----------------------------------------------------------------------

http://git-wip-us.apache.org/repos/asf/cassandra/blob/a87fd715/src/java/org/apache/cassandra/db/rows/ComplexColumnData.java
----------------------------------------------------------------------

http://git-wip-us.apache.org/repos/asf/cassandra/blob/a87fd715/src/java/org/apache/cassandra/db/rows/NativeCell.java
----------------------------------------------------------------------
diff --cc src/java/org/apache/cassandra/db/rows/NativeCell.java
index 9d816f3,0000000..5930332
mode 100644,000000..100644
--- a/src/java/org/apache/cassandra/db/rows/NativeCell.java
+++ b/src/java/org/apache/cassandra/db/rows/NativeCell.java
@@@ -1,151 -1,0 +1,156 @@@
 +/*
 + * Licensed to the Apache Software Foundation (ASF) under one
 + * or more contributor license agreements.  See the NOTICE file
 + * distributed with this work for additional information
 + * regarding copyright ownership.  The ASF licenses this file
 + * to you under the Apache License, Version 2.0 (the
 + * "License"); you may not use this file except in compliance
 + * with the License.  You may obtain a copy of the License at
 + *
 + *     http://www.apache.org/licenses/LICENSE-2.0
 + *
 + * Unless required by applicable law or agreed to in writing, software
 + * distributed under the License is distributed on an "AS IS" BASIS,
 + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
 + * See the License for the specific language governing permissions and
 + * limitations under the License.
 + */
 +package org.apache.cassandra.db.rows;
 +
 +import java.nio.ByteBuffer;
 +import java.nio.ByteOrder;
 +
 +import org.apache.cassandra.config.ColumnDefinition;
 +import org.apache.cassandra.utils.ObjectSizes;
 +import org.apache.cassandra.utils.concurrent.OpOrder;
 +import org.apache.cassandra.utils.memory.MemoryUtil;
 +import org.apache.cassandra.utils.memory.NativeAllocator;
 +
 +public class NativeCell extends AbstractCell
 +{
 +    private static final long EMPTY_SIZE = ObjectSizes.measure(new NativeCell());
 +
 +    private static final long HAS_CELLPATH = 0;
 +    private static final long TIMESTAMP = 1;
 +    private static final long TTL = 9;
 +    private static final long DELETION = 13;
 +    private static final long LENGTH = 17;
 +    private static final long VALUE = 21;
 +
 +    private final long peer;
 +
 +    private NativeCell()
 +    {
 +        super(null);
 +        this.peer = 0;
 +    }
 +
 +    public NativeCell(NativeAllocator allocator,
 +                      OpOrder.Group writeOp,
 +                      Cell cell)
 +    {
 +        this(allocator,
 +             writeOp,
 +             cell.column(),
 +             cell.timestamp(),
 +             cell.ttl(),
 +             cell.localDeletionTime(),
 +             cell.value(),
 +             cell.path());
 +    }
 +
 +    public NativeCell(NativeAllocator allocator,
 +                      OpOrder.Group writeOp,
 +                      ColumnDefinition column,
 +                      long timestamp,
 +                      int ttl,
 +                      int localDeletionTime,
 +                      ByteBuffer value,
 +                      CellPath path)
 +    {
 +        super(column);
 +        long size = simpleSize(value.remaining());
 +
 +        assert value.order() == ByteOrder.BIG_ENDIAN;
 +        assert column.isComplex() == (path != null);
 +        if (path != null)
 +        {
 +            assert path.size() == 1;
 +            size += 4 + path.get(0).remaining();
 +        }
 +
 +        if (size > Integer.MAX_VALUE)
 +            throw new IllegalStateException();
 +
 +        // cellpath? : timestamp : ttl : localDeletionTime : length : <data> : [cell path length] : [<cell path data>]
 +        peer = allocator.allocate((int) size, writeOp);
 +        MemoryUtil.setByte(peer + HAS_CELLPATH, (byte)(path == null ? 0 : 1));
 +        MemoryUtil.setLong(peer + TIMESTAMP, timestamp);
 +        MemoryUtil.setInt(peer + TTL, ttl);
 +        MemoryUtil.setInt(peer + DELETION, localDeletionTime);
 +        MemoryUtil.setInt(peer + LENGTH, value.remaining());
 +        MemoryUtil.setBytes(peer + VALUE, value);
 +
 +        if (path != null)
 +        {
 +            ByteBuffer pathbuffer = path.get(0);
 +            assert pathbuffer.order() == ByteOrder.BIG_ENDIAN;
 +
 +            long offset = peer + VALUE + value.remaining();
 +            MemoryUtil.setInt(offset, pathbuffer.remaining());
 +            MemoryUtil.setBytes(offset + 4, pathbuffer);
 +        }
 +    }
 +
 +    private static long simpleSize(int length)
 +    {
 +        return VALUE + length;
 +    }
 +
 +    public long timestamp()
 +    {
 +        return MemoryUtil.getLong(peer + TIMESTAMP);
 +    }
 +
 +    public int ttl()
 +    {
 +        return MemoryUtil.getInt(peer + TTL);
 +    }
 +
 +    public int localDeletionTime()
 +    {
 +        return MemoryUtil.getInt(peer + DELETION);
 +    }
 +
 +    public ByteBuffer value()
 +    {
 +        int length = MemoryUtil.getInt(peer + LENGTH);
 +        return MemoryUtil.getByteBuffer(peer + VALUE, length, ByteOrder.BIG_ENDIAN);
 +    }
 +
 +    public CellPath path()
 +    {
 +        if (MemoryUtil.getByte(peer+ HAS_CELLPATH) == 0)
 +            return null;
 +
 +        long offset = peer + VALUE + MemoryUtil.getInt(peer + LENGTH);
 +        int size = MemoryUtil.getInt(offset);
 +        return CellPath.create(MemoryUtil.getByteBuffer(offset + 4, size, ByteOrder.BIG_ENDIAN));
 +    }
 +
 +    public Cell withUpdatedValue(ByteBuffer newValue)
 +    {
 +        throw new UnsupportedOperationException();
 +    }
 +
++    public Cell withUpdatedColumn(ColumnDefinition column)
++    {
++        return new BufferCell(column, timestamp(), ttl(), localDeletionTime(), value(), path());
++    }
++
 +    public long unsharedHeapSizeExcludingData()
 +    {
 +        return EMPTY_SIZE;
 +    }
 +
 +}

http://git-wip-us.apache.org/repos/asf/cassandra/blob/a87fd715/src/java/org/apache/cassandra/db/rows/Row.java
----------------------------------------------------------------------

http://git-wip-us.apache.org/repos/asf/cassandra/blob/a87fd715/src/java/org/apache/cassandra/db/rows/RowDiffListener.java
----------------------------------------------------------------------

http://git-wip-us.apache.org/repos/asf/cassandra/blob/a87fd715/src/java/org/apache/cassandra/db/rows/UnfilteredRowIterators.java
----------------------------------------------------------------------

http://git-wip-us.apache.org/repos/asf/cassandra/blob/a87fd715/src/java/org/apache/cassandra/db/view/TableViews.java
----------------------------------------------------------------------
diff --cc src/java/org/apache/cassandra/db/view/TableViews.java
index 0000000,893bdd5..e97e01c
mode 000000,100644..100644
--- a/src/java/org/apache/cassandra/db/view/TableViews.java
+++ b/src/java/org/apache/cassandra/db/view/TableViews.java
@@@ -1,0 -1,481 +1,481 @@@
+ /*
+  * Licensed to the Apache Software Foundation (ASF) under one
+  * or more contributor license agreements.  See the NOTICE file
+  * distributed with this work for additional information
+  * regarding copyright ownership.  The ASF licenses this file
+  * to you under the Apache License, Version 2.0 (the
+  * "License"); you may not use this file except in compliance
+  * with the License.  You may obtain a copy of the License at
+  *
+  *     http://www.apache.org/licenses/LICENSE-2.0
+  *
+  * Unless required by applicable law or agreed to in writing, software
+  * distributed under the License is distributed on an "AS IS" BASIS,
+  * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+  * See the License for the specific language governing permissions and
+  * limitations under the License.
+  */
+ package org.apache.cassandra.db.view;
+ 
+ import java.util.*;
+ import java.util.concurrent.CopyOnWriteArrayList;
+ import java.util.concurrent.TimeUnit;
+ import java.util.concurrent.atomic.AtomicLong;
+ 
+ import com.google.common.collect.Iterables;
+ import com.google.common.collect.Iterators;
+ import com.google.common.collect.PeekingIterator;
+ 
+ import org.apache.cassandra.config.CFMetaData;
+ import org.apache.cassandra.db.*;
+ import org.apache.cassandra.db.commitlog.ReplayPosition;
+ import org.apache.cassandra.db.filter.*;
+ import org.apache.cassandra.db.rows.*;
+ import org.apache.cassandra.db.partitions.*;
+ import org.apache.cassandra.dht.Token;
+ import org.apache.cassandra.service.StorageProxy;
+ import org.apache.cassandra.utils.FBUtilities;
+ import org.apache.cassandra.utils.btree.BTreeSet;
+ 
+ 
+ /**
+  * Groups all the views for a given table.
+  */
+ public class TableViews extends AbstractCollection<View>
+ {
+     private final CFMetaData baseTableMetadata;
+ 
+     // We need this to be thread-safe, but the number of times this is changed (when a view is created in the keyspace)
+     // massively exceeds the number of time it's read (for every mutation on the keyspace), so a copy-on-write list is the best option.
+     private final List<View> views = new CopyOnWriteArrayList();
+ 
+     public TableViews(CFMetaData baseTableMetadata)
+     {
+         this.baseTableMetadata = baseTableMetadata;
+     }
+ 
+     public int size()
+     {
+         return views.size();
+     }
+ 
+     public Iterator<View> iterator()
+     {
+         return views.iterator();
+     }
+ 
+     public boolean contains(String viewName)
+     {
+         return Iterables.any(views, view -> view.name.equals(viewName));
+     }
+ 
+     public boolean add(View view)
+     {
+         // We should have validated that there is no existing view with this name at this point
+         assert !contains(view.name);
+         return views.add(view);
+     }
+ 
+     public Iterable<ColumnFamilyStore> allViewsCfs()
+     {
+         Keyspace keyspace = Keyspace.open(baseTableMetadata.ksName);
+         return Iterables.transform(views, view -> keyspace.getColumnFamilyStore(view.getDefinition().viewName));
+     }
+ 
+     public void forceBlockingFlush()
+     {
+         for (ColumnFamilyStore viewCfs : allViewsCfs())
+             viewCfs.forceBlockingFlush();
+     }
+ 
+     public void dumpMemtables()
+     {
+         for (ColumnFamilyStore viewCfs : allViewsCfs())
+             viewCfs.dumpMemtable();
+     }
+ 
+     public void truncateBlocking(long truncatedAt)
+     {
+         for (ColumnFamilyStore viewCfs : allViewsCfs())
+         {
+             ReplayPosition replayAfter = viewCfs.discardSSTables(truncatedAt);
+             SystemKeyspace.saveTruncationRecord(viewCfs, truncatedAt, replayAfter);
+         }
+     }
+ 
+     public void removeByName(String viewName)
+     {
+         views.removeIf(v -> v.name.equals(viewName));
+     }
+ 
+     /**
+      * Calculates and pushes updates to the views replicas. The replicas are determined by
+      * {@link ViewUtils#getViewNaturalEndpoint(String, Token, Token)}.
+      *
+      * @param update an update on the base table represented by this object.
+      * @param writeCommitLog whether we should write the commit log for the view updates.
+      * @param baseComplete time from epoch in ms that the local base mutation was (or will be) completed
+      */
+     public void pushViewReplicaUpdates(PartitionUpdate update, boolean writeCommitLog, AtomicLong baseComplete)
+     {
+         assert update.metadata().cfId.equals(baseTableMetadata.cfId);
+ 
+         Collection<View> views = updatedViews(update);
+         if (views.isEmpty())
+             return;
+ 
+         // Read modified rows
+         int nowInSec = FBUtilities.nowInSeconds();
+         SinglePartitionReadCommand command = readExistingRowsCommand(update, views, nowInSec);
+         if (command == null)
+             return;
+ 
+         ColumnFamilyStore cfs = Keyspace.openAndGetStore(update.metadata());
+         long start = System.nanoTime();
+         Collection<Mutation> mutations;
 -        try (ReadOrderGroup orderGroup = command.startOrderGroup();
++        try (ReadExecutionController orderGroup = command.executionController();
+              UnfilteredRowIterator existings = UnfilteredPartitionIterators.getOnlyElement(command.executeLocally(orderGroup), command);
+              UnfilteredRowIterator updates = update.unfilteredIterator())
+         {
+             mutations = generateViewUpdates(views, updates, existings, nowInSec);
+         }
+         Keyspace.openAndGetStore(update.metadata()).metric.viewReadTime.update(System.nanoTime() - start, TimeUnit.NANOSECONDS);
+ 
+         if (!mutations.isEmpty())
+             StorageProxy.mutateMV(update.partitionKey().getKey(), mutations, writeCommitLog, baseComplete);
+     }
+ 
+     /**
+      * Given some updates on the base table of this object and the existing values for the rows affected by that update, generates the
+      * mutation to be applied to the provided views.
+      *
+      * @param views the views potentially affected by {@code updates}.
+      * @param updates the base table updates being applied.
+      * @param existings the existing values for the rows affected by {@code updates}. This is used to decide if a view is
+      * obsoleted by the update and should be removed, gather the values for columns that may not be part of the update if
+      * a new view entry needs to be created, and compute the minimal updates to be applied if the view entry isn't changed
+      * 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.
+      * @return the mutations to apply to the {@code views}. This can be empty.
+      */
+     public Collection<Mutation> generateViewUpdates(Collection<View> views, UnfilteredRowIterator updates, UnfilteredRowIterator existings, int nowInSec)
+     {
+         assert updates.metadata().cfId.equals(baseTableMetadata.cfId);
+ 
+         List<ViewUpdateGenerator> generators = new ArrayList<>(views.size());
+         for (View view : views)
+             generators.add(new ViewUpdateGenerator(view, updates.partitionKey(), nowInSec));
+ 
+         DeletionTracker existingsDeletion = new DeletionTracker(existings.partitionLevelDeletion());
+         DeletionTracker updatesDeletion = new DeletionTracker(updates.partitionLevelDeletion());
+ 
+         /*
+          * We iterate through the updates and the existing rows in parallel. This allows us to know the consequence
+          * on the view of each update.
+          */
+         PeekingIterator<Unfiltered> existingsIter = Iterators.peekingIterator(existings);
+         PeekingIterator<Unfiltered> updatesIter = Iterators.peekingIterator(updates);
+ 
+         while (existingsIter.hasNext() && updatesIter.hasNext())
+         {
+             Unfiltered existing = existingsIter.peek();
+             Unfiltered update = updatesIter.peek();
+ 
+             Row existingRow;
+             Row updateRow;
+             int cmp = baseTableMetadata.comparator.compare(update, existing);
+             if (cmp < 0)
+             {
+                 // We have an update where there was nothing before
+                 if (update.isRangeTombstoneMarker())
+                 {
+                     updatesDeletion.update(updatesIter.next());
+                     continue;
+                 }
+ 
+                 updateRow = ((Row)updatesIter.next()).withRowDeletion(updatesDeletion.currentDeletion());
+                 existingRow = emptyRow(updateRow.clustering(), existingsDeletion.currentDeletion());
+             }
+             else if (cmp > 0)
+             {
+                 // We have something existing but no update (which will happen either because it's a range tombstone marker in
+                 // existing, or because we've fetched the existing row due to some partition/range deletion in the updates)
+                 if (existing.isRangeTombstoneMarker())
+                 {
+                     existingsDeletion.update(existingsIter.next());
+                     continue;
+                 }
+ 
+                 existingRow = ((Row)existingsIter.next()).withRowDeletion(existingsDeletion.currentDeletion());
+                 updateRow = emptyRow(existingRow.clustering(), updatesDeletion.currentDeletion());
+ 
+                 // The way we build the read command used for existing rows, we should always have updatesDeletion.currentDeletion()
+                 // that is not live, since we wouldn't have read the existing row otherwise. And we could assert that, but if we ever
+                 // change the read method so that it can slightly over-read in some case, that would be an easily avoiding bug lurking,
+                 // so we just handle the case.
+                 if (updateRow == null)
+                     continue;
+             }
+             else
+             {
+                 // We're updating a row that had pre-existing data
+                 if (update.isRangeTombstoneMarker())
+                 {
+                     assert existing.isRangeTombstoneMarker();
+                     updatesDeletion.update(updatesIter.next());
+                     existingsDeletion.update(existingsIter.next());
+                     continue;
+                 }
+ 
+                 assert !existing.isRangeTombstoneMarker();
+                 existingRow = ((Row)existingsIter.next()).withRowDeletion(existingsDeletion.currentDeletion());
+                 updateRow = ((Row)updatesIter.next()).withRowDeletion(updatesDeletion.currentDeletion());
+             }
+ 
+             addToViewUpdateGenerators(existingRow, updateRow, generators, nowInSec);
+         }
+ 
+         // We only care about more existing rows if the update deletion isn't live, i.e. if we had a partition deletion
+         if (!updatesDeletion.currentDeletion().isLive())
+         {
+             while (existingsIter.hasNext())
+             {
+                 Unfiltered existing = existingsIter.next();
+                 // If it's a range tombstone, we don't care, we're only looking for existing entry that gets deleted by
+                 // the new partition deletion
+                 if (existing.isRangeTombstoneMarker())
+                     continue;
+ 
+                 Row existingRow = (Row)existing;
+                 addToViewUpdateGenerators(existingRow, emptyRow(existingRow.clustering(), updatesDeletion.currentDeletion()), generators, nowInSec);
+             }
+         }
+         while (updatesIter.hasNext())
+         {
+             Unfiltered update = updatesIter.next();
+             // If it's a range tombstone, it removes nothing pre-exisiting, so we can ignore it for view updates
+             if (update.isRangeTombstoneMarker())
+                 continue;
+ 
+             Row updateRow = (Row)update;
+             addToViewUpdateGenerators(emptyRow(updateRow.clustering(), DeletionTime.LIVE), updateRow, generators, nowInSec);
+         }
+ 
+         return buildMutations(baseTableMetadata, generators);
+     }
+ 
+     /**
+      * Return the views that are potentially updated by the provided updates.
+      *
+      * @param updates the updates applied to the base table.
+      * @return the views affected by {@code updates}.
+      */
+     public Collection<View> updatedViews(PartitionUpdate updates)
+     {
+         List<View> matchingViews = new ArrayList<>(views.size());
+ 
+         for (View view : views)
+         {
+             ReadQuery selectQuery = view.getReadQuery();
+             if (!selectQuery.selectsKey(updates.partitionKey()))
+                 continue;
+ 
+             matchingViews.add(view);
+         }
+         return matchingViews;
+     }
+ 
+     /**
+      * Returns the command to use to read the existing rows required to generate view updates for the provided base
+      * base updates.
+      *
+      * @param updates the base table updates being applied.
+      * @param views the views potentially affected by {@code updates}.
+      * @param nowInSec the current time in seconds.
+      * @return the command to use to read the base table rows required to generate view updates for {@code updates}.
+      */
+     private SinglePartitionReadCommand readExistingRowsCommand(PartitionUpdate updates, Collection<View> views, int nowInSec)
+     {
+         Slices.Builder sliceBuilder = null;
+         DeletionInfo deletionInfo = updates.deletionInfo();
+         CFMetaData metadata = updates.metadata();
+         DecoratedKey key = updates.partitionKey();
+         // TODO: This is subtle: we need to gather all the slices that we have to fetch between partition del, range tombstones and rows.
+         if (!deletionInfo.isLive())
+         {
+             sliceBuilder = new Slices.Builder(metadata.comparator);
+             // Everything covered by a deletion might invalidate an existing view entry, which means we must read it to know. In practice
+             // though, the views involved might filter some base table clustering columns, in which case we can restrict what we read
+             // using those restrictions.
+             // If there is a partition deletion, then we can simply take each slices from each view select filter. They may overlap but
+             // the Slices.Builder handles that for us. Note that in many case this will just involve reading everything (as soon as any
+             // view involved has no clustering restrictions for instance).
+             // For range tombstone, we should theoretically take the difference between the range tombstoned and the slices selected
+             // by every views, but as we don't an easy way to compute that right now, we keep it simple and just use the tombstoned
+             // range.
+             // TODO: we should improve that latter part.
+             if (!deletionInfo.getPartitionDeletion().isLive())
+             {
+                 for (View view : views)
+                     sliceBuilder.addAll(view.getSelectStatement().clusteringIndexFilterAsSlices());
+             }
+             else
+             {
+                 assert deletionInfo.hasRanges();
+                 Iterator<RangeTombstone> iter = deletionInfo.rangeIterator(false);
+                 while (iter.hasNext())
+                     sliceBuilder.add(iter.next().deletedSlice());
+             }
+         }
+ 
+         // We need to read every row that is updated, unless we can prove that it has no impact on any view entries.
+ 
+         // If we had some slices from the deletions above, we'll continue using that. Otherwise, it's more efficient to build
+         // a names query.
+         BTreeSet.Builder<Clustering> namesBuilder = sliceBuilder == null ? BTreeSet.builder(metadata.comparator) : null;
+         for (Row row : updates)
+         {
+             // Don't read the existing state if we can prove the update won't affect any views
+             if (!affectsAnyViews(key, row, views))
+                 continue;
+ 
+             if (namesBuilder == null)
+                 sliceBuilder.add(Slice.make(row.clustering()));
+             else
+                 namesBuilder.add(row.clustering());
+         }
+ 
+         NavigableSet<Clustering> names = namesBuilder == null ? null : namesBuilder.build();
+         // If we have a slice builder, it means we had some deletions and we have to read. But if we had
+         // only row updates, it's possible none of them affected the views, in which case we have nothing
+         // to do.
+         if (names != null && names.isEmpty())
+             return null;
+ 
+         ClusteringIndexFilter clusteringFilter = names == null
+                                                ? new ClusteringIndexSliceFilter(sliceBuilder.build(), false)
+                                                : new ClusteringIndexNamesFilter(names, false);
+         // 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);
+         // 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.
+         // TODO: we could still make sense to special case for when there is a single view and a small number of updates (and
+         // no deletions). Indeed, in that case we could check whether any of the update modify any of the restricted regular
+         // column, and if that's not the case we could use view filter. We keep it simple for now though.
+         RowFilter rowFilter = RowFilter.NONE;
+         return SinglePartitionReadCommand.create(metadata, nowInSec, queriedColumns, rowFilter, DataLimits.NONE, key, clusteringFilter);
+     }
+ 
+     private boolean affectsAnyViews(DecoratedKey partitionKey, Row update, Collection<View> views)
+     {
+         for (View view : views)
+         {
+             if (view.mayBeAffectedBy(partitionKey, update))
+                 return true;
+         }
+         return false;
+     }
+ 
+     /**
+      * Given an existing base row and the update that we're going to apply to this row, generate the modifications
+      * to apply to MVs using the provided {@code ViewUpdateGenerator}s.
+      *
+      * @param existingBaseRow the base table row as it is before an update.
+      * @param updateBaseRow the newly updates made to {@code existingBaseRow}.
+      * @param generators the view update generators to add the new changes to.
+      * @param nowInSec the current time in seconds. Used to decide if data is live or not.
+      */
+     private static void addToViewUpdateGenerators(Row existingBaseRow, Row updateBaseRow, Collection<ViewUpdateGenerator> generators, int nowInSec)
+     {
+         // Having existing empty is useful, it just means we'll insert a brand new entry for updateBaseRow,
+         // but if we have no update at all, we shouldn't get there.
+         assert !updateBaseRow.isEmpty();
+ 
+         // We allow existingBaseRow to be null, which we treat the same as being empty as an small optimization
+         // to avoid allocating empty row objects when we know there was nothing existing.
+         Row mergedBaseRow = existingBaseRow == null ? updateBaseRow : Rows.merge(existingBaseRow, updateBaseRow, nowInSec);
+         for (ViewUpdateGenerator generator : generators)
+             generator.addBaseTableUpdate(existingBaseRow, mergedBaseRow);
+     }
+ 
+     private static Row emptyRow(Clustering clustering, DeletionTime deletion)
+     {
+         // Returning null for an empty row is slightly ugly, but the case where there is no pre-existing row is fairly common
+         // (especially when building the view), so we want to avoid a dummy allocation of an empty row every time.
+         // And MultiViewUpdateBuilder knows how to deal with that.
+         return deletion.isLive() ? null : BTreeRow.emptyDeletedRow(clustering, Row.Deletion.regular(deletion));
+     }
+ 
+     /**
+      * Extracts (and potentially groups) the mutations generated by the provided view update generator.
+      * Returns the mutation that needs to be done to the views given the base table updates
+      * passed to {@link #addBaseTableUpdate}.
+      *
+      * @param baseTableMetadata the metadata for the base table being updated.
+      * @param generators the generators from which to extract the view mutations from.
+      * @return the mutations created by all the generators in {@code generators}.
+      */
+     private Collection<Mutation> buildMutations(CFMetaData baseTableMetadata, List<ViewUpdateGenerator> generators)
+     {
+         // One view is probably common enough and we can optimize a bit easily
+         if (generators.size() == 1)
+         {
+             Collection<PartitionUpdate> updates = generators.get(0).generateViewUpdates();
+             List<Mutation> mutations = new ArrayList<>(updates.size());
+             for (PartitionUpdate update : updates)
+                 mutations.add(new Mutation(update));
+             return mutations;
+         }
+ 
+         Map<DecoratedKey, Mutation> mutations = new HashMap<>();
+         for (ViewUpdateGenerator generator : generators)
+         {
+             for (PartitionUpdate update : generator.generateViewUpdates())
+             {
+                 DecoratedKey key = update.partitionKey();
+                 Mutation mutation = mutations.get(key);
+                 if (mutation == null)
+                 {
+                     mutation = new Mutation(baseTableMetadata.ksName, key);
+                     mutations.put(key, mutation);
+                 }
+                 mutation.add(update);
+             }
+         }
+         return mutations.values();
+     }
+ 
+     /**
+      * A simple helper that tracks for a given {@code UnfilteredRowIterator} what is the current deletion at any time of the
+      * iteration. It will be the currently open range tombstone deletion if there is one and the partition deletion otherwise.
+      */
+     private static class DeletionTracker
+     {
+         private final DeletionTime partitionDeletion;
+         private DeletionTime deletion;
+ 
+         public DeletionTracker(DeletionTime partitionDeletion)
+         {
+             this.partitionDeletion = partitionDeletion;
+         }
+ 
+         public void update(Unfiltered marker)
+         {
+             assert marker instanceof RangeTombstoneMarker;
+             RangeTombstoneMarker rtm = (RangeTombstoneMarker)marker;
+             this.deletion = rtm.isOpen(false)
+                           ? rtm.openDeletionTime(false)
+                           : null;
+         }
+ 
+         public DeletionTime currentDeletion()
+         {
+             return deletion == null ? partitionDeletion : deletion;
+         }
+     }
+ }

http://git-wip-us.apache.org/repos/asf/cassandra/blob/a87fd715/src/java/org/apache/cassandra/db/view/ViewBuilder.java
----------------------------------------------------------------------
diff --cc src/java/org/apache/cassandra/db/view/ViewBuilder.java
index 8944122,b2b409b..65e26e2
--- a/src/java/org/apache/cassandra/db/view/ViewBuilder.java
+++ b/src/java/org/apache/cassandra/db/view/ViewBuilder.java
@@@ -77,28 -76,22 +77,22 @@@ public class ViewBuilder extends Compac
          if (!selectQuery.selectsKey(key))
              return;
  
-         QueryPager pager = view.getSelectStatement().internalReadForView(key, FBUtilities.nowInSeconds()).getPager(null, Server.CURRENT_VERSION);
+         int nowInSec = FBUtilities.nowInSeconds();
+         SinglePartitionReadCommand command = view.getSelectStatement().internalReadForView(key, nowInSec);
  
-         while (!pager.isExhausted())
+         // We're rebuilding everything from what's on disk, so we read everything, consider that as new updates
+         // and pretend that there is nothing pre-existing.
+         UnfilteredRowIterator empty = UnfilteredRowIterators.noRowsIterator(baseCfs.metadata, key, Rows.EMPTY_STATIC_ROW, DeletionTime.LIVE, false);
+ 
+         Collection<Mutation> mutations;
 -        try (ReadOrderGroup orderGroup = command.startOrderGroup();
++        try (ReadExecutionController orderGroup = command.executionController();
+              UnfilteredRowIterator data = UnfilteredPartitionIterators.getOnlyElement(command.executeLocally(orderGroup), command))
          {
-            try (ReadExecutionController executionController = pager.executionController();
-                 PartitionIterator partitionIterator = pager.fetchPageInternal(128, executionController))
-            {
-                if (!partitionIterator.hasNext())
-                    return;
- 
-                try (RowIterator rowIterator = partitionIterator.next())
-                {
-                    FilteredPartition partition = FilteredPartition.create(rowIterator);
-                    TemporalRow.Set temporalRows = view.getTemporalRowSet(partition, null, true);
- 
-                    Collection<Mutation> mutations = view.createMutations(partition, temporalRows, true);
- 
-                    if (mutations != null)
-                        StorageProxy.mutateMV(key.getKey(), mutations, true, noBase);
-                }
-            }
+             mutations = baseCfs.keyspace.viewManager.forTable(baseCfs.metadata).generateViewUpdates(Collections.singleton(view), data, empty, nowInSec);
          }
+ 
+         if (!mutations.isEmpty())
+             StorageProxy.mutateMV(key.getKey(), mutations, true, noBase);
      }
  
      public void run()

http://git-wip-us.apache.org/repos/asf/cassandra/blob/a87fd715/src/java/org/apache/cassandra/db/view/ViewManager.java
----------------------------------------------------------------------
diff --cc src/java/org/apache/cassandra/db/view/ViewManager.java
index 6b8fe0d,fd04b97..bd73733
--- a/src/java/org/apache/cassandra/db/view/ViewManager.java
+++ b/src/java/org/apache/cassandra/db/view/ViewManager.java
@@@ -30,24 -31,24 +31,25 @@@ import org.apache.cassandra.config.CFMe
  import org.apache.cassandra.config.DatabaseDescriptor;
  import org.apache.cassandra.config.ViewDefinition;
  import org.apache.cassandra.db.*;
- import org.apache.cassandra.db.commitlog.ReplayPosition;
- import org.apache.cassandra.db.partitions.PartitionUpdate;
- import org.apache.cassandra.dht.Token;
+ import org.apache.cassandra.db.rows.*;
+ import org.apache.cassandra.db.partitions.*;
 -
 +import org.apache.cassandra.repair.SystemDistributedKeyspace;
- import org.apache.cassandra.service.StorageProxy;
 +import org.apache.cassandra.service.StorageService;
  
- import org.slf4j.Logger;
- import org.slf4j.LoggerFactory;
- 
  /**
   * Manages {@link View}'s for a single {@link ColumnFamilyStore}. All of the views for that table are created when this
   * manager is initialized.
   *
   * The main purposes of the manager are to provide a single location for updates to be vetted to see whether they update
 - * any views {@link ViewManager#updatesAffectView(Collection, boolean)}, provide locks to prevent multiple
 - * updates from creating incoherent updates in the view {@link ViewManager#acquireLockFor(ByteBuffer)}, and
 + * any views {@link #updatesAffectView(Collection, boolean)}, provide locks to prevent multiple
 + * updates from creating incoherent updates in the view {@link #acquireLockFor(int)}, and
   * to affect change on the view.
+  *
+  * TODO: I think we can get rid of that class. For addition/removal of view by names, we could move it Keyspace. And we
+  * not sure it's even worth keeping viewsByName as none of the related operation are performance sensitive so we could
+  * find the view by iterating over the CFStore.viewManager directly.
+  * For the lock, it could move to Keyspace too, but I don't remmenber why it has to be at the keyspace level and if it
+  * can be at the table level, maybe that's where it should be.
   */
  public class ViewManager
  {
@@@ -250,9 -148,8 +163,9 @@@
          if (view == null)
              return;
  
-         forTable(view.getDefinition().baseTableId).removeView(name);
+         forTable(view.getDefinition().baseTableMetadata()).removeByName(name);
          SystemKeyspace.setViewRemoved(keyspace.getName(), view.name);
 +        SystemDistributedKeyspace.setViewRemoved(keyspace.getName(), view.name);
      }
  
      public void buildAllViews()
@@@ -261,22 -158,23 +174,23 @@@
              view.build();
      }
  
-     public ForStore forTable(UUID baseId)
+     public TableViews forTable(CFMetaData metadata)
      {
-         ForStore forStore = viewManagersByStore.get(baseId);
-         if (forStore == null)
+         UUID baseId = metadata.cfId;
+         TableViews views = viewsByBaseTable.get(baseId);
+         if (views == null)
          {
-             forStore = new ForStore();
-             ForStore previous = viewManagersByStore.put(baseId, forStore);
+             views = new TableViews(metadata);
+             TableViews previous = viewsByBaseTable.putIfAbsent(baseId, views);
              if (previous != null)
-                 forStore = previous;
+                 views = previous;
          }
-         return forStore;
+         return views;
      }
  
 -    public static Lock acquireLockFor(ByteBuffer key)
 +    public static Lock acquireLockFor(int keyAndCfidHash)
      {
 -        Lock lock = LOCKS.get(key);
 +        Lock lock = LOCKS.get(keyAndCfidHash);
  
          if (lock.tryLock())
              return lock;

http://git-wip-us.apache.org/repos/asf/cassandra/blob/a87fd715/src/java/org/apache/cassandra/db/view/ViewUpdateGenerator.java
----------------------------------------------------------------------
diff --cc src/java/org/apache/cassandra/db/view/ViewUpdateGenerator.java
index 0000000,af025cb..4c6dbb7
mode 000000,100644..100644
--- a/src/java/org/apache/cassandra/db/view/ViewUpdateGenerator.java
+++ b/src/java/org/apache/cassandra/db/view/ViewUpdateGenerator.java
@@@ -1,0 -1,549 +1,549 @@@
+ /*
+  * Licensed to the Apache Software Foundation (ASF) under one
+  * or more contributor license agreements.  See the NOTICE file
+  * distributed with this work for additional information
+  * regarding copyright ownership.  The ASF licenses this file
+  * to you under the Apache License, Version 2.0 (the
+  * "License"); you may not use this file except in compliance
+  * with the License.  You may obtain a copy of the License at
+  *
+  *     http://www.apache.org/licenses/LICENSE-2.0
+  *
+  * Unless required by applicable law or agreed to in writing, software
+  * distributed under the License is distributed on an "AS IS" BASIS,
+  * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+  * See the License for the specific language governing permissions and
+  * limitations under the License.
+  */
+ package org.apache.cassandra.db.view;
+ 
+ import java.nio.ByteBuffer;
+ import java.util.*;
+ 
+ import com.google.common.collect.Iterators;
+ import com.google.common.collect.PeekingIterator;
+ 
+ import org.apache.cassandra.config.CFMetaData;
+ import org.apache.cassandra.config.ColumnDefinition;
+ import org.apache.cassandra.config.ViewDefinition;
+ import org.apache.cassandra.db.*;
+ import org.apache.cassandra.db.rows.*;
+ import org.apache.cassandra.db.partitions.*;
+ import org.apache.cassandra.db.marshal.AbstractType;
+ import org.apache.cassandra.db.marshal.CompositeType;
+ 
+ /**
+  * Creates the updates to apply to a view given the existing rows in the base
+  * table and the updates that we're applying to them (this handles updates
+  * on a single partition only).
+  *
+  * This class is used by passing the updates made to the base table to
+  * {@link #addBaseTableUpdate} and calling {@link #generateViewUpdates} once all updates have
+  * been handled to get the resulting view mutations.
+  */
+ public class ViewUpdateGenerator
+ {
+     private final View view;
+     private final int nowInSec;
+ 
+     private final CFMetaData baseMetadata;
+     private final DecoratedKey baseDecoratedKey;
+     private final ByteBuffer[] basePartitionKey;
+ 
+     private final CFMetaData viewMetadata;
+ 
+     private final Map<DecoratedKey, PartitionUpdate> updates = new HashMap<>();
+ 
+     // Reused internally to build a new entry
+     private final ByteBuffer[] currentViewEntryPartitionKey;
+     private final Row.Builder currentViewEntryBuilder;
+ 
+     /**
+      * The type of type update action to perform to the view for a given base table
+      * update.
+      */
+     private enum UpdateAction
+     {
+         NONE,            // There was no view entry and none should be added
+         NEW_ENTRY,       // There was no entry but there is one post-update
+         DELETE_OLD,      // There was an entry but there is nothing after update
+         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}.
+      *
+      * @param view the view for which this will be building updates for.
+      * @param basePartitionKey the partition key for the base table partition for which
+      * we'll handle updates for.
+      * @param nowInSec the current time in seconds. Used to decide if data are live or not
+      * and as base reference for new deletions.
+      */
+     public ViewUpdateGenerator(View view, DecoratedKey basePartitionKey, int nowInSec)
+     {
+         this.view = view;
+         this.nowInSec = nowInSec;
+ 
+         this.baseMetadata = view.getDefinition().baseTableMetadata();
+         this.baseDecoratedKey = basePartitionKey;
+         this.basePartitionKey = extractKeyComponents(basePartitionKey, baseMetadata.getKeyValidator());
+ 
+         this.viewMetadata = view.getDefinition().metadata;
+ 
+         this.currentViewEntryPartitionKey = new ByteBuffer[viewMetadata.partitionKeyColumns().size()];
+         this.currentViewEntryBuilder = BTreeRow.sortedBuilder();
+     }
+ 
+     private static ByteBuffer[] extractKeyComponents(DecoratedKey partitionKey, AbstractType<?> type)
+     {
+         return type instanceof CompositeType
+              ? ((CompositeType)type).split(partitionKey.getKey())
+              : new ByteBuffer[]{ partitionKey.getKey() };
+     }
+ 
+     /**
+      * Adds to this generator the updates to be made to the view given a base table row
+      * before and after an update.
+      *
+      * @param existingBaseRow the base table row as it is before an update.
+      * @param mergedBaseRow the base table row after the update is applied (note that
+      * this is not just the new update, but rather the resulting row).
+      */
+     public void addBaseTableUpdate(Row existingBaseRow, Row mergedBaseRow)
+     {
+         switch (updateAction(existingBaseRow, mergedBaseRow))
+         {
+             case NONE:
+                 return;
+             case NEW_ENTRY:
+                 createEntry(mergedBaseRow);
+                 return;
+             case DELETE_OLD:
+                 deleteOldEntry(existingBaseRow);
+                 return;
+             case UPDATE_EXISTING:
+                 updateEntry(existingBaseRow, mergedBaseRow);
+                 return;
+             case SWITCH_ENTRY:
+                 createEntry(mergedBaseRow);
+                 deleteOldEntry(existingBaseRow);
+                 return;
+         }
+     }
+ 
+     /**
+      * Returns the updates that needs to be done to the view given the base table updates
+      * passed to {@link #generateViewMutations}.
+      *
+      * @return the updates to do to the view.
+      */
+     public Collection<PartitionUpdate> generateViewUpdates()
+     {
+         return updates.values();
+     }
+ 
+     /**
+      * Compute which type of action needs to be performed to the view for a base table row
+      * before and after an update.
+      */
+     private UpdateAction updateAction(Row existingBaseRow, Row mergedBaseRow)
+     {
+         // Having existing empty is useful, it just means we'll insert a brand new entry for mergedBaseRow,
+         // but if we have no update at all, we shouldn't get there.
+         assert !mergedBaseRow.isEmpty();
+ 
+         // Note that none of the base PK columns will differ since we're intrinsically dealing
+         // with the same base row. So we have to check 3 things:
+         //   1) that the clustering doesn't have a null, which can happen for compact tables. If that's the case,
+         //      there is no corresponding entries.
+         //   2) if there is a column not part of the base PK in the view PK, whether it is changed by the update.
+         //   3) whether mergedBaseRow actually match the view SELECT filter
+ 
+         if (baseMetadata.isCompactTable())
+         {
+             Clustering clustering = mergedBaseRow.clustering();
+             for (int i = 0; i < clustering.size(); i++)
+             {
+                 if (clustering.get(i) == null)
+                     return UpdateAction.NONE;
+             }
+         }
+ 
+         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.
+ 
+             // Note that we allow existingBaseRow to be null and treat it as empty (see MultiViewUpdateBuilder.generateViewsMutations).
+             boolean existingHasLiveData = existingBaseRow != null && existingBaseRow.hasLiveData(nowInSec);
+             boolean mergedHasLiveData = mergedBaseRow.hasLiveData(nowInSec);
+             return existingHasLiveData
+                  ? (mergedHasLiveData ? UpdateAction.UPDATE_EXISTING : UpdateAction.DELETE_OLD)
+                  : (mergedHasLiveData ? UpdateAction.NEW_ENTRY : UpdateAction.NONE);
+         }
+ 
+         ColumnDefinition baseColumn = view.baseNonPKColumnsInViewPK.get(0);
+         assert !baseColumn.isComplex() : "A complex column couldn't be part of the view PK";
+         Cell before = existingBaseRow == null ? null : existingBaseRow.getCell(baseColumn);
+         Cell after = mergedBaseRow.getCell(baseColumn);
+ 
+         // If the update didn't modified this column, the cells will be the same object so it's worth checking
+         if (before == after)
+             return before == null ? UpdateAction.NONE : UpdateAction.UPDATE_EXISTING;
+ 
+         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
+              : UpdateAction.SWITCH_ENTRY;
+     }
+ 
+     private boolean matchesViewFilter(Row baseRow)
+     {
+         return view.matchesViewFilter(baseDecoratedKey, baseRow, nowInSec);
+     }
+ 
+     private boolean isLive(Cell cell)
+     {
+         return cell != null && cell.isLive(nowInSec);
+     }
+ 
+     /**
+      * Creates a view entry corresponding to the provided base row.
+      * <p>
+      * This method checks that the base row does match the view filter before applying it.
+      */
+     private void createEntry(Row baseRow)
+     {
+         // Before create a new entry, make sure it matches the view filter
+         if (!matchesViewFilter(baseRow))
+             return;
+ 
+         startNewUpdate(baseRow);
+         currentViewEntryBuilder.addPrimaryKeyLivenessInfo(computeLivenessInfoForEntry(baseRow));
+         currentViewEntryBuilder.addRowDeletion(baseRow.deletion());
+ 
+         for (ColumnData data : baseRow)
+         {
+             ColumnDefinition viewColumn = view.getViewColumn(data.column());
+             // If that base table column is not denormalized in the view, we had nothing to do.
+             // Alose, if it's part of the view PK it's already been taken into account in the clustering.
+             if (viewColumn == null || viewColumn.isPrimaryKeyColumn())
+                 continue;
+ 
+             addColumnData(viewColumn, data);
+         }
+ 
+         submitUpdate();
+     }
+ 
+     /**
+      * Creates the updates to apply to the existing view entry given the base table row before
+      * and after the update, assuming that the update hasn't changed to which view entry the
+      * row correspond (that is, we know the columns composing the view PK haven't changed).
+      * <p>
+      * This method checks that the base row (before and after) does match the view filter before
+      * applying anything.
+      */
+     private void updateEntry(Row existingBaseRow, Row mergedBaseRow)
+     {
+         // While we know existingBaseRow and mergedBaseRow are corresponding to the same view entry,
+         // they may not match the view filter.
+         if (!matchesViewFilter(existingBaseRow))
+         {
+             createEntry(mergedBaseRow);
+             return;
+         }
+         if (!matchesViewFilter(mergedBaseRow))
+         {
+             deleteOldEntryInternal(existingBaseRow);
+             return;
+         }
+ 
+         startNewUpdate(mergedBaseRow);
+ 
+         // In theory, it may be the PK liveness and row deletion hasn't been change by the update
+         // and we could condition the 2 additions below. In practice though, it's as fast (if not
+         // faster) to compute those info than to check if they have changed so we keep it simple.
+         currentViewEntryBuilder.addPrimaryKeyLivenessInfo(computeLivenessInfoForEntry(mergedBaseRow));
+         currentViewEntryBuilder.addRowDeletion(mergedBaseRow.deletion());
+ 
+         // 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
+         // mergedBaseRow (note that including more cells than we strictly should isn't a problem
+         // for correction, so even if the code change and pointer equality don't work anymore, it'll
+         // only a slightly inefficiency which we can fix then).
+         // Note: we could alternatively use Rows.diff() for this, but because it is a bit more generic
+         // than what we need here, it's also a bit less efficient (it allocates more in particular),
+         // and this might be called a lot of time for view updates. So, given that this is not a whole
+         // lot of code anyway, it's probably doing the diff manually.
+         PeekingIterator<ColumnData> existingIter = Iterators.peekingIterator(existingBaseRow.iterator());
+         for (ColumnData mergedData : mergedBaseRow)
+         {
+             ColumnDefinition baseColumn = mergedData.column();
+             ColumnDefinition viewColumn = view.getViewColumn(baseColumn);
+             // If that base table column is not denormalized in the view, we had nothing to do.
+             // Alose, if it's part of the view PK it's already been taken into account in the clustering.
+             if (viewColumn == null || viewColumn.isPrimaryKeyColumn())
+                 continue;
+ 
+             ColumnData existingData = null;
+             // Find if there is data for that column in the existing row
+             while (existingIter.hasNext())
+             {
+                 int cmp = baseColumn.compareTo(existingIter.peek().column());
+                 if (cmp < 0)
+                     break;
+ 
+                 ColumnData next = existingIter.next();
+                 if (cmp == 0)
+                 {
+                     existingData = next;
+                     break;
+                 }
+             }
+ 
+             if (existingData == null)
+             {
+                 addColumnData(viewColumn, mergedData);
+                 continue;
+             }
+ 
+             if (mergedData == existingData)
+                 continue;
+ 
+             if (baseColumn.isComplex())
+             {
+                 ComplexColumnData mergedComplexData = (ComplexColumnData)mergedData;
+                 ComplexColumnData existingComplexData = (ComplexColumnData)existingData;
+                 if (mergedComplexData.complexDeletion().supersedes(existingComplexData.complexDeletion()))
+                     currentViewEntryBuilder.addComplexDeletion(viewColumn, mergedComplexData.complexDeletion());
+ 
+                 PeekingIterator<Cell> existingCells = Iterators.peekingIterator(existingComplexData.iterator());
+                 for (Cell mergedCell : mergedComplexData)
+                 {
+                     Cell existingCell = null;
+                     // Find if there is corresponding cell in the existing row
+                     while (existingCells.hasNext())
+                     {
+                         int cmp = baseColumn.cellPathComparator().compare(mergedCell.path(), existingCells.peek().path());
+                         if (cmp > 0)
+                             break;
+ 
+                         Cell next = existingCells.next();
+                         if (cmp == 0)
+                         {
+                             existingCell = next;
+                             break;
+                         }
+                     }
+ 
+                     if (mergedCell != existingCell)
+                         addCell(viewColumn, mergedCell);
+                 }
+             }
+             else
+             {
+                 // Note that we've already eliminated the case where merged == existing
+                 addCell(viewColumn, (Cell)mergedData);
+             }
+         }
+ 
+         submitUpdate();
+     }
+ 
+     /**
+      * Deletes the view entry corresponding to the provided base row.
+      * <p>
+      * This method checks that the base row does match the view filter before bothering.
+      */
+     private void deleteOldEntry(Row existingBaseRow)
+     {
+         // 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);
+     }
+ 
+     private void deleteOldEntryInternal(Row existingBaseRow)
+     {
+         startNewUpdate(existingBaseRow);
+         DeletionTime dt = new DeletionTime(computeTimestampForEntryDeletion(existingBaseRow), nowInSec);
+         currentViewEntryBuilder.addRowDeletion(Row.Deletion.shadowable(dt));
+         submitUpdate();
+     }
+ 
+     /**
+      * Computes the partition key and clustering for a new view entry, and setup the internal
+      * row builder for the new row.
+      *
+      * This assumes that there is corresponding entry, i.e. no values for the partition key and
+      * clustering are null (since we have eliminated that case through updateAction).
+      */
+     private void startNewUpdate(Row baseRow)
+     {
+         ByteBuffer[] clusteringValues = new ByteBuffer[viewMetadata.clusteringColumns().size()];
+         for (ColumnDefinition viewColumn : viewMetadata.primaryKeyColumns())
+         {
+             ColumnDefinition baseColumn = view.getBaseColumn(viewColumn);
+             ByteBuffer value = getValueForPK(baseColumn, baseRow);
+             if (viewColumn.isPartitionKey())
+                 currentViewEntryPartitionKey[viewColumn.position()] = value;
+             else
+                 clusteringValues[viewColumn.position()] = value;
+         }
+ 
 -        currentViewEntryBuilder.newRow(new Clustering(clusteringValues));
++        currentViewEntryBuilder.newRow(Clustering.make(clusteringValues));
+     }
+ 
+     private LivenessInfo computeLivenessInfoForEntry(Row baseRow)
+     {
+         /*
+          * We need to compute both the timestamp and expiration.
+          *
+          * 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())
+         {
+             int ttl = baseLiveness.ttl();
+             int expirationTime = baseLiveness.localExpirationTime();
+             for (Cell cell : baseRow.cells())
+             {
+                 if (cell.ttl() > ttl)
+                 {
+                     ttl = cell.ttl();
+                     expirationTime = cell.localDeletionTime();
+                 }
+             }
+             return ttl == baseLiveness.ttl()
+                  ? baseLiveness
 -                 : LivenessInfo.create(baseLiveness.timestamp(), ttl, expirationTime);
++                 : LivenessInfo.withExpirationTime(baseLiveness.timestamp(), ttl, expirationTime);
+         }
+ 
+         ColumnDefinition baseColumn = view.baseNonPKColumnsInViewPK.get(0);
+         Cell cell = baseRow.getCell(baseColumn);
+         assert isLive(cell) : "We shouldn't have got there is 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.withExpirationTime(timestamp, cell.ttl(), cell.localDeletionTime());
+     }
+ 
+     private long computeTimestampForEntryDeletion(Row baseRow)
+     {
+         // 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)
+         {
+             if (!view.getDefinition().includes(data.column().name))
+                 continue;
+ 
+             timestamp = Math.max(timestamp, data.maxTimestamp());
+         }
+         return timestamp;
+     }
+ 
+     private void addColumnData(ColumnDefinition viewColumn, ColumnData baseTableData)
+     {
+         assert viewColumn.isComplex() == baseTableData.column().isComplex();
+         if (!viewColumn.isComplex())
+         {
+             addCell(viewColumn, (Cell)baseTableData);
+             return;
+         }
+ 
+         ComplexColumnData complexData = (ComplexColumnData)baseTableData;
+         currentViewEntryBuilder.addComplexDeletion(viewColumn, complexData.complexDeletion());
+         for (Cell cell : complexData)
+             addCell(viewColumn, cell);
+     }
+ 
+     private void addCell(ColumnDefinition viewColumn, Cell baseTableCell)
+     {
+         assert !viewColumn.isPrimaryKeyColumn();
+         currentViewEntryBuilder.addCell(baseTableCell.withUpdatedColumn(viewColumn));
+     }
+ 
+     /**
+      * Finish building the currently updated view entry and add it to the other built
+      * updates.
+      */
+     private void submitUpdate()
+     {
+         Row row = currentViewEntryBuilder.build();
+         // I'm not sure we can reach there is there is nothing is updated, but adding an empty row breaks things
+         // and it costs us nothing to be prudent here.
+         if (row.isEmpty())
+             return;
+ 
+         DecoratedKey partitionKey = makeCurrentPartitionKey();
+         PartitionUpdate update = updates.get(partitionKey);
+         if (update == null)
+         {
+             // We can't really know which columns of the view will be updated nor how many row will be updated for this key
+             // so we rely on hopefully sane defaults.
+             update = new PartitionUpdate(viewMetadata, partitionKey, viewMetadata.partitionColumns(), 4);
+             updates.put(partitionKey, update);
+         }
+         update.add(row);
+     }
+ 
+     private DecoratedKey makeCurrentPartitionKey()
+     {
+         ByteBuffer rawKey = viewMetadata.partitionKeyColumns().size() == 1
+                           ? currentViewEntryPartitionKey[0]
+                           : CompositeType.build(currentViewEntryPartitionKey);
+ 
+         return viewMetadata.decorateKey(rawKey);
+     }
+ 
+     private ByteBuffer getValueForPK(ColumnDefinition column, Row row)
+     {
+         switch (column.kind)
+         {
+             case PARTITION_KEY:
+                 return basePartitionKey[column.position()];
+             case CLUSTERING:
+                 return row.clustering().get(column.position());
+             default:
+                 // This shouldn't NPE as we shouldn't get there if the value can be null (or there is a bug in updateAction())
+                 return row.getCell(column).value();
+         }
+     }
+ }

http://git-wip-us.apache.org/repos/asf/cassandra/blob/a87fd715/test/unit/org/apache/cassandra/db/rows/RowsTest.java
----------------------------------------------------------------------
diff --cc test/unit/org/apache/cassandra/db/rows/RowsTest.java
index 00ab6ca,b47bea2..ba03478
--- a/test/unit/org/apache/cassandra/db/rows/RowsTest.java
+++ b/test/unit/org/apache/cassandra/db/rows/RowsTest.java
@@@ -235,11 -235,12 +235,12 @@@ public class RowsTes
          originalBuilder.addPrimaryKeyLivenessInfo(liveness);
          DeletionTime complexDeletion = new DeletionTime(ts-1, now);
          originalBuilder.addComplexDeletion(m, complexDeletion);
 -        List<Cell> expectedCells = Lists.newArrayList(BufferCell.live(kcvm, v, secondToTs(now), BB1),
 -                                                      BufferCell.live(kcvm, m, secondToTs(now), BB1, CellPath.create(BB1)),
 -                                                      BufferCell.live(kcvm, m, secondToTs(now), BB2, CellPath.create(BB2)));
 +        List<Cell> expectedCells = Lists.newArrayList(BufferCell.live(v, secondToTs(now), BB1),
 +                                                      BufferCell.live(m, secondToTs(now), BB1, CellPath.create(BB1)),
 +                                                      BufferCell.live(m, secondToTs(now), BB2, CellPath.create(BB2)));
          expectedCells.forEach(originalBuilder::addCell);
-         Row.Deletion rowDeletion = new Row.Deletion(new DeletionTime(ts, now), false);
+         // We need to use ts-1 so the deletion doesn't shadow what we've created
+         Row.Deletion rowDeletion = new Row.Deletion(new DeletionTime(ts-1, now), false);
          originalBuilder.addRowDeletion(rowDeletion);
  
          RowBuilder builder = new RowBuilder();
@@@ -263,11 -264,12 +264,12 @@@
          builder.addPrimaryKeyLivenessInfo(liveness);
          DeletionTime complexDeletion = new DeletionTime(ts-1, now);
          builder.addComplexDeletion(m, complexDeletion);
 -        List<Cell> expectedCells = Lists.newArrayList(BufferCell.live(kcvm, v, ts, BB1),
 -                                                      BufferCell.live(kcvm, m, ts, BB1, CellPath.create(BB1)),
 -                                                      BufferCell.live(kcvm, m, ts, BB2, CellPath.create(BB2)));
 +        List<Cell> expectedCells = Lists.newArrayList(BufferCell.live(v, ts, BB1),
 +                                                      BufferCell.live(m, ts, BB1, CellPath.create(BB1)),
 +                                                      BufferCell.live(m, ts, BB2, CellPath.create(BB2)));
          expectedCells.forEach(builder::addCell);
-         Row.Deletion rowDeletion = new Row.Deletion(new DeletionTime(ts, now), false);
+         // We need to use ts-1 so the deletion doesn't shadow what we've created
+         Row.Deletion rowDeletion = new Row.Deletion(new DeletionTime(ts-1, now), false);
          builder.addRowDeletion(rowDeletion);
  
          StatsCollector collector = new StatsCollector();


[6/9] cassandra git commit: Refactor MV code

Posted by sl...@apache.org.
Refactor MV code

patch by slebresne; reviewed by carlyeks for CASSANDRA-11475


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

Branch: refs/heads/trunk
Commit: 86ba227477b9f8595eb610ecaf950cfbc29dd36b
Parents: c19066e
Author: Sylvain Lebresne <sy...@datastax.com>
Authored: Fri Mar 11 14:19:38 2016 +0100
Committer: Sylvain Lebresne <sy...@datastax.com>
Committed: Fri May 6 13:41:41 2016 +0200

----------------------------------------------------------------------
 CHANGES.txt                                     |   3 +
 .../org/apache/cassandra/config/CFMetaData.java |   6 +
 .../apache/cassandra/config/ViewDefinition.java |   1 -
 .../cql3/statements/CreateViewStatement.java    |   4 +-
 .../cql3/statements/SelectStatement.java        |  41 +-
 .../apache/cassandra/db/ColumnFamilyStore.java  |   6 +-
 src/java/org/apache/cassandra/db/Keyspace.java  |   2 +-
 .../db/SinglePartitionReadCommand.java          |  33 +
 src/java/org/apache/cassandra/db/Slices.java    |   7 +
 .../apache/cassandra/db/filter/RowFilter.java   |  24 +
 .../SingletonUnfilteredPartitionIterator.java   |   3 +-
 .../apache/cassandra/db/rows/AbstractCell.java  |   5 +
 .../org/apache/cassandra/db/rows/BTreeRow.java  |  34 +-
 .../apache/cassandra/db/rows/BufferCell.java    |   5 +
 src/java/org/apache/cassandra/db/rows/Cell.java |   2 +
 .../apache/cassandra/db/rows/ColumnData.java    |   2 +
 .../cassandra/db/rows/ComplexColumnData.java    |   8 +
 src/java/org/apache/cassandra/db/rows/Row.java  |  35 +-
 .../cassandra/db/rows/RowDiffListener.java      |   2 +-
 .../db/rows/UnfilteredRowIterators.java         |   2 +-
 .../apache/cassandra/db/view/TableViews.java    | 481 ++++++++++++++
 .../apache/cassandra/db/view/TemporalRow.java   | 610 ------------------
 src/java/org/apache/cassandra/db/view/View.java | 629 ++-----------------
 .../apache/cassandra/db/view/ViewBuilder.java   |  38 +-
 .../apache/cassandra/db/view/ViewManager.java   | 146 +----
 .../cassandra/db/view/ViewUpdateGenerator.java  | 549 ++++++++++++++++
 .../org/apache/cassandra/cql3/ViewTest.java     |  52 +-
 .../org/apache/cassandra/db/rows/RowsTest.java  |   6 +-
 28 files changed, 1399 insertions(+), 1337 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/cassandra/blob/86ba2274/CHANGES.txt
----------------------------------------------------------------------
diff --git a/CHANGES.txt b/CHANGES.txt
index 0679e11..3a49f6a 100644
--- a/CHANGES.txt
+++ b/CHANGES.txt
@@ -1,3 +1,6 @@
+3.0.7
+ * Refactor Materialized View code (CASSANDRA-11475)
+
 3.0.6
  * Disallow creating view with a static column (CASSANDRA-11602)
  * Reduce the amount of object allocations caused by the getFunctions methods (CASSANDRA-11593)

http://git-wip-us.apache.org/repos/asf/cassandra/blob/86ba2274/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 79cd779..e263697 100644
--- a/src/java/org/apache/cassandra/config/CFMetaData.java
+++ b/src/java/org/apache/cassandra/config/CFMetaData.java
@@ -31,6 +31,7 @@ import com.google.common.annotations.VisibleForTesting;
 import com.google.common.base.MoreObjects;
 import com.google.common.base.Objects;
 import com.google.common.collect.ImmutableSet;
+import com.google.common.collect.Iterables;
 import com.google.common.collect.Sets;
 import org.apache.commons.lang3.ArrayUtils;
 import org.apache.commons.lang3.builder.HashCodeBuilder;
@@ -612,6 +613,11 @@ public final class CFMetaData
         };
     }
 
+    public Iterable<ColumnDefinition> primaryKeyColumns()
+    {
+        return Iterables.concat(partitionKeyColumns, clusteringColumns);
+    }
+
     public List<ColumnDefinition> partitionKeyColumns()
     {
         return partitionKeyColumns;

http://git-wip-us.apache.org/repos/asf/cassandra/blob/86ba2274/src/java/org/apache/cassandra/config/ViewDefinition.java
----------------------------------------------------------------------
diff --git a/src/java/org/apache/cassandra/config/ViewDefinition.java b/src/java/org/apache/cassandra/config/ViewDefinition.java
index b29a8f9..5300f56 100644
--- a/src/java/org/apache/cassandra/config/ViewDefinition.java
+++ b/src/java/org/apache/cassandra/config/ViewDefinition.java
@@ -37,7 +37,6 @@ public class ViewDefinition
     public final UUID baseTableId;
     public final String baseTableName;
     public final boolean includeAllColumns;
-    // The order of partititon columns and clustering columns is important, so we cannot switch these two to sets
     public final CFMetaData metadata;
 
     public SelectStatement.RawStatement select;

http://git-wip-us.apache.org/repos/asf/cassandra/blob/86ba2274/src/java/org/apache/cassandra/cql3/statements/CreateViewStatement.java
----------------------------------------------------------------------
diff --git a/src/java/org/apache/cassandra/cql3/statements/CreateViewStatement.java b/src/java/org/apache/cassandra/cql3/statements/CreateViewStatement.java
index 45231b7..6446602 100644
--- a/src/java/org/apache/cassandra/cql3/statements/CreateViewStatement.java
+++ b/src/java/org/apache/cassandra/cql3/statements/CreateViewStatement.java
@@ -227,10 +227,10 @@ public class CreateViewStatement extends SchemaAlteringStatement
         // This is only used as an intermediate state; this is to catch whether multiple non-PK columns are used
         boolean hasNonPKColumn = false;
         for (ColumnIdentifier.Raw raw : partitionKeys)
-            hasNonPKColumn = getColumnIdentifier(cfm, basePrimaryKeyCols, hasNonPKColumn, raw, targetPartitionKeys, restrictions);
+            hasNonPKColumn |= getColumnIdentifier(cfm, basePrimaryKeyCols, hasNonPKColumn, raw, targetPartitionKeys, restrictions);
 
         for (ColumnIdentifier.Raw raw : clusteringKeys)
-            hasNonPKColumn = getColumnIdentifier(cfm, basePrimaryKeyCols, hasNonPKColumn, raw, targetClusteringColumns, restrictions);
+            hasNonPKColumn |= getColumnIdentifier(cfm, basePrimaryKeyCols, hasNonPKColumn, raw, targetClusteringColumns, restrictions);
 
         // We need to include all of the primary key columns from the base table in order to make sure that we do not
         // overwrite values in the view. We cannot support "collapsing" the base table into a smaller number of rows in

http://git-wip-us.apache.org/repos/asf/cassandra/blob/86ba2274/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 c3e13f4..0e33475 100644
--- a/src/java/org/apache/cassandra/cql3/statements/SelectStatement.java
+++ b/src/java/org/apache/cassandra/cql3/statements/SelectStatement.java
@@ -151,6 +151,15 @@ public class SelectStatement implements CQLStatement
         return builder.build();
     }
 
+    /**
+     * The columns to fetch internally for this SELECT statement (which can be more than the one selected by the
+     * user as it also include any restricted column in particular).
+     */
+    public ColumnFilter queriedColumns()
+    {
+        return queriedColumns;
+    }
+
     // Creates a simple select based on the given selection.
     // Note that the results select statement should not be used for actual queries, but only for processing already
     // queried data through processColumnFamily.
@@ -473,7 +482,29 @@ public class SelectStatement implements CQLStatement
     }
 
     /**
-     * Returns a read command that can be used internally to filter individual rows for materialized views.
+     * Returns the slices fetched by this SELECT, assuming an internal call (no bound values in particular).
+     * <p>
+     * Note that if the SELECT intrinsically selects rows by names, we convert them into equivalent slices for
+     * the purpose of this method. This is used for MVs to restrict what needs to be read when we want to read
+     * everything that could be affected by a given view (and so, if the view SELECT statement has restrictions
+     * on the clustering columns, we can restrict what we read).
+     */
+    public Slices clusteringIndexFilterAsSlices()
+    {
+        QueryOptions options = QueryOptions.forInternalCalls(Collections.emptyList());
+        ClusteringIndexFilter filter = makeClusteringIndexFilter(options);
+        if (filter instanceof ClusteringIndexSliceFilter)
+            return ((ClusteringIndexSliceFilter)filter).requestedSlices();
+
+        Slices.Builder builder = new Slices.Builder(cfm.comparator);
+        for (Clustering clustering: ((ClusteringIndexNamesFilter)filter).requestedRows())
+            builder.add(Slice.make(clustering));
+        return builder.build();
+    }
+
+    /**
+     * Returns a read command that can be used internally to query all the rows queried by this SELECT for a
+     * give key (used for materialized views).
      */
     public SinglePartitionReadCommand internalReadForView(DecoratedKey key, int nowInSec)
     {
@@ -483,6 +514,14 @@ public class SelectStatement implements CQLStatement
         return SinglePartitionReadCommand.create(cfm, nowInSec, queriedColumns, rowFilter, DataLimits.NONE, key, filter);
     }
 
+    /**
+     * The {@code RowFilter} for this SELECT, assuming an internal call (no bound values in particular).
+     */
+    public RowFilter rowFilterForInternalCalls()
+    {
+        return getRowFilter(QueryOptions.forInternalCalls(Collections.emptyList()));
+    }
+
     private ReadQuery getRangeCommand(QueryOptions options, DataLimits limit, int nowInSec) throws RequestValidationException
     {
         ClusteringIndexFilter clusteringIndexFilter = makeClusteringIndexFilter(options);

http://git-wip-us.apache.org/repos/asf/cassandra/blob/86ba2274/src/java/org/apache/cassandra/db/ColumnFamilyStore.java
----------------------------------------------------------------------
diff --git a/src/java/org/apache/cassandra/db/ColumnFamilyStore.java b/src/java/org/apache/cassandra/db/ColumnFamilyStore.java
index 919fed6..5b8de8f 100644
--- a/src/java/org/apache/cassandra/db/ColumnFamilyStore.java
+++ b/src/java/org/apache/cassandra/db/ColumnFamilyStore.java
@@ -48,7 +48,7 @@ import org.apache.cassandra.db.commitlog.ReplayPosition;
 import org.apache.cassandra.db.compaction.*;
 import org.apache.cassandra.db.filter.ClusteringIndexFilter;
 import org.apache.cassandra.db.filter.DataLimits;
-import org.apache.cassandra.db.view.ViewManager;
+import org.apache.cassandra.db.view.TableViews;
 import org.apache.cassandra.db.lifecycle.*;
 import org.apache.cassandra.db.partitions.CachedPartition;
 import org.apache.cassandra.db.partitions.PartitionUpdate;
@@ -198,7 +198,7 @@ public class ColumnFamilyStore implements ColumnFamilyStoreMBean
     private final AtomicInteger fileIndexGenerator = new AtomicInteger(0);
 
     public final SecondaryIndexManager indexManager;
-    public final ViewManager.ForStore viewManager;
+    public final TableViews viewManager;
 
     /* These are locally held copies to be changed from the config during runtime */
     private volatile DefaultValue<Integer> minCompactionThreshold;
@@ -373,7 +373,7 @@ public class ColumnFamilyStore implements ColumnFamilyStoreMBean
         maxCompactionThreshold = new DefaultValue<>(metadata.params.compaction.maxCompactionThreshold());
         crcCheckChance = new DefaultValue<>(metadata.params.crcCheckChance);
         indexManager = new SecondaryIndexManager(this);
-        viewManager = keyspace.viewManager.forTable(metadata.cfId);
+        viewManager = keyspace.viewManager.forTable(metadata);
         metric = new TableMetrics(this);
         fileIndexGenerator.set(generation);
         sampleLatencyNanos = DatabaseDescriptor.getReadRpcTimeout() / 2;

http://git-wip-us.apache.org/repos/asf/cassandra/blob/86ba2274/src/java/org/apache/cassandra/db/Keyspace.java
----------------------------------------------------------------------
diff --git a/src/java/org/apache/cassandra/db/Keyspace.java b/src/java/org/apache/cassandra/db/Keyspace.java
index 5783b41..273946e 100644
--- a/src/java/org/apache/cassandra/db/Keyspace.java
+++ b/src/java/org/apache/cassandra/db/Keyspace.java
@@ -484,7 +484,7 @@ public class Keyspace
                     try
                     {
                         Tracing.trace("Creating materialized view mutations from base table replica");
-                        viewManager.pushViewReplicaUpdates(upd, !isClReplay, baseComplete);
+                        viewManager.forTable(upd.metadata()).pushViewReplicaUpdates(upd, !isClReplay, baseComplete);
                     }
                     catch (Throwable t)
                     {

http://git-wip-us.apache.org/repos/asf/cassandra/blob/86ba2274/src/java/org/apache/cassandra/db/SinglePartitionReadCommand.java
----------------------------------------------------------------------
diff --git a/src/java/org/apache/cassandra/db/SinglePartitionReadCommand.java b/src/java/org/apache/cassandra/db/SinglePartitionReadCommand.java
index 14923b9..d5f2dc4 100644
--- a/src/java/org/apache/cassandra/db/SinglePartitionReadCommand.java
+++ b/src/java/org/apache/cassandra/db/SinglePartitionReadCommand.java
@@ -229,6 +229,39 @@ public class SinglePartitionReadCommand extends ReadCommand
         return create(metadata, nowInSec, metadata.decorateKey(key), slices);
     }
 
+    /**
+     * Creates a new single partition name command for the provided rows.
+     *
+     * @param metadata the table to query.
+     * @param nowInSec the time in seconds to use are "now" for this query.
+     * @param key the partition key for the partition to query.
+     * @param names the clustering for the rows to query.
+     *
+     * @return a newly created read command that queries the {@code names} in {@code key}. The returned query will
+     * query every columns (without limit or row filtering) and be in forward order.
+     */
+    public static SinglePartitionReadCommand create(CFMetaData metadata, int nowInSec, DecoratedKey key, NavigableSet<Clustering> names)
+    {
+        ClusteringIndexNamesFilter filter = new ClusteringIndexNamesFilter(names, false);
+        return SinglePartitionReadCommand.create(metadata, nowInSec, ColumnFilter.all(metadata), RowFilter.NONE, DataLimits.NONE, key, filter);
+    }
+
+    /**
+     * Creates a new single partition name command for the provided row.
+     *
+     * @param metadata the table to query.
+     * @param nowInSec the time in seconds to use are "now" for this query.
+     * @param key the partition key for the partition to query.
+     * @param name the clustering for the row to query.
+     *
+     * @return a newly created read command that queries {@code name} in {@code key}. The returned query will
+     * query every columns (without limit or row filtering).
+     */
+    public static SinglePartitionReadCommand create(CFMetaData metadata, int nowInSec, DecoratedKey key, Clustering name)
+    {
+        return create(metadata, nowInSec, key, FBUtilities.singleton(name, metadata.comparator));
+    }
+
     public SinglePartitionReadCommand copy()
     {
         return new SinglePartitionReadCommand(isDigestQuery(), digestVersion(), isForThrift(), metadata(), nowInSec(), columnFilter(), rowFilter(), limits(), partitionKey(), clusteringIndexFilter());

http://git-wip-us.apache.org/repos/asf/cassandra/blob/86ba2274/src/java/org/apache/cassandra/db/Slices.java
----------------------------------------------------------------------
diff --git a/src/java/org/apache/cassandra/db/Slices.java b/src/java/org/apache/cassandra/db/Slices.java
index 8fa9337..bb354a1 100644
--- a/src/java/org/apache/cassandra/db/Slices.java
+++ b/src/java/org/apache/cassandra/db/Slices.java
@@ -210,6 +210,13 @@ public abstract class Slices implements Iterable<Slice>
             return this;
         }
 
+        public Builder addAll(Slices slices)
+        {
+            for (Slice slice : slices)
+                add(slice);
+            return this;
+        }
+
         public int size()
         {
             return slices.size();

http://git-wip-us.apache.org/repos/asf/cassandra/blob/86ba2274/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 8060f23..11cfb87 100644
--- a/src/java/org/apache/cassandra/db/filter/RowFilter.java
+++ b/src/java/org/apache/cassandra/db/filter/RowFilter.java
@@ -122,6 +122,30 @@ public abstract class RowFilter implements Iterable<RowFilter.Expression>
     public abstract UnfilteredPartitionIterator filter(UnfilteredPartitionIterator iter, int nowInSec);
 
     /**
+     * Whether the provided row in the provided partition satisfies this filter.
+     *
+     * @param metadata the table metadata.
+     * @param partitionKey the partition key for partition to test.
+     * @param row the row to test.
+     * @param nowInSec the current time in seconds (to know what is live and what isn't).
+     * @return {@code true} if {@code row} in partition {@code partitionKey} satisfies this row filter.
+     */
+    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);
+        if (purged == null)
+            return expressions.isEmpty();
+
+        for (Expression e : expressions)
+        {
+            if (!e.isSatisfiedBy(metadata, partitionKey, purged))
+                return false;
+        }
+        return true;
+    }
+
+    /**
      * Returns true if all of the expressions within this filter that apply to the partition key are satisfied by
      * the given key, false otherwise.
      */

http://git-wip-us.apache.org/repos/asf/cassandra/blob/86ba2274/src/java/org/apache/cassandra/db/partitions/SingletonUnfilteredPartitionIterator.java
----------------------------------------------------------------------
diff --git a/src/java/org/apache/cassandra/db/partitions/SingletonUnfilteredPartitionIterator.java b/src/java/org/apache/cassandra/db/partitions/SingletonUnfilteredPartitionIterator.java
index bfa6690..1f966db 100644
--- a/src/java/org/apache/cassandra/db/partitions/SingletonUnfilteredPartitionIterator.java
+++ b/src/java/org/apache/cassandra/db/partitions/SingletonUnfilteredPartitionIterator.java
@@ -65,6 +65,7 @@ public class SingletonUnfilteredPartitionIterator implements UnfilteredPartition
 
     public void close()
     {
-        iter.close();
+        if (!returned)
+            iter.close();
     }
 }

http://git-wip-us.apache.org/repos/asf/cassandra/blob/86ba2274/src/java/org/apache/cassandra/db/rows/AbstractCell.java
----------------------------------------------------------------------
diff --git a/src/java/org/apache/cassandra/db/rows/AbstractCell.java b/src/java/org/apache/cassandra/db/rows/AbstractCell.java
index 00fc286..7e93c2e 100644
--- a/src/java/org/apache/cassandra/db/rows/AbstractCell.java
+++ b/src/java/org/apache/cassandra/db/rows/AbstractCell.java
@@ -74,6 +74,11 @@ public abstract class AbstractCell extends Cell
             column().validateCellPath(path());
     }
 
+    public long maxTimestamp()
+    {
+        return timestamp();
+    }
+
     @Override
     public boolean equals(Object other)
     {

http://git-wip-us.apache.org/repos/asf/cassandra/blob/86ba2274/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 e8667e0..ea1d9e0 100644
--- a/src/java/org/apache/cassandra/db/rows/BTreeRow.java
+++ b/src/java/org/apache/cassandra/db/rows/BTreeRow.java
@@ -322,6 +322,18 @@ public class BTreeRow extends AbstractRow
         return transformAndFilter(newInfo, newDeletion, (cd) -> cd.updateAllTimestamp(newTimestamp));
     }
 
+    public Row withRowDeletion(DeletionTime newDeletion)
+    {
+        // Note that:
+        //  - it is a contract with the caller that the new deletion shouldn't shadow anything in
+        //    the row, and so in particular it can't shadow the row deletion. So if there is a
+        //    already a row deletion we have nothing to do.
+        //  - we set the minLocalDeletionTime to MIN_VALUE because we know the deletion is live
+        return newDeletion.isLive() || !deletion.isLive()
+             ? this
+             : new BTreeRow(clustering, primaryKeyLivenessInfo, Deletion.regular(newDeletion), btree, Integer.MIN_VALUE);
+    }
+
     public Row purge(DeletionPurger purger, int nowInSec)
     {
         if (!hasDeletion(nowInSec))
@@ -566,6 +578,17 @@ public class BTreeRow extends AbstractRow
                 }
 
                 List<Object> buildFrom = Arrays.asList(cells).subList(lb, ub);
+                if (deletion != DeletionTime.LIVE)
+                {
+                    // Make sure we don't include any shadowed cells
+                    List<Object> filtered = new ArrayList<>(buildFrom.size());
+                    for (Object c : buildFrom)
+                    {
+                        if (((Cell)c).timestamp() >= deletion.markedForDeleteAt())
+                            filtered.add(c);
+                    }
+                    buildFrom = filtered;
+                }
                 Object[] btree = BTree.build(buildFrom, UpdateFunction.noOp());
                 return new ComplexColumnData(column, btree, deletion);
             }
@@ -621,17 +644,26 @@ public class BTreeRow extends AbstractRow
 
         public void addPrimaryKeyLivenessInfo(LivenessInfo info)
         {
-            this.primaryKeyLivenessInfo = info;
+            // The check is only required for unsorted builders, but it's worth the extra safety to have it unconditional
+            if (!deletion.deletes(info))
+                this.primaryKeyLivenessInfo = info;
         }
 
         public void addRowDeletion(Deletion deletion)
         {
             this.deletion = deletion;
+            // The check is only required for unsorted builders, but it's worth the extra safety to have it unconditional
+            if (deletion.deletes(primaryKeyLivenessInfo))
+                this.primaryKeyLivenessInfo = LivenessInfo.EMPTY;
         }
 
         public void addCell(Cell cell)
         {
             assert cell.column().isStatic() == (clustering == Clustering.STATIC_CLUSTERING) : "Column is " + cell.column() + ", clustering = " + clustering;
+            // In practice, only unsorted builder have to deal with shadowed cells, but it doesn't cost us much to deal with it unconditionally in this case
+            if (deletion.deletes(cell))
+                return;
+
             cells.add(cell);
             hasComplex |= cell.column.isComplex();
         }

http://git-wip-us.apache.org/repos/asf/cassandra/blob/86ba2274/src/java/org/apache/cassandra/db/rows/BufferCell.java
----------------------------------------------------------------------
diff --git a/src/java/org/apache/cassandra/db/rows/BufferCell.java b/src/java/org/apache/cassandra/db/rows/BufferCell.java
index 8912f59..0a2c528 100644
--- a/src/java/org/apache/cassandra/db/rows/BufferCell.java
+++ b/src/java/org/apache/cassandra/db/rows/BufferCell.java
@@ -133,6 +133,11 @@ public class BufferCell extends AbstractCell
         return path;
     }
 
+    public Cell withUpdatedColumn(ColumnDefinition newColumn)
+    {
+        return new BufferCell(newColumn, timestamp, ttl, localDeletionTime, value, path);
+    }
+
     public Cell withUpdatedValue(ByteBuffer newValue)
     {
         return new BufferCell(column, timestamp, ttl, localDeletionTime, newValue, path);

http://git-wip-us.apache.org/repos/asf/cassandra/blob/86ba2274/src/java/org/apache/cassandra/db/rows/Cell.java
----------------------------------------------------------------------
diff --git a/src/java/org/apache/cassandra/db/rows/Cell.java b/src/java/org/apache/cassandra/db/rows/Cell.java
index 73d9e44..b10ce06 100644
--- a/src/java/org/apache/cassandra/db/rows/Cell.java
+++ b/src/java/org/apache/cassandra/db/rows/Cell.java
@@ -129,6 +129,8 @@ public abstract class Cell extends ColumnData
      */
     public abstract CellPath path();
 
+    public abstract Cell withUpdatedColumn(ColumnDefinition newColumn);
+
     public abstract Cell withUpdatedValue(ByteBuffer newValue);
 
     public abstract Cell copy(AbstractAllocator allocator);

http://git-wip-us.apache.org/repos/asf/cassandra/blob/86ba2274/src/java/org/apache/cassandra/db/rows/ColumnData.java
----------------------------------------------------------------------
diff --git a/src/java/org/apache/cassandra/db/rows/ColumnData.java b/src/java/org/apache/cassandra/db/rows/ColumnData.java
index 84763e5..933da6a 100644
--- a/src/java/org/apache/cassandra/db/rows/ColumnData.java
+++ b/src/java/org/apache/cassandra/db/rows/ColumnData.java
@@ -82,4 +82,6 @@ public abstract class ColumnData
     public abstract ColumnData markCounterLocalToBeCleared();
 
     public abstract ColumnData purge(DeletionPurger purger, int nowInSec);
+
+    public abstract long maxTimestamp();
 }

http://git-wip-us.apache.org/repos/asf/cassandra/blob/86ba2274/src/java/org/apache/cassandra/db/rows/ComplexColumnData.java
----------------------------------------------------------------------
diff --git a/src/java/org/apache/cassandra/db/rows/ComplexColumnData.java b/src/java/org/apache/cassandra/db/rows/ComplexColumnData.java
index fab529b..d67d079 100644
--- a/src/java/org/apache/cassandra/db/rows/ComplexColumnData.java
+++ b/src/java/org/apache/cassandra/db/rows/ComplexColumnData.java
@@ -184,6 +184,14 @@ public class ComplexColumnData extends ColumnData implements Iterable<Cell>
         return transformAndFilter(newDeletion, (cell) -> (Cell) cell.updateAllTimestamp(newTimestamp));
     }
 
+    public long maxTimestamp()
+    {
+        long timestamp = complexDeletion.markedForDeleteAt();
+        for (Cell cell : this)
+            timestamp = Math.max(timestamp, cell.timestamp());
+        return timestamp;
+    }
+
     // This is the partner in crime of ArrayBackedRow.setValue. The exact warning apply. The short
     // version is: "don't use that method".
     void setValue(CellPath path, ByteBuffer value)

http://git-wip-us.apache.org/repos/asf/cassandra/blob/86ba2274/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 5f79a66..82c07a7 100644
--- a/src/java/org/apache/cassandra/db/rows/Row.java
+++ b/src/java/org/apache/cassandra/db/rows/Row.java
@@ -215,6 +215,18 @@ public interface Row extends Unfiltered, Collection<ColumnData>
      */
     public Row updateAllTimestamp(long newTimestamp);
 
+    /**
+     * Returns a copy of this row with the new deletion as row deletion if it is more recent
+     * than the current row deletion.
+     * <p>
+     * WARNING: this method <b>does not</b> check that nothing in the row is shadowed by the provided
+     * deletion and if that is the case, the created row will be <b>invalid</b>. It is thus up to the
+     * caller to verify that this is not the case and the only reasonable use case of this is probably
+     * when the row and the deletion comes from the same {@code UnfilteredRowIterator} since that gives
+     * use this guarantee.
+     */
+    public Row withRowDeletion(DeletionTime deletion);
+
     public int dataSize();
 
     public long unsharedHeapSizeExcludingData();
@@ -227,12 +239,15 @@ public interface Row extends Unfiltered, Collection<ColumnData>
      * A row deletion mostly consists of the time of said deletion, but there is 2 variants: shadowable
      * and regular row deletion.
      * <p>
-     * A shadowable row deletion only exists if the row timestamp ({@code primaryKeyLivenessInfo().timestamp()})
-     * is lower than the deletion timestamp. That is, if a row has a shadowable deletion with timestamp A and an update is made
-     * to that row with a timestamp B such that B > A, then the shadowable deletion is 'shadowed' by that update. A concrete
-     * consequence is that if said update has cells with timestamp lower than A, then those cells are preserved
-     * (since the deletion is removed), and this contrarily to a normal (regular) deletion where the deletion is preserved
-     * and such cells are removed.
+     * A shadowable row deletion only exists if the row has no timestamp. In other words, the deletion is only
+     * valid as long as no newer insert is done (thus setting a row timestap; note that if the row timestamp set
+     * is lower than the deletion, it is shadowed (and thus ignored) as usual).
+     * <p>
+     * That is, if a row has a shadowable deletion with timestamp A and an update is madeto that row with a
+     * timestamp B such that B > A (and that update sets the row timestamp), then the shadowable deletion is 'shadowed'
+     * by that update. A concrete consequence is that if said update has cells with timestamp lower than A, then those
+     * cells are preserved(since the deletion is removed), and this contrarily to a normal (regular) deletion where the
+     * deletion is preserved and such cells are removed.
      * <p>
      * Currently, the only use of shadowable row deletions is Materialized Views, see CASSANDRA-10261.
      */
@@ -312,6 +327,11 @@ public interface Row extends Unfiltered, Collection<ColumnData>
             return time.deletes(info);
         }
 
+        public boolean deletes(Cell cell)
+        {
+            return time.deletes(cell);
+        }
+
         public void digest(MessageDigest digest)
         {
             time.digest(digest);
@@ -361,6 +381,9 @@ public interface Row extends Unfiltered, Collection<ColumnData>
      *      any column before {@code c} and before any call for any column after {@code c}.
      *   5) Calls to {@link #addCell} are further done in strictly increasing cell order (the one defined by
      *      {@link Cell#comparator}. That is, for a give column, cells are passed in {@code CellPath} order.
+     *   6) No shadowed data should be added. Concretely, this means that if a a row deletion is added, it doesn't
+     *      deletes the row timestamp or any cell added later, and similarly no cell added is deleted by the complex
+     *      deletion of the column this is a cell of.
      *
      * An unsorted builder will not expect those last rules however: {@link #addCell} and {@link #addComplexDeletion}
      * can be done in any order. And in particular unsorted builder allows multiple calls for the same column/cell. In

http://git-wip-us.apache.org/repos/asf/cassandra/blob/86ba2274/src/java/org/apache/cassandra/db/rows/RowDiffListener.java
----------------------------------------------------------------------
diff --git a/src/java/org/apache/cassandra/db/rows/RowDiffListener.java b/src/java/org/apache/cassandra/db/rows/RowDiffListener.java
index f209bfc..ec848a0 100644
--- a/src/java/org/apache/cassandra/db/rows/RowDiffListener.java
+++ b/src/java/org/apache/cassandra/db/rows/RowDiffListener.java
@@ -71,7 +71,7 @@ public interface RowDiffListener
      * @param i the input row from which {@code original} is from.
      * @param clustering the clustering for the row that is merged.
      * @param merged the cell of the merged row. Will be {@code null} if input {@code i} had a cell but that cell is no present
-     * in the mergd result (it has been deleted/shadowed).
+     * in the merged result (it has been deleted/shadowed).
      * @param original the cell of input {@code i}. May be {@code null} if input {@code i} had cell corresponding to {@code merged}.
      */
     public void onCell(int i, Clustering clustering, Cell merged, Cell original);

http://git-wip-us.apache.org/repos/asf/cassandra/blob/86ba2274/src/java/org/apache/cassandra/db/rows/UnfilteredRowIterators.java
----------------------------------------------------------------------
diff --git a/src/java/org/apache/cassandra/db/rows/UnfilteredRowIterators.java b/src/java/org/apache/cassandra/db/rows/UnfilteredRowIterators.java
index 9416896..ce5fffe 100644
--- a/src/java/org/apache/cassandra/db/rows/UnfilteredRowIterators.java
+++ b/src/java/org/apache/cassandra/db/rows/UnfilteredRowIterators.java
@@ -92,7 +92,7 @@ public abstract class UnfilteredRowIterators
     }
 
     /**
-     * Returns an empty atom iterator for a given partition.
+     * Returns an empty unfiltered iterator for a given partition.
      */
     public static UnfilteredRowIterator noRowsIterator(final CFMetaData cfm, final DecoratedKey partitionKey, final Row staticRow, final DeletionTime partitionDeletion, final boolean isReverseOrder)
     {

http://git-wip-us.apache.org/repos/asf/cassandra/blob/86ba2274/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
new file mode 100644
index 0000000..893bdd5
--- /dev/null
+++ b/src/java/org/apache/cassandra/db/view/TableViews.java
@@ -0,0 +1,481 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.cassandra.db.view;
+
+import java.util.*;
+import java.util.concurrent.CopyOnWriteArrayList;
+import java.util.concurrent.TimeUnit;
+import java.util.concurrent.atomic.AtomicLong;
+
+import com.google.common.collect.Iterables;
+import com.google.common.collect.Iterators;
+import com.google.common.collect.PeekingIterator;
+
+import org.apache.cassandra.config.CFMetaData;
+import org.apache.cassandra.db.*;
+import org.apache.cassandra.db.commitlog.ReplayPosition;
+import org.apache.cassandra.db.filter.*;
+import org.apache.cassandra.db.rows.*;
+import org.apache.cassandra.db.partitions.*;
+import org.apache.cassandra.dht.Token;
+import org.apache.cassandra.service.StorageProxy;
+import org.apache.cassandra.utils.FBUtilities;
+import org.apache.cassandra.utils.btree.BTreeSet;
+
+
+/**
+ * Groups all the views for a given table.
+ */
+public class TableViews extends AbstractCollection<View>
+{
+    private final CFMetaData baseTableMetadata;
+
+    // We need this to be thread-safe, but the number of times this is changed (when a view is created in the keyspace)
+    // massively exceeds the number of time it's read (for every mutation on the keyspace), so a copy-on-write list is the best option.
+    private final List<View> views = new CopyOnWriteArrayList();
+
+    public TableViews(CFMetaData baseTableMetadata)
+    {
+        this.baseTableMetadata = baseTableMetadata;
+    }
+
+    public int size()
+    {
+        return views.size();
+    }
+
+    public Iterator<View> iterator()
+    {
+        return views.iterator();
+    }
+
+    public boolean contains(String viewName)
+    {
+        return Iterables.any(views, view -> view.name.equals(viewName));
+    }
+
+    public boolean add(View view)
+    {
+        // We should have validated that there is no existing view with this name at this point
+        assert !contains(view.name);
+        return views.add(view);
+    }
+
+    public Iterable<ColumnFamilyStore> allViewsCfs()
+    {
+        Keyspace keyspace = Keyspace.open(baseTableMetadata.ksName);
+        return Iterables.transform(views, view -> keyspace.getColumnFamilyStore(view.getDefinition().viewName));
+    }
+
+    public void forceBlockingFlush()
+    {
+        for (ColumnFamilyStore viewCfs : allViewsCfs())
+            viewCfs.forceBlockingFlush();
+    }
+
+    public void dumpMemtables()
+    {
+        for (ColumnFamilyStore viewCfs : allViewsCfs())
+            viewCfs.dumpMemtable();
+    }
+
+    public void truncateBlocking(long truncatedAt)
+    {
+        for (ColumnFamilyStore viewCfs : allViewsCfs())
+        {
+            ReplayPosition replayAfter = viewCfs.discardSSTables(truncatedAt);
+            SystemKeyspace.saveTruncationRecord(viewCfs, truncatedAt, replayAfter);
+        }
+    }
+
+    public void removeByName(String viewName)
+    {
+        views.removeIf(v -> v.name.equals(viewName));
+    }
+
+    /**
+     * Calculates and pushes updates to the views replicas. The replicas are determined by
+     * {@link ViewUtils#getViewNaturalEndpoint(String, Token, Token)}.
+     *
+     * @param update an update on the base table represented by this object.
+     * @param writeCommitLog whether we should write the commit log for the view updates.
+     * @param baseComplete time from epoch in ms that the local base mutation was (or will be) completed
+     */
+    public void pushViewReplicaUpdates(PartitionUpdate update, boolean writeCommitLog, AtomicLong baseComplete)
+    {
+        assert update.metadata().cfId.equals(baseTableMetadata.cfId);
+
+        Collection<View> views = updatedViews(update);
+        if (views.isEmpty())
+            return;
+
+        // Read modified rows
+        int nowInSec = FBUtilities.nowInSeconds();
+        SinglePartitionReadCommand command = readExistingRowsCommand(update, views, nowInSec);
+        if (command == null)
+            return;
+
+        ColumnFamilyStore cfs = Keyspace.openAndGetStore(update.metadata());
+        long start = System.nanoTime();
+        Collection<Mutation> mutations;
+        try (ReadOrderGroup orderGroup = command.startOrderGroup();
+             UnfilteredRowIterator existings = UnfilteredPartitionIterators.getOnlyElement(command.executeLocally(orderGroup), command);
+             UnfilteredRowIterator updates = update.unfilteredIterator())
+        {
+            mutations = generateViewUpdates(views, updates, existings, nowInSec);
+        }
+        Keyspace.openAndGetStore(update.metadata()).metric.viewReadTime.update(System.nanoTime() - start, TimeUnit.NANOSECONDS);
+
+        if (!mutations.isEmpty())
+            StorageProxy.mutateMV(update.partitionKey().getKey(), mutations, writeCommitLog, baseComplete);
+    }
+
+    /**
+     * Given some updates on the base table of this object and the existing values for the rows affected by that update, generates the
+     * mutation to be applied to the provided views.
+     *
+     * @param views the views potentially affected by {@code updates}.
+     * @param updates the base table updates being applied.
+     * @param existings the existing values for the rows affected by {@code updates}. This is used to decide if a view is
+     * obsoleted by the update and should be removed, gather the values for columns that may not be part of the update if
+     * a new view entry needs to be created, and compute the minimal updates to be applied if the view entry isn't changed
+     * 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.
+     * @return the mutations to apply to the {@code views}. This can be empty.
+     */
+    public Collection<Mutation> generateViewUpdates(Collection<View> views, UnfilteredRowIterator updates, UnfilteredRowIterator existings, int nowInSec)
+    {
+        assert updates.metadata().cfId.equals(baseTableMetadata.cfId);
+
+        List<ViewUpdateGenerator> generators = new ArrayList<>(views.size());
+        for (View view : views)
+            generators.add(new ViewUpdateGenerator(view, updates.partitionKey(), nowInSec));
+
+        DeletionTracker existingsDeletion = new DeletionTracker(existings.partitionLevelDeletion());
+        DeletionTracker updatesDeletion = new DeletionTracker(updates.partitionLevelDeletion());
+
+        /*
+         * We iterate through the updates and the existing rows in parallel. This allows us to know the consequence
+         * on the view of each update.
+         */
+        PeekingIterator<Unfiltered> existingsIter = Iterators.peekingIterator(existings);
+        PeekingIterator<Unfiltered> updatesIter = Iterators.peekingIterator(updates);
+
+        while (existingsIter.hasNext() && updatesIter.hasNext())
+        {
+            Unfiltered existing = existingsIter.peek();
+            Unfiltered update = updatesIter.peek();
+
+            Row existingRow;
+            Row updateRow;
+            int cmp = baseTableMetadata.comparator.compare(update, existing);
+            if (cmp < 0)
+            {
+                // We have an update where there was nothing before
+                if (update.isRangeTombstoneMarker())
+                {
+                    updatesDeletion.update(updatesIter.next());
+                    continue;
+                }
+
+                updateRow = ((Row)updatesIter.next()).withRowDeletion(updatesDeletion.currentDeletion());
+                existingRow = emptyRow(updateRow.clustering(), existingsDeletion.currentDeletion());
+            }
+            else if (cmp > 0)
+            {
+                // We have something existing but no update (which will happen either because it's a range tombstone marker in
+                // existing, or because we've fetched the existing row due to some partition/range deletion in the updates)
+                if (existing.isRangeTombstoneMarker())
+                {
+                    existingsDeletion.update(existingsIter.next());
+                    continue;
+                }
+
+                existingRow = ((Row)existingsIter.next()).withRowDeletion(existingsDeletion.currentDeletion());
+                updateRow = emptyRow(existingRow.clustering(), updatesDeletion.currentDeletion());
+
+                // The way we build the read command used for existing rows, we should always have updatesDeletion.currentDeletion()
+                // that is not live, since we wouldn't have read the existing row otherwise. And we could assert that, but if we ever
+                // change the read method so that it can slightly over-read in some case, that would be an easily avoiding bug lurking,
+                // so we just handle the case.
+                if (updateRow == null)
+                    continue;
+            }
+            else
+            {
+                // We're updating a row that had pre-existing data
+                if (update.isRangeTombstoneMarker())
+                {
+                    assert existing.isRangeTombstoneMarker();
+                    updatesDeletion.update(updatesIter.next());
+                    existingsDeletion.update(existingsIter.next());
+                    continue;
+                }
+
+                assert !existing.isRangeTombstoneMarker();
+                existingRow = ((Row)existingsIter.next()).withRowDeletion(existingsDeletion.currentDeletion());
+                updateRow = ((Row)updatesIter.next()).withRowDeletion(updatesDeletion.currentDeletion());
+            }
+
+            addToViewUpdateGenerators(existingRow, updateRow, generators, nowInSec);
+        }
+
+        // We only care about more existing rows if the update deletion isn't live, i.e. if we had a partition deletion
+        if (!updatesDeletion.currentDeletion().isLive())
+        {
+            while (existingsIter.hasNext())
+            {
+                Unfiltered existing = existingsIter.next();
+                // If it's a range tombstone, we don't care, we're only looking for existing entry that gets deleted by
+                // the new partition deletion
+                if (existing.isRangeTombstoneMarker())
+                    continue;
+
+                Row existingRow = (Row)existing;
+                addToViewUpdateGenerators(existingRow, emptyRow(existingRow.clustering(), updatesDeletion.currentDeletion()), generators, nowInSec);
+            }
+        }
+        while (updatesIter.hasNext())
+        {
+            Unfiltered update = updatesIter.next();
+            // If it's a range tombstone, it removes nothing pre-exisiting, so we can ignore it for view updates
+            if (update.isRangeTombstoneMarker())
+                continue;
+
+            Row updateRow = (Row)update;
+            addToViewUpdateGenerators(emptyRow(updateRow.clustering(), DeletionTime.LIVE), updateRow, generators, nowInSec);
+        }
+
+        return buildMutations(baseTableMetadata, generators);
+    }
+
+    /**
+     * Return the views that are potentially updated by the provided updates.
+     *
+     * @param updates the updates applied to the base table.
+     * @return the views affected by {@code updates}.
+     */
+    public Collection<View> updatedViews(PartitionUpdate updates)
+    {
+        List<View> matchingViews = new ArrayList<>(views.size());
+
+        for (View view : views)
+        {
+            ReadQuery selectQuery = view.getReadQuery();
+            if (!selectQuery.selectsKey(updates.partitionKey()))
+                continue;
+
+            matchingViews.add(view);
+        }
+        return matchingViews;
+    }
+
+    /**
+     * Returns the command to use to read the existing rows required to generate view updates for the provided base
+     * base updates.
+     *
+     * @param updates the base table updates being applied.
+     * @param views the views potentially affected by {@code updates}.
+     * @param nowInSec the current time in seconds.
+     * @return the command to use to read the base table rows required to generate view updates for {@code updates}.
+     */
+    private SinglePartitionReadCommand readExistingRowsCommand(PartitionUpdate updates, Collection<View> views, int nowInSec)
+    {
+        Slices.Builder sliceBuilder = null;
+        DeletionInfo deletionInfo = updates.deletionInfo();
+        CFMetaData metadata = updates.metadata();
+        DecoratedKey key = updates.partitionKey();
+        // TODO: This is subtle: we need to gather all the slices that we have to fetch between partition del, range tombstones and rows.
+        if (!deletionInfo.isLive())
+        {
+            sliceBuilder = new Slices.Builder(metadata.comparator);
+            // Everything covered by a deletion might invalidate an existing view entry, which means we must read it to know. In practice
+            // though, the views involved might filter some base table clustering columns, in which case we can restrict what we read
+            // using those restrictions.
+            // If there is a partition deletion, then we can simply take each slices from each view select filter. They may overlap but
+            // the Slices.Builder handles that for us. Note that in many case this will just involve reading everything (as soon as any
+            // view involved has no clustering restrictions for instance).
+            // For range tombstone, we should theoretically take the difference between the range tombstoned and the slices selected
+            // by every views, but as we don't an easy way to compute that right now, we keep it simple and just use the tombstoned
+            // range.
+            // TODO: we should improve that latter part.
+            if (!deletionInfo.getPartitionDeletion().isLive())
+            {
+                for (View view : views)
+                    sliceBuilder.addAll(view.getSelectStatement().clusteringIndexFilterAsSlices());
+            }
+            else
+            {
+                assert deletionInfo.hasRanges();
+                Iterator<RangeTombstone> iter = deletionInfo.rangeIterator(false);
+                while (iter.hasNext())
+                    sliceBuilder.add(iter.next().deletedSlice());
+            }
+        }
+
+        // We need to read every row that is updated, unless we can prove that it has no impact on any view entries.
+
+        // If we had some slices from the deletions above, we'll continue using that. Otherwise, it's more efficient to build
+        // a names query.
+        BTreeSet.Builder<Clustering> namesBuilder = sliceBuilder == null ? BTreeSet.builder(metadata.comparator) : null;
+        for (Row row : updates)
+        {
+            // Don't read the existing state if we can prove the update won't affect any views
+            if (!affectsAnyViews(key, row, views))
+                continue;
+
+            if (namesBuilder == null)
+                sliceBuilder.add(Slice.make(row.clustering()));
+            else
+                namesBuilder.add(row.clustering());
+        }
+
+        NavigableSet<Clustering> names = namesBuilder == null ? null : namesBuilder.build();
+        // If we have a slice builder, it means we had some deletions and we have to read. But if we had
+        // only row updates, it's possible none of them affected the views, in which case we have nothing
+        // to do.
+        if (names != null && names.isEmpty())
+            return null;
+
+        ClusteringIndexFilter clusteringFilter = names == null
+                                               ? new ClusteringIndexSliceFilter(sliceBuilder.build(), false)
+                                               : new ClusteringIndexNamesFilter(names, false);
+        // 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);
+        // 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.
+        // TODO: we could still make sense to special case for when there is a single view and a small number of updates (and
+        // no deletions). Indeed, in that case we could check whether any of the update modify any of the restricted regular
+        // column, and if that's not the case we could use view filter. We keep it simple for now though.
+        RowFilter rowFilter = RowFilter.NONE;
+        return SinglePartitionReadCommand.create(metadata, nowInSec, queriedColumns, rowFilter, DataLimits.NONE, key, clusteringFilter);
+    }
+
+    private boolean affectsAnyViews(DecoratedKey partitionKey, Row update, Collection<View> views)
+    {
+        for (View view : views)
+        {
+            if (view.mayBeAffectedBy(partitionKey, update))
+                return true;
+        }
+        return false;
+    }
+
+    /**
+     * Given an existing base row and the update that we're going to apply to this row, generate the modifications
+     * to apply to MVs using the provided {@code ViewUpdateGenerator}s.
+     *
+     * @param existingBaseRow the base table row as it is before an update.
+     * @param updateBaseRow the newly updates made to {@code existingBaseRow}.
+     * @param generators the view update generators to add the new changes to.
+     * @param nowInSec the current time in seconds. Used to decide if data is live or not.
+     */
+    private static void addToViewUpdateGenerators(Row existingBaseRow, Row updateBaseRow, Collection<ViewUpdateGenerator> generators, int nowInSec)
+    {
+        // Having existing empty is useful, it just means we'll insert a brand new entry for updateBaseRow,
+        // but if we have no update at all, we shouldn't get there.
+        assert !updateBaseRow.isEmpty();
+
+        // We allow existingBaseRow to be null, which we treat the same as being empty as an small optimization
+        // to avoid allocating empty row objects when we know there was nothing existing.
+        Row mergedBaseRow = existingBaseRow == null ? updateBaseRow : Rows.merge(existingBaseRow, updateBaseRow, nowInSec);
+        for (ViewUpdateGenerator generator : generators)
+            generator.addBaseTableUpdate(existingBaseRow, mergedBaseRow);
+    }
+
+    private static Row emptyRow(Clustering clustering, DeletionTime deletion)
+    {
+        // Returning null for an empty row is slightly ugly, but the case where there is no pre-existing row is fairly common
+        // (especially when building the view), so we want to avoid a dummy allocation of an empty row every time.
+        // And MultiViewUpdateBuilder knows how to deal with that.
+        return deletion.isLive() ? null : BTreeRow.emptyDeletedRow(clustering, Row.Deletion.regular(deletion));
+    }
+
+    /**
+     * Extracts (and potentially groups) the mutations generated by the provided view update generator.
+     * Returns the mutation that needs to be done to the views given the base table updates
+     * passed to {@link #addBaseTableUpdate}.
+     *
+     * @param baseTableMetadata the metadata for the base table being updated.
+     * @param generators the generators from which to extract the view mutations from.
+     * @return the mutations created by all the generators in {@code generators}.
+     */
+    private Collection<Mutation> buildMutations(CFMetaData baseTableMetadata, List<ViewUpdateGenerator> generators)
+    {
+        // One view is probably common enough and we can optimize a bit easily
+        if (generators.size() == 1)
+        {
+            Collection<PartitionUpdate> updates = generators.get(0).generateViewUpdates();
+            List<Mutation> mutations = new ArrayList<>(updates.size());
+            for (PartitionUpdate update : updates)
+                mutations.add(new Mutation(update));
+            return mutations;
+        }
+
+        Map<DecoratedKey, Mutation> mutations = new HashMap<>();
+        for (ViewUpdateGenerator generator : generators)
+        {
+            for (PartitionUpdate update : generator.generateViewUpdates())
+            {
+                DecoratedKey key = update.partitionKey();
+                Mutation mutation = mutations.get(key);
+                if (mutation == null)
+                {
+                    mutation = new Mutation(baseTableMetadata.ksName, key);
+                    mutations.put(key, mutation);
+                }
+                mutation.add(update);
+            }
+        }
+        return mutations.values();
+    }
+
+    /**
+     * A simple helper that tracks for a given {@code UnfilteredRowIterator} what is the current deletion at any time of the
+     * iteration. It will be the currently open range tombstone deletion if there is one and the partition deletion otherwise.
+     */
+    private static class DeletionTracker
+    {
+        private final DeletionTime partitionDeletion;
+        private DeletionTime deletion;
+
+        public DeletionTracker(DeletionTime partitionDeletion)
+        {
+            this.partitionDeletion = partitionDeletion;
+        }
+
+        public void update(Unfiltered marker)
+        {
+            assert marker instanceof RangeTombstoneMarker;
+            RangeTombstoneMarker rtm = (RangeTombstoneMarker)marker;
+            this.deletion = rtm.isOpen(false)
+                          ? rtm.openDeletionTime(false)
+                          : null;
+        }
+
+        public DeletionTime currentDeletion()
+        {
+            return deletion == null ? partitionDeletion : deletion;
+        }
+    }
+}

http://git-wip-us.apache.org/repos/asf/cassandra/blob/86ba2274/src/java/org/apache/cassandra/db/view/TemporalRow.java
----------------------------------------------------------------------
diff --git a/src/java/org/apache/cassandra/db/view/TemporalRow.java b/src/java/org/apache/cassandra/db/view/TemporalRow.java
deleted file mode 100644
index 23705b9..0000000
--- a/src/java/org/apache/cassandra/db/view/TemporalRow.java
+++ /dev/null
@@ -1,610 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements.  See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership.  The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License.  You may obtain a copy of the License at
- *
- *     http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-
-package org.apache.cassandra.db.view;
-
-import static java.util.Comparator.naturalOrder;
-import static java.util.Comparator.reverseOrder;
-
-import java.nio.ByteBuffer;
-import java.util.ArrayList;
-import java.util.Collection;
-import java.util.Collections;
-import java.util.Comparator;
-import java.util.HashMap;
-import java.util.Iterator;
-import java.util.List;
-import java.util.Map;
-
-import com.google.common.base.MoreObjects;
-import com.google.common.collect.Iterables;
-
-import org.apache.cassandra.config.CFMetaData;
-import org.apache.cassandra.config.ColumnDefinition;
-import org.apache.cassandra.cql3.ColumnIdentifier;
-import org.apache.cassandra.db.CBuilder;
-import org.apache.cassandra.db.Clustering;
-import org.apache.cassandra.db.ColumnFamilyStore;
-import org.apache.cassandra.db.Conflicts;
-import org.apache.cassandra.db.DecoratedKey;
-import org.apache.cassandra.db.DeletionInfo;
-import org.apache.cassandra.db.DeletionTime;
-import org.apache.cassandra.db.LivenessInfo;
-import org.apache.cassandra.db.RangeTombstone;
-import org.apache.cassandra.db.Slice;
-import org.apache.cassandra.db.marshal.CompositeType;
-import org.apache.cassandra.db.partitions.AbstractBTreePartition;
-import org.apache.cassandra.db.rows.BufferCell;
-import org.apache.cassandra.db.rows.Cell;
-import org.apache.cassandra.db.rows.CellPath;
-import org.apache.cassandra.db.rows.Row;
-import org.apache.cassandra.utils.ByteBufferUtil;
-import org.apache.cassandra.utils.FBUtilities;
-
-/**
- * Represents a single CQL Row in a base table, with both the currently persisted value and the update's value. The
- * values are stored in timestamp order, but also indicate whether they are from the currently persisted, allowing a
- * {@link TemporalRow.Resolver} to resolve if the value is an old value that has been updated; if it sorts after the
- * update's value, then it does not qualify.
- */
-public class TemporalRow
-{
-    private static final int NO_TTL = LivenessInfo.NO_TTL;
-    private static final long NO_TIMESTAMP = LivenessInfo.NO_TIMESTAMP;
-    private static final int NO_DELETION_TIME = DeletionTime.LIVE.localDeletionTime();
-
-    public interface Resolver
-    {
-        /**
-         * @param cellVersions  all cells for a certain TemporalRow's Cell
-         * @return      A single TemporalCell from the iterable which satisfies the resolution criteria, or null if
-         *              there is no cell which qualifies
-         */
-        TemporalCell resolve(TemporalCell.Versions cellVersions);
-    }
-
-    public static final Resolver oldValueIfUpdated = TemporalCell.Versions::getOldCellIfUpdated;
-    public static final Resolver earliest = TemporalCell.Versions::getEarliestCell;
-    public static final Resolver latest = TemporalCell.Versions::getLatestCell;
-
-    private static class TemporalCell
-    {
-        public final ByteBuffer value;
-        public final long timestamp;
-        public final int ttl;
-        public final int localDeletionTime;
-        public final boolean isNew;
-
-        private TemporalCell(ByteBuffer value, long timestamp, int ttl, int localDeletionTime, boolean isNew)
-        {
-            this.value = value;
-            this.timestamp = timestamp;
-            this.ttl = ttl;
-            this.localDeletionTime = localDeletionTime;
-            this.isNew = isNew;
-        }
-
-        @Override
-        public String toString()
-        {
-            return MoreObjects.toStringHelper(this)
-                    .add("value", value == null ? "null" : ByteBufferUtil.bytesToHex(value))
-                    .add("timestamp", timestamp)
-                    .add("ttl", ttl)
-                    .add("localDeletionTime", localDeletionTime)
-                    .add("isNew", isNew)
-                    .toString();
-        }
-
-        public TemporalCell reconcile(TemporalCell that)
-        {
-            int now = FBUtilities.nowInSeconds();
-            Conflicts.Resolution resolution = Conflicts.resolveRegular(that.timestamp,
-                                                                       that.isLive(now),
-                                                                       that.localDeletionTime,
-                                                                       that.value,
-                                                                       this.timestamp,
-                                                                       this.isLive(now),
-                                                                       this.localDeletionTime,
-                                                                       this.value);
-            assert resolution != Conflicts.Resolution.MERGE;
-            if (resolution == Conflicts.Resolution.LEFT_WINS)
-                return that;
-            return this;
-        }
-
-        private boolean isLive(int now)
-        {
-            return localDeletionTime == NO_DELETION_TIME || (ttl != NO_TTL && now < localDeletionTime);
-        }
-
-        public Cell cell(ColumnDefinition definition, CellPath cellPath)
-        {
-            return new BufferCell(definition, timestamp, ttl, localDeletionTime, value, cellPath);
-        }
-
-        public boolean equals(Object o)
-        {
-            if (this == o) return true;
-            if (o == null || getClass() != o.getClass()) return false;
-
-            TemporalCell that = (TemporalCell) o;
-
-            if (timestamp != that.timestamp) return false;
-            if (ttl != that.ttl) return false;
-            if (localDeletionTime != that.localDeletionTime) return false;
-            if (isNew != that.isNew) return false;
-            return !(value != null ? !value.equals(that.value) : that.value != null);
-        }
-
-        public int hashCode()
-        {
-            int result = value != null ? value.hashCode() : 0;
-            result = 31 * result + (int) (timestamp ^ (timestamp >>> 32));
-            result = 31 * result + ttl;
-            result = 31 * result + localDeletionTime;
-            result = 31 * result + (isNew ? 1 : 0);
-            return result;
-        }
-
-        /**
-         * Tracks the versions of a cell for a given TemporalRow.
-         * There are only two possible versions, existing and new.
-         *
-         */
-        static class Versions
-        {
-            private TemporalCell existingCell = null;
-            private TemporalCell newCell = null;
-            private int numSet = 0;
-
-
-            /**
-             * @return the cell that is earliest
-             * Or would be overwritten in the case of a timestamp conflict
-             */
-            public TemporalCell getEarliestCell()
-            {
-                assert numSet > 0;
-
-                if (numSet == 1)
-                    return existingCell == null ? newCell : existingCell;
-
-                TemporalCell latest = existingCell.reconcile(newCell);
-
-                return latest == newCell ? existingCell : newCell;
-            }
-
-            /**
-             * @return the cell that is latest
-             * Or would be the winner in the case of a timestamp conflict
-             */
-            public TemporalCell getLatestCell()
-            {
-                assert numSet > 0;
-
-                if (numSet == 1)
-                    return existingCell == null ? newCell : existingCell;
-
-                return existingCell.reconcile(newCell);
-            }
-
-            /**
-             * @return the new cell if it updates the existing cell
-             */
-            public TemporalCell getOldCellIfUpdated()
-            {
-                assert numSet > 0;
-
-                if (numSet == 1)
-                   return null;
-
-                TemporalCell value = existingCell.reconcile(newCell);
-
-                return ByteBufferUtil.compareUnsigned(existingCell.value, value.value) != 0 ? existingCell : null;
-            }
-
-            void setVersion(TemporalCell cell)
-            {
-                assert cell != null;
-
-                if (cell.isNew)
-                {
-                    assert newCell == null || newCell.equals(cell) : "Only one cell version can be marked New; newCell: " + newCell + ", cell: " + cell;
-                    newCell = cell;
-                    numSet = existingCell == null ? 1 : 2;
-                }
-                else
-                {
-                    assert existingCell == null || existingCell.equals(cell) : "Only one cell version can be marked Existing; existingCell: " + existingCell + ", cell: " + cell;
-                    existingCell = cell;
-                    numSet = newCell == null ? 1 : 2;
-                }
-            }
-
-            public void addToRow(TemporalRow row, ColumnIdentifier column, CellPath path)
-            {
-                if (existingCell != null)
-                    row.addColumnValue(column, path, existingCell.timestamp, existingCell.ttl,
-                                       existingCell.localDeletionTime, existingCell.value, existingCell.isNew);
-
-                if (newCell != null)
-                    row.addColumnValue(column, path, newCell.timestamp, newCell.ttl,
-                                       newCell.localDeletionTime, newCell.value, newCell.isNew);
-            }
-
-            @Override
-            public String toString()
-            {
-                return MoreObjects.toStringHelper(this)
-                        .add("numSet", numSet)
-                        .add("existingCell", existingCell)
-                        .add("newCell", newCell)
-                        .toString();
-            }
-        }
-    }
-
-    private final ColumnFamilyStore baseCfs;
-    private final java.util.Set<ColumnIdentifier> viewPrimaryKey;
-    private final ByteBuffer basePartitionKey;
-    private final Map<ColumnIdentifier, ByteBuffer> clusteringColumns;
-    private final Row startRow;
-    private final boolean startIsNew;
-
-    public final int nowInSec;
-    private final Map<ColumnIdentifier, Map<CellPath, TemporalCell.Versions>> columnValues = new HashMap<>();
-    private int viewClusteringTtl = NO_TTL;
-    private long viewClusteringTimestamp = NO_TIMESTAMP;
-    private int viewClusteringLocalDeletionTime = NO_DELETION_TIME;
-
-    TemporalRow(ColumnFamilyStore baseCfs, java.util.Set<ColumnIdentifier> viewPrimaryKey, ByteBuffer key, Row row, int nowInSec, boolean isNew)
-    {
-        this.baseCfs = baseCfs;
-        this.viewPrimaryKey = viewPrimaryKey;
-        this.basePartitionKey = key;
-        this.startRow = row;
-        this.startIsNew = isNew;
-        this.nowInSec = nowInSec;
-
-        LivenessInfo liveness = row.primaryKeyLivenessInfo();
-        updateLiveness(liveness.ttl(), liveness.timestamp(), row.deletion().time().localDeletionTime());
-
-        List<ColumnDefinition> clusteringDefs = baseCfs.metadata.clusteringColumns();
-        clusteringColumns = new HashMap<>();
-
-        for (int i = 0; i < clusteringDefs.size(); i++)
-        {
-            ColumnDefinition cdef = clusteringDefs.get(i);
-            clusteringColumns.put(cdef.name, row.clustering().get(i));
-
-            addColumnValue(cdef.name, null, NO_TIMESTAMP, NO_TTL, NO_DELETION_TIME, row.clustering().get(i), isNew);
-        }
-    }
-
-    /*
-     * PK ts:5, ttl:1, deletion: 2
-     * Col ts:4, ttl:2, deletion: 3
-     *
-     * TTL use min, since it expires at the lowest time which we are expiring. If we have the above values, we
-     * would want to return 1, since the base row expires in 1 second.
-     *
-     * Timestamp uses max, as this is the time that the row has been written to the view. See CASSANDRA-10910.
-     *
-     * Local Deletion Time should use max, as this deletion will cover all previous values written.
-     */
-    private void updateLiveness(int ttl, long timestamp, int localDeletionTime)
-    {
-        // We are returning whichever is higher from valueIfSet
-        // Natural order will return the max: 1.compareTo(2) < 0, so 2 is returned
-        // Reverse order will return the min: 1.compareTo(2) > 0, so 1 is returned
-        this.viewClusteringTtl = valueIfSet(viewClusteringTtl, ttl, NO_TTL, reverseOrder());
-        this.viewClusteringTimestamp = valueIfSet(viewClusteringTimestamp, timestamp, NO_TIMESTAMP, naturalOrder());
-        this.viewClusteringLocalDeletionTime = valueIfSet(viewClusteringLocalDeletionTime, localDeletionTime, NO_DELETION_TIME, naturalOrder());
-    }
-
-    @Override
-    public String toString()
-    {
-        return MoreObjects.toStringHelper(this)
-                .add("table", baseCfs.keyspace.getName() + "." + baseCfs.getTableName())
-                .add("basePartitionKey", ByteBufferUtil.bytesToHex(basePartitionKey))
-                .add("startRow", startRow.toString(baseCfs.metadata))
-                .add("startIsNew", startIsNew)
-                .add("nowInSec", nowInSec)
-                .add("viewClusteringTtl", viewClusteringTtl)
-                .add("viewClusteringTimestamp", viewClusteringTimestamp)
-                .add("viewClusteringLocalDeletionTime", viewClusteringLocalDeletionTime)
-                .add("columnValues", columnValues)
-                .toString();
-    }
-
-    @Override
-    public boolean equals(Object o)
-    {
-        if (this == o) return true;
-        if (o == null || getClass() != o.getClass()) return false;
-
-        TemporalRow that = (TemporalRow) o;
-
-        if (!clusteringColumns.equals(that.clusteringColumns)) return false;
-        if (!basePartitionKey.equals(that.basePartitionKey)) return false;
-
-        return true;
-    }
-
-    @Override
-    public int hashCode()
-    {
-        int result = basePartitionKey.hashCode();
-        result = 31 * result + clusteringColumns.hashCode();
-        return result;
-    }
-
-    public void addColumnValue(ColumnIdentifier identifier,
-                               CellPath cellPath,
-                               long timestamp,
-                               int ttl,
-                               int localDeletionTime,
-                               ByteBuffer value,  boolean isNew)
-    {
-        if (!columnValues.containsKey(identifier))
-            columnValues.put(identifier, new HashMap<>());
-
-        Map<CellPath, TemporalCell.Versions> innerMap = columnValues.get(identifier);
-
-        if (!innerMap.containsKey(cellPath))
-            innerMap.put(cellPath, new TemporalCell.Versions());
-
-        // If this column is part of the view's primary keys
-        if (viewPrimaryKey.contains(identifier))
-        {
-            updateLiveness(ttl, timestamp, localDeletionTime);
-        }
-
-        innerMap.get(cellPath).setVersion(new TemporalCell(value, timestamp, ttl, localDeletionTime, isNew));
-    }
-
-    /**
-     * @return
-     * <ul>
-     *     <li>
-     *         If both existing and update are defaultValue, return defaultValue
-     *     </li>
-     *     <li>
-     *         If only one of existing or existing are defaultValue, return the one which is not
-     *     </li>
-     *     <li>
-     *         If both existing and update are not defaultValue, compare using comparator and return the higher one.
-     *     </li>
-     * </ul>
-     */
-    private static <T> T valueIfSet(T existing, T update, T defaultValue, Comparator<T> comparator)
-    {
-        if (existing.equals(defaultValue))
-            return update;
-        if (update.equals(defaultValue))
-            return existing;
-        return comparator.compare(existing, update) > 0 ? existing : update;
-    }
-
-    public int viewClusteringTtl()
-    {
-        return viewClusteringTtl;
-    }
-
-    public long viewClusteringTimestamp()
-    {
-        return viewClusteringTimestamp;
-    }
-
-    public int viewClusteringLocalDeletionTime()
-    {
-        return viewClusteringLocalDeletionTime;
-    }
-
-    public void addCell(Cell cell, boolean isNew)
-    {
-        addColumnValue(cell.column().name, cell.path(), cell.timestamp(), cell.ttl(), cell.localDeletionTime(), cell.value(), isNew);
-    }
-
-    // The Definition here is actually the *base table* definition
-    public ByteBuffer clusteringValue(ColumnDefinition definition, Resolver resolver)
-    {
-        ColumnDefinition baseDefinition = definition.cfName.equals(baseCfs.name)
-                                          ? definition
-                                          : baseCfs.metadata.getColumnDefinition(definition.name);
-
-        if (baseDefinition.isPartitionKey())
-        {
-            if (baseCfs.metadata.getKeyValidator() instanceof CompositeType)
-            {
-                CompositeType keyComparator = (CompositeType) baseCfs.metadata.getKeyValidator();
-                ByteBuffer[] components = keyComparator.split(basePartitionKey);
-                return components[baseDefinition.position()];
-            }
-            else
-            {
-                return basePartitionKey;
-            }
-        }
-        else
-        {
-            ColumnIdentifier columnIdentifier = baseDefinition.name;
-
-            if (clusteringColumns.containsKey(columnIdentifier))
-                return clusteringColumns.get(columnIdentifier);
-
-            Collection<org.apache.cassandra.db.rows.Cell> val = values(definition, resolver);
-            if (val != null && val.size() == 1)
-            {
-                org.apache.cassandra.db.rows.Cell cell = Iterables.getOnlyElement(val);
-                // handle single-column deletions correctly
-                return cell.isTombstone() ? null : cell.value();
-            }
-        }
-        return null;
-    }
-
-    public DeletionTime deletionTime(AbstractBTreePartition partition)
-    {
-        DeletionInfo deletionInfo = partition.deletionInfo();
-        if (!deletionInfo.getPartitionDeletion().isLive())
-            return deletionInfo.getPartitionDeletion();
-
-        Clustering baseClustering = baseClusteringBuilder().build();
-        RangeTombstone clusterTombstone = deletionInfo.rangeCovering(baseClustering);
-        if (clusterTombstone != null)
-            return clusterTombstone.deletionTime();
-
-        Row row = partition.getRow(baseClustering);
-        return row == null || row.deletion().isLive() ? DeletionTime.LIVE : row.deletion().time();
-    }
-
-    public Collection<org.apache.cassandra.db.rows.Cell> values(ColumnDefinition definition, Resolver resolver)
-    {
-        Map<CellPath, TemporalCell.Versions> innerMap = columnValues.get(definition.name);
-        if (innerMap == null)
-        {
-            return Collections.emptyList();
-        }
-
-        Collection<org.apache.cassandra.db.rows.Cell> value = new ArrayList<>();
-        for (Map.Entry<CellPath, TemporalCell.Versions> pathAndCells : innerMap.entrySet())
-        {
-            TemporalCell cell = resolver.resolve(pathAndCells.getValue());
-
-            if (cell != null)
-                value.add(cell.cell(definition, pathAndCells.getKey()));
-        }
-        return value;
-    }
-
-    public Slice baseSlice()
-    {
-        return baseClusteringBuilder().buildSlice();
-    }
-
-    private CBuilder baseClusteringBuilder()
-    {
-        CFMetaData metadata = baseCfs.metadata;
-        CBuilder builder = CBuilder.create(metadata.comparator);
-
-        ByteBuffer[] buffers = new ByteBuffer[clusteringColumns.size()];
-        for (Map.Entry<ColumnIdentifier, ByteBuffer> buffer : clusteringColumns.entrySet())
-            buffers[metadata.getColumnDefinition(buffer.getKey()).position()] = buffer.getValue();
-
-        for (ByteBuffer byteBuffer : buffers)
-            builder = builder.add(byteBuffer);
-
-        return builder;
-    }
-
-    public Clustering baseClustering()
-    {
-        return startRow.clustering();
-    }
-
-    static class Set implements Iterable<TemporalRow>
-    {
-        private final ColumnFamilyStore baseCfs;
-        private final java.util.Set<ColumnIdentifier> viewPrimaryKey;
-        private final ByteBuffer key;
-        public final DecoratedKey dk;
-        private final Map<Clustering, TemporalRow> clusteringToRow;
-        final int nowInSec = FBUtilities.nowInSeconds();
-        private boolean hasTombstonedExisting = false;
-
-        Set(ColumnFamilyStore baseCfs, java.util.Set<ColumnIdentifier> viewPrimaryKey, ByteBuffer key)
-        {
-            this.baseCfs = baseCfs;
-            this.viewPrimaryKey = viewPrimaryKey;
-            this.key = key;
-            this.dk = baseCfs.decorateKey(key);
-            this.clusteringToRow = new HashMap<>();
-        }
-
-        public Iterator<TemporalRow> iterator()
-        {
-            return clusteringToRow.values().iterator();
-        }
-
-        public TemporalRow getClustering(Clustering clustering)
-        {
-            return clusteringToRow.get(clustering);
-        }
-
-        public void addRow(Row row, boolean isNew)
-        {
-            TemporalRow temporalRow = clusteringToRow.get(row.clustering());
-            if (temporalRow == null)
-            {
-                temporalRow = new TemporalRow(baseCfs, viewPrimaryKey, key, row, nowInSec, isNew);
-                clusteringToRow.put(row.clustering(), temporalRow);
-            }
-
-            for (Cell cell : row.cells())
-            {
-                temporalRow.addCell(cell, isNew);
-            }
-        }
-
-        private void addRow(TemporalRow row)
-        {
-            TemporalRow newRow = new TemporalRow(baseCfs, viewPrimaryKey, key, row.startRow, nowInSec, row.startIsNew);
-
-            TemporalRow existing = clusteringToRow.put(row.startRow.clustering(), newRow);
-            assert existing == null;
-
-            for (Map.Entry<ColumnIdentifier, Map<CellPath, TemporalCell.Versions>> entry : row.columnValues.entrySet())
-            {
-                for (Map.Entry<CellPath, TemporalCell.Versions> cellPathEntry : entry.getValue().entrySet())
-                {
-                    TemporalCell.Versions cellVersions = cellPathEntry.getValue();
-
-                    cellVersions.addToRow(newRow, entry.getKey(), cellPathEntry.getKey());
-                }
-            }
-        }
-
-        public TemporalRow.Set withNewViewPrimaryKey(java.util.Set<ColumnIdentifier> viewPrimaryKey)
-        {
-            TemporalRow.Set newSet = new Set(baseCfs, viewPrimaryKey, key);
-
-            for (TemporalRow row : this)
-                newSet.addRow(row);
-
-            return newSet;
-        }
-
-        public boolean hasTombstonedExisting()
-        {
-            return hasTombstonedExisting;
-        }
-
-        public void setTombstonedExisting()
-        {
-            hasTombstonedExisting = true;
-        }
-
-        public int size()
-        {
-            return clusteringToRow.size();
-        }
-    }
-}


[8/9] cassandra git commit: Merge commit '86ba227477b9f8595eb610ecaf950cfbc29dd36b' into cassandra-3.7

Posted by sl...@apache.org.
Merge commit '86ba227477b9f8595eb610ecaf950cfbc29dd36b' into cassandra-3.7


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

Branch: refs/heads/cassandra-3.7
Commit: a87fd715d6b26128603a404074ec3df42a595b2e
Parents: 4e364d7 86ba227
Author: Sylvain Lebresne <sy...@datastax.com>
Authored: Fri May 6 13:43:44 2016 +0200
Committer: Sylvain Lebresne <sy...@datastax.com>
Committed: Fri May 6 13:44:12 2016 +0200

----------------------------------------------------------------------
 .../org/apache/cassandra/config/CFMetaData.java |   6 +
 .../apache/cassandra/config/ViewDefinition.java |   1 -
 .../cql3/statements/CreateViewStatement.java    |   4 +-
 .../cql3/statements/SelectStatement.java        |  41 +-
 .../apache/cassandra/db/ColumnFamilyStore.java  |   6 +-
 src/java/org/apache/cassandra/db/Keyspace.java  |   2 +-
 .../db/SinglePartitionReadCommand.java          |  33 +
 src/java/org/apache/cassandra/db/Slices.java    |   7 +
 .../apache/cassandra/db/filter/RowFilter.java   |  24 +
 .../SingletonUnfilteredPartitionIterator.java   |   3 +-
 .../apache/cassandra/db/rows/AbstractCell.java  |   5 +
 .../org/apache/cassandra/db/rows/BTreeRow.java  |  34 +-
 .../apache/cassandra/db/rows/BufferCell.java    |   5 +
 src/java/org/apache/cassandra/db/rows/Cell.java |   2 +
 .../apache/cassandra/db/rows/ColumnData.java    |   2 +
 .../cassandra/db/rows/ComplexColumnData.java    |   8 +
 .../apache/cassandra/db/rows/NativeCell.java    |   5 +
 src/java/org/apache/cassandra/db/rows/Row.java  |  35 +-
 .../cassandra/db/rows/RowDiffListener.java      |   2 +-
 .../db/rows/UnfilteredRowIterators.java         |   2 +-
 .../apache/cassandra/db/view/TableViews.java    | 481 ++++++++++++++
 .../apache/cassandra/db/view/TemporalRow.java   | 601 ------------------
 src/java/org/apache/cassandra/db/view/View.java | 629 ++-----------------
 .../apache/cassandra/db/view/ViewBuilder.java   |  38 +-
 .../apache/cassandra/db/view/ViewManager.java   | 146 +----
 .../cassandra/db/view/ViewUpdateGenerator.java  | 549 ++++++++++++++++
 .../org/apache/cassandra/cql3/ViewTest.java     |  52 +-
 .../org/apache/cassandra/db/rows/RowsTest.java  |   6 +-
 28 files changed, 1400 insertions(+), 1329 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/cassandra/blob/a87fd715/src/java/org/apache/cassandra/config/CFMetaData.java
----------------------------------------------------------------------

http://git-wip-us.apache.org/repos/asf/cassandra/blob/a87fd715/src/java/org/apache/cassandra/config/ViewDefinition.java
----------------------------------------------------------------------

http://git-wip-us.apache.org/repos/asf/cassandra/blob/a87fd715/src/java/org/apache/cassandra/cql3/statements/CreateViewStatement.java
----------------------------------------------------------------------

http://git-wip-us.apache.org/repos/asf/cassandra/blob/a87fd715/src/java/org/apache/cassandra/cql3/statements/SelectStatement.java
----------------------------------------------------------------------

http://git-wip-us.apache.org/repos/asf/cassandra/blob/a87fd715/src/java/org/apache/cassandra/db/ColumnFamilyStore.java
----------------------------------------------------------------------

http://git-wip-us.apache.org/repos/asf/cassandra/blob/a87fd715/src/java/org/apache/cassandra/db/Keyspace.java
----------------------------------------------------------------------

http://git-wip-us.apache.org/repos/asf/cassandra/blob/a87fd715/src/java/org/apache/cassandra/db/SinglePartitionReadCommand.java
----------------------------------------------------------------------

http://git-wip-us.apache.org/repos/asf/cassandra/blob/a87fd715/src/java/org/apache/cassandra/db/Slices.java
----------------------------------------------------------------------

http://git-wip-us.apache.org/repos/asf/cassandra/blob/a87fd715/src/java/org/apache/cassandra/db/filter/RowFilter.java
----------------------------------------------------------------------

http://git-wip-us.apache.org/repos/asf/cassandra/blob/a87fd715/src/java/org/apache/cassandra/db/rows/AbstractCell.java
----------------------------------------------------------------------

http://git-wip-us.apache.org/repos/asf/cassandra/blob/a87fd715/src/java/org/apache/cassandra/db/rows/BTreeRow.java
----------------------------------------------------------------------

http://git-wip-us.apache.org/repos/asf/cassandra/blob/a87fd715/src/java/org/apache/cassandra/db/rows/BufferCell.java
----------------------------------------------------------------------

http://git-wip-us.apache.org/repos/asf/cassandra/blob/a87fd715/src/java/org/apache/cassandra/db/rows/Cell.java
----------------------------------------------------------------------

http://git-wip-us.apache.org/repos/asf/cassandra/blob/a87fd715/src/java/org/apache/cassandra/db/rows/ComplexColumnData.java
----------------------------------------------------------------------

http://git-wip-us.apache.org/repos/asf/cassandra/blob/a87fd715/src/java/org/apache/cassandra/db/rows/NativeCell.java
----------------------------------------------------------------------
diff --cc src/java/org/apache/cassandra/db/rows/NativeCell.java
index 9d816f3,0000000..5930332
mode 100644,000000..100644
--- a/src/java/org/apache/cassandra/db/rows/NativeCell.java
+++ b/src/java/org/apache/cassandra/db/rows/NativeCell.java
@@@ -1,151 -1,0 +1,156 @@@
 +/*
 + * Licensed to the Apache Software Foundation (ASF) under one
 + * or more contributor license agreements.  See the NOTICE file
 + * distributed with this work for additional information
 + * regarding copyright ownership.  The ASF licenses this file
 + * to you under the Apache License, Version 2.0 (the
 + * "License"); you may not use this file except in compliance
 + * with the License.  You may obtain a copy of the License at
 + *
 + *     http://www.apache.org/licenses/LICENSE-2.0
 + *
 + * Unless required by applicable law or agreed to in writing, software
 + * distributed under the License is distributed on an "AS IS" BASIS,
 + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
 + * See the License for the specific language governing permissions and
 + * limitations under the License.
 + */
 +package org.apache.cassandra.db.rows;
 +
 +import java.nio.ByteBuffer;
 +import java.nio.ByteOrder;
 +
 +import org.apache.cassandra.config.ColumnDefinition;
 +import org.apache.cassandra.utils.ObjectSizes;
 +import org.apache.cassandra.utils.concurrent.OpOrder;
 +import org.apache.cassandra.utils.memory.MemoryUtil;
 +import org.apache.cassandra.utils.memory.NativeAllocator;
 +
 +public class NativeCell extends AbstractCell
 +{
 +    private static final long EMPTY_SIZE = ObjectSizes.measure(new NativeCell());
 +
 +    private static final long HAS_CELLPATH = 0;
 +    private static final long TIMESTAMP = 1;
 +    private static final long TTL = 9;
 +    private static final long DELETION = 13;
 +    private static final long LENGTH = 17;
 +    private static final long VALUE = 21;
 +
 +    private final long peer;
 +
 +    private NativeCell()
 +    {
 +        super(null);
 +        this.peer = 0;
 +    }
 +
 +    public NativeCell(NativeAllocator allocator,
 +                      OpOrder.Group writeOp,
 +                      Cell cell)
 +    {
 +        this(allocator,
 +             writeOp,
 +             cell.column(),
 +             cell.timestamp(),
 +             cell.ttl(),
 +             cell.localDeletionTime(),
 +             cell.value(),
 +             cell.path());
 +    }
 +
 +    public NativeCell(NativeAllocator allocator,
 +                      OpOrder.Group writeOp,
 +                      ColumnDefinition column,
 +                      long timestamp,
 +                      int ttl,
 +                      int localDeletionTime,
 +                      ByteBuffer value,
 +                      CellPath path)
 +    {
 +        super(column);
 +        long size = simpleSize(value.remaining());
 +
 +        assert value.order() == ByteOrder.BIG_ENDIAN;
 +        assert column.isComplex() == (path != null);
 +        if (path != null)
 +        {
 +            assert path.size() == 1;
 +            size += 4 + path.get(0).remaining();
 +        }
 +
 +        if (size > Integer.MAX_VALUE)
 +            throw new IllegalStateException();
 +
 +        // cellpath? : timestamp : ttl : localDeletionTime : length : <data> : [cell path length] : [<cell path data>]
 +        peer = allocator.allocate((int) size, writeOp);
 +        MemoryUtil.setByte(peer + HAS_CELLPATH, (byte)(path == null ? 0 : 1));
 +        MemoryUtil.setLong(peer + TIMESTAMP, timestamp);
 +        MemoryUtil.setInt(peer + TTL, ttl);
 +        MemoryUtil.setInt(peer + DELETION, localDeletionTime);
 +        MemoryUtil.setInt(peer + LENGTH, value.remaining());
 +        MemoryUtil.setBytes(peer + VALUE, value);
 +
 +        if (path != null)
 +        {
 +            ByteBuffer pathbuffer = path.get(0);
 +            assert pathbuffer.order() == ByteOrder.BIG_ENDIAN;
 +
 +            long offset = peer + VALUE + value.remaining();
 +            MemoryUtil.setInt(offset, pathbuffer.remaining());
 +            MemoryUtil.setBytes(offset + 4, pathbuffer);
 +        }
 +    }
 +
 +    private static long simpleSize(int length)
 +    {
 +        return VALUE + length;
 +    }
 +
 +    public long timestamp()
 +    {
 +        return MemoryUtil.getLong(peer + TIMESTAMP);
 +    }
 +
 +    public int ttl()
 +    {
 +        return MemoryUtil.getInt(peer + TTL);
 +    }
 +
 +    public int localDeletionTime()
 +    {
 +        return MemoryUtil.getInt(peer + DELETION);
 +    }
 +
 +    public ByteBuffer value()
 +    {
 +        int length = MemoryUtil.getInt(peer + LENGTH);
 +        return MemoryUtil.getByteBuffer(peer + VALUE, length, ByteOrder.BIG_ENDIAN);
 +    }
 +
 +    public CellPath path()
 +    {
 +        if (MemoryUtil.getByte(peer+ HAS_CELLPATH) == 0)
 +            return null;
 +
 +        long offset = peer + VALUE + MemoryUtil.getInt(peer + LENGTH);
 +        int size = MemoryUtil.getInt(offset);
 +        return CellPath.create(MemoryUtil.getByteBuffer(offset + 4, size, ByteOrder.BIG_ENDIAN));
 +    }
 +
 +    public Cell withUpdatedValue(ByteBuffer newValue)
 +    {
 +        throw new UnsupportedOperationException();
 +    }
 +
++    public Cell withUpdatedColumn(ColumnDefinition column)
++    {
++        return new BufferCell(column, timestamp(), ttl(), localDeletionTime(), value(), path());
++    }
++
 +    public long unsharedHeapSizeExcludingData()
 +    {
 +        return EMPTY_SIZE;
 +    }
 +
 +}

http://git-wip-us.apache.org/repos/asf/cassandra/blob/a87fd715/src/java/org/apache/cassandra/db/rows/Row.java
----------------------------------------------------------------------

http://git-wip-us.apache.org/repos/asf/cassandra/blob/a87fd715/src/java/org/apache/cassandra/db/rows/RowDiffListener.java
----------------------------------------------------------------------

http://git-wip-us.apache.org/repos/asf/cassandra/blob/a87fd715/src/java/org/apache/cassandra/db/rows/UnfilteredRowIterators.java
----------------------------------------------------------------------

http://git-wip-us.apache.org/repos/asf/cassandra/blob/a87fd715/src/java/org/apache/cassandra/db/view/TableViews.java
----------------------------------------------------------------------
diff --cc src/java/org/apache/cassandra/db/view/TableViews.java
index 0000000,893bdd5..e97e01c
mode 000000,100644..100644
--- a/src/java/org/apache/cassandra/db/view/TableViews.java
+++ b/src/java/org/apache/cassandra/db/view/TableViews.java
@@@ -1,0 -1,481 +1,481 @@@
+ /*
+  * Licensed to the Apache Software Foundation (ASF) under one
+  * or more contributor license agreements.  See the NOTICE file
+  * distributed with this work for additional information
+  * regarding copyright ownership.  The ASF licenses this file
+  * to you under the Apache License, Version 2.0 (the
+  * "License"); you may not use this file except in compliance
+  * with the License.  You may obtain a copy of the License at
+  *
+  *     http://www.apache.org/licenses/LICENSE-2.0
+  *
+  * Unless required by applicable law or agreed to in writing, software
+  * distributed under the License is distributed on an "AS IS" BASIS,
+  * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+  * See the License for the specific language governing permissions and
+  * limitations under the License.
+  */
+ package org.apache.cassandra.db.view;
+ 
+ import java.util.*;
+ import java.util.concurrent.CopyOnWriteArrayList;
+ import java.util.concurrent.TimeUnit;
+ import java.util.concurrent.atomic.AtomicLong;
+ 
+ import com.google.common.collect.Iterables;
+ import com.google.common.collect.Iterators;
+ import com.google.common.collect.PeekingIterator;
+ 
+ import org.apache.cassandra.config.CFMetaData;
+ import org.apache.cassandra.db.*;
+ import org.apache.cassandra.db.commitlog.ReplayPosition;
+ import org.apache.cassandra.db.filter.*;
+ import org.apache.cassandra.db.rows.*;
+ import org.apache.cassandra.db.partitions.*;
+ import org.apache.cassandra.dht.Token;
+ import org.apache.cassandra.service.StorageProxy;
+ import org.apache.cassandra.utils.FBUtilities;
+ import org.apache.cassandra.utils.btree.BTreeSet;
+ 
+ 
+ /**
+  * Groups all the views for a given table.
+  */
+ public class TableViews extends AbstractCollection<View>
+ {
+     private final CFMetaData baseTableMetadata;
+ 
+     // We need this to be thread-safe, but the number of times this is changed (when a view is created in the keyspace)
+     // massively exceeds the number of time it's read (for every mutation on the keyspace), so a copy-on-write list is the best option.
+     private final List<View> views = new CopyOnWriteArrayList();
+ 
+     public TableViews(CFMetaData baseTableMetadata)
+     {
+         this.baseTableMetadata = baseTableMetadata;
+     }
+ 
+     public int size()
+     {
+         return views.size();
+     }
+ 
+     public Iterator<View> iterator()
+     {
+         return views.iterator();
+     }
+ 
+     public boolean contains(String viewName)
+     {
+         return Iterables.any(views, view -> view.name.equals(viewName));
+     }
+ 
+     public boolean add(View view)
+     {
+         // We should have validated that there is no existing view with this name at this point
+         assert !contains(view.name);
+         return views.add(view);
+     }
+ 
+     public Iterable<ColumnFamilyStore> allViewsCfs()
+     {
+         Keyspace keyspace = Keyspace.open(baseTableMetadata.ksName);
+         return Iterables.transform(views, view -> keyspace.getColumnFamilyStore(view.getDefinition().viewName));
+     }
+ 
+     public void forceBlockingFlush()
+     {
+         for (ColumnFamilyStore viewCfs : allViewsCfs())
+             viewCfs.forceBlockingFlush();
+     }
+ 
+     public void dumpMemtables()
+     {
+         for (ColumnFamilyStore viewCfs : allViewsCfs())
+             viewCfs.dumpMemtable();
+     }
+ 
+     public void truncateBlocking(long truncatedAt)
+     {
+         for (ColumnFamilyStore viewCfs : allViewsCfs())
+         {
+             ReplayPosition replayAfter = viewCfs.discardSSTables(truncatedAt);
+             SystemKeyspace.saveTruncationRecord(viewCfs, truncatedAt, replayAfter);
+         }
+     }
+ 
+     public void removeByName(String viewName)
+     {
+         views.removeIf(v -> v.name.equals(viewName));
+     }
+ 
+     /**
+      * Calculates and pushes updates to the views replicas. The replicas are determined by
+      * {@link ViewUtils#getViewNaturalEndpoint(String, Token, Token)}.
+      *
+      * @param update an update on the base table represented by this object.
+      * @param writeCommitLog whether we should write the commit log for the view updates.
+      * @param baseComplete time from epoch in ms that the local base mutation was (or will be) completed
+      */
+     public void pushViewReplicaUpdates(PartitionUpdate update, boolean writeCommitLog, AtomicLong baseComplete)
+     {
+         assert update.metadata().cfId.equals(baseTableMetadata.cfId);
+ 
+         Collection<View> views = updatedViews(update);
+         if (views.isEmpty())
+             return;
+ 
+         // Read modified rows
+         int nowInSec = FBUtilities.nowInSeconds();
+         SinglePartitionReadCommand command = readExistingRowsCommand(update, views, nowInSec);
+         if (command == null)
+             return;
+ 
+         ColumnFamilyStore cfs = Keyspace.openAndGetStore(update.metadata());
+         long start = System.nanoTime();
+         Collection<Mutation> mutations;
 -        try (ReadOrderGroup orderGroup = command.startOrderGroup();
++        try (ReadExecutionController orderGroup = command.executionController();
+              UnfilteredRowIterator existings = UnfilteredPartitionIterators.getOnlyElement(command.executeLocally(orderGroup), command);
+              UnfilteredRowIterator updates = update.unfilteredIterator())
+         {
+             mutations = generateViewUpdates(views, updates, existings, nowInSec);
+         }
+         Keyspace.openAndGetStore(update.metadata()).metric.viewReadTime.update(System.nanoTime() - start, TimeUnit.NANOSECONDS);
+ 
+         if (!mutations.isEmpty())
+             StorageProxy.mutateMV(update.partitionKey().getKey(), mutations, writeCommitLog, baseComplete);
+     }
+ 
+     /**
+      * Given some updates on the base table of this object and the existing values for the rows affected by that update, generates the
+      * mutation to be applied to the provided views.
+      *
+      * @param views the views potentially affected by {@code updates}.
+      * @param updates the base table updates being applied.
+      * @param existings the existing values for the rows affected by {@code updates}. This is used to decide if a view is
+      * obsoleted by the update and should be removed, gather the values for columns that may not be part of the update if
+      * a new view entry needs to be created, and compute the minimal updates to be applied if the view entry isn't changed
+      * 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.
+      * @return the mutations to apply to the {@code views}. This can be empty.
+      */
+     public Collection<Mutation> generateViewUpdates(Collection<View> views, UnfilteredRowIterator updates, UnfilteredRowIterator existings, int nowInSec)
+     {
+         assert updates.metadata().cfId.equals(baseTableMetadata.cfId);
+ 
+         List<ViewUpdateGenerator> generators = new ArrayList<>(views.size());
+         for (View view : views)
+             generators.add(new ViewUpdateGenerator(view, updates.partitionKey(), nowInSec));
+ 
+         DeletionTracker existingsDeletion = new DeletionTracker(existings.partitionLevelDeletion());
+         DeletionTracker updatesDeletion = new DeletionTracker(updates.partitionLevelDeletion());
+ 
+         /*
+          * We iterate through the updates and the existing rows in parallel. This allows us to know the consequence
+          * on the view of each update.
+          */
+         PeekingIterator<Unfiltered> existingsIter = Iterators.peekingIterator(existings);
+         PeekingIterator<Unfiltered> updatesIter = Iterators.peekingIterator(updates);
+ 
+         while (existingsIter.hasNext() && updatesIter.hasNext())
+         {
+             Unfiltered existing = existingsIter.peek();
+             Unfiltered update = updatesIter.peek();
+ 
+             Row existingRow;
+             Row updateRow;
+             int cmp = baseTableMetadata.comparator.compare(update, existing);
+             if (cmp < 0)
+             {
+                 // We have an update where there was nothing before
+                 if (update.isRangeTombstoneMarker())
+                 {
+                     updatesDeletion.update(updatesIter.next());
+                     continue;
+                 }
+ 
+                 updateRow = ((Row)updatesIter.next()).withRowDeletion(updatesDeletion.currentDeletion());
+                 existingRow = emptyRow(updateRow.clustering(), existingsDeletion.currentDeletion());
+             }
+             else if (cmp > 0)
+             {
+                 // We have something existing but no update (which will happen either because it's a range tombstone marker in
+                 // existing, or because we've fetched the existing row due to some partition/range deletion in the updates)
+                 if (existing.isRangeTombstoneMarker())
+                 {
+                     existingsDeletion.update(existingsIter.next());
+                     continue;
+                 }
+ 
+                 existingRow = ((Row)existingsIter.next()).withRowDeletion(existingsDeletion.currentDeletion());
+                 updateRow = emptyRow(existingRow.clustering(), updatesDeletion.currentDeletion());
+ 
+                 // The way we build the read command used for existing rows, we should always have updatesDeletion.currentDeletion()
+                 // that is not live, since we wouldn't have read the existing row otherwise. And we could assert that, but if we ever
+                 // change the read method so that it can slightly over-read in some case, that would be an easily avoiding bug lurking,
+                 // so we just handle the case.
+                 if (updateRow == null)
+                     continue;
+             }
+             else
+             {
+                 // We're updating a row that had pre-existing data
+                 if (update.isRangeTombstoneMarker())
+                 {
+                     assert existing.isRangeTombstoneMarker();
+                     updatesDeletion.update(updatesIter.next());
+                     existingsDeletion.update(existingsIter.next());
+                     continue;
+                 }
+ 
+                 assert !existing.isRangeTombstoneMarker();
+                 existingRow = ((Row)existingsIter.next()).withRowDeletion(existingsDeletion.currentDeletion());
+                 updateRow = ((Row)updatesIter.next()).withRowDeletion(updatesDeletion.currentDeletion());
+             }
+ 
+             addToViewUpdateGenerators(existingRow, updateRow, generators, nowInSec);
+         }
+ 
+         // We only care about more existing rows if the update deletion isn't live, i.e. if we had a partition deletion
+         if (!updatesDeletion.currentDeletion().isLive())
+         {
+             while (existingsIter.hasNext())
+             {
+                 Unfiltered existing = existingsIter.next();
+                 // If it's a range tombstone, we don't care, we're only looking for existing entry that gets deleted by
+                 // the new partition deletion
+                 if (existing.isRangeTombstoneMarker())
+                     continue;
+ 
+                 Row existingRow = (Row)existing;
+                 addToViewUpdateGenerators(existingRow, emptyRow(existingRow.clustering(), updatesDeletion.currentDeletion()), generators, nowInSec);
+             }
+         }
+         while (updatesIter.hasNext())
+         {
+             Unfiltered update = updatesIter.next();
+             // If it's a range tombstone, it removes nothing pre-exisiting, so we can ignore it for view updates
+             if (update.isRangeTombstoneMarker())
+                 continue;
+ 
+             Row updateRow = (Row)update;
+             addToViewUpdateGenerators(emptyRow(updateRow.clustering(), DeletionTime.LIVE), updateRow, generators, nowInSec);
+         }
+ 
+         return buildMutations(baseTableMetadata, generators);
+     }
+ 
+     /**
+      * Return the views that are potentially updated by the provided updates.
+      *
+      * @param updates the updates applied to the base table.
+      * @return the views affected by {@code updates}.
+      */
+     public Collection<View> updatedViews(PartitionUpdate updates)
+     {
+         List<View> matchingViews = new ArrayList<>(views.size());
+ 
+         for (View view : views)
+         {
+             ReadQuery selectQuery = view.getReadQuery();
+             if (!selectQuery.selectsKey(updates.partitionKey()))
+                 continue;
+ 
+             matchingViews.add(view);
+         }
+         return matchingViews;
+     }
+ 
+     /**
+      * Returns the command to use to read the existing rows required to generate view updates for the provided base
+      * base updates.
+      *
+      * @param updates the base table updates being applied.
+      * @param views the views potentially affected by {@code updates}.
+      * @param nowInSec the current time in seconds.
+      * @return the command to use to read the base table rows required to generate view updates for {@code updates}.
+      */
+     private SinglePartitionReadCommand readExistingRowsCommand(PartitionUpdate updates, Collection<View> views, int nowInSec)
+     {
+         Slices.Builder sliceBuilder = null;
+         DeletionInfo deletionInfo = updates.deletionInfo();
+         CFMetaData metadata = updates.metadata();
+         DecoratedKey key = updates.partitionKey();
+         // TODO: This is subtle: we need to gather all the slices that we have to fetch between partition del, range tombstones and rows.
+         if (!deletionInfo.isLive())
+         {
+             sliceBuilder = new Slices.Builder(metadata.comparator);
+             // Everything covered by a deletion might invalidate an existing view entry, which means we must read it to know. In practice
+             // though, the views involved might filter some base table clustering columns, in which case we can restrict what we read
+             // using those restrictions.
+             // If there is a partition deletion, then we can simply take each slices from each view select filter. They may overlap but
+             // the Slices.Builder handles that for us. Note that in many case this will just involve reading everything (as soon as any
+             // view involved has no clustering restrictions for instance).
+             // For range tombstone, we should theoretically take the difference between the range tombstoned and the slices selected
+             // by every views, but as we don't an easy way to compute that right now, we keep it simple and just use the tombstoned
+             // range.
+             // TODO: we should improve that latter part.
+             if (!deletionInfo.getPartitionDeletion().isLive())
+             {
+                 for (View view : views)
+                     sliceBuilder.addAll(view.getSelectStatement().clusteringIndexFilterAsSlices());
+             }
+             else
+             {
+                 assert deletionInfo.hasRanges();
+                 Iterator<RangeTombstone> iter = deletionInfo.rangeIterator(false);
+                 while (iter.hasNext())
+                     sliceBuilder.add(iter.next().deletedSlice());
+             }
+         }
+ 
+         // We need to read every row that is updated, unless we can prove that it has no impact on any view entries.
+ 
+         // If we had some slices from the deletions above, we'll continue using that. Otherwise, it's more efficient to build
+         // a names query.
+         BTreeSet.Builder<Clustering> namesBuilder = sliceBuilder == null ? BTreeSet.builder(metadata.comparator) : null;
+         for (Row row : updates)
+         {
+             // Don't read the existing state if we can prove the update won't affect any views
+             if (!affectsAnyViews(key, row, views))
+                 continue;
+ 
+             if (namesBuilder == null)
+                 sliceBuilder.add(Slice.make(row.clustering()));
+             else
+                 namesBuilder.add(row.clustering());
+         }
+ 
+         NavigableSet<Clustering> names = namesBuilder == null ? null : namesBuilder.build();
+         // If we have a slice builder, it means we had some deletions and we have to read. But if we had
+         // only row updates, it's possible none of them affected the views, in which case we have nothing
+         // to do.
+         if (names != null && names.isEmpty())
+             return null;
+ 
+         ClusteringIndexFilter clusteringFilter = names == null
+                                                ? new ClusteringIndexSliceFilter(sliceBuilder.build(), false)
+                                                : new ClusteringIndexNamesFilter(names, false);
+         // 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);
+         // 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.
+         // TODO: we could still make sense to special case for when there is a single view and a small number of updates (and
+         // no deletions). Indeed, in that case we could check whether any of the update modify any of the restricted regular
+         // column, and if that's not the case we could use view filter. We keep it simple for now though.
+         RowFilter rowFilter = RowFilter.NONE;
+         return SinglePartitionReadCommand.create(metadata, nowInSec, queriedColumns, rowFilter, DataLimits.NONE, key, clusteringFilter);
+     }
+ 
+     private boolean affectsAnyViews(DecoratedKey partitionKey, Row update, Collection<View> views)
+     {
+         for (View view : views)
+         {
+             if (view.mayBeAffectedBy(partitionKey, update))
+                 return true;
+         }
+         return false;
+     }
+ 
+     /**
+      * Given an existing base row and the update that we're going to apply to this row, generate the modifications
+      * to apply to MVs using the provided {@code ViewUpdateGenerator}s.
+      *
+      * @param existingBaseRow the base table row as it is before an update.
+      * @param updateBaseRow the newly updates made to {@code existingBaseRow}.
+      * @param generators the view update generators to add the new changes to.
+      * @param nowInSec the current time in seconds. Used to decide if data is live or not.
+      */
+     private static void addToViewUpdateGenerators(Row existingBaseRow, Row updateBaseRow, Collection<ViewUpdateGenerator> generators, int nowInSec)
+     {
+         // Having existing empty is useful, it just means we'll insert a brand new entry for updateBaseRow,
+         // but if we have no update at all, we shouldn't get there.
+         assert !updateBaseRow.isEmpty();
+ 
+         // We allow existingBaseRow to be null, which we treat the same as being empty as an small optimization
+         // to avoid allocating empty row objects when we know there was nothing existing.
+         Row mergedBaseRow = existingBaseRow == null ? updateBaseRow : Rows.merge(existingBaseRow, updateBaseRow, nowInSec);
+         for (ViewUpdateGenerator generator : generators)
+             generator.addBaseTableUpdate(existingBaseRow, mergedBaseRow);
+     }
+ 
+     private static Row emptyRow(Clustering clustering, DeletionTime deletion)
+     {
+         // Returning null for an empty row is slightly ugly, but the case where there is no pre-existing row is fairly common
+         // (especially when building the view), so we want to avoid a dummy allocation of an empty row every time.
+         // And MultiViewUpdateBuilder knows how to deal with that.
+         return deletion.isLive() ? null : BTreeRow.emptyDeletedRow(clustering, Row.Deletion.regular(deletion));
+     }
+ 
+     /**
+      * Extracts (and potentially groups) the mutations generated by the provided view update generator.
+      * Returns the mutation that needs to be done to the views given the base table updates
+      * passed to {@link #addBaseTableUpdate}.
+      *
+      * @param baseTableMetadata the metadata for the base table being updated.
+      * @param generators the generators from which to extract the view mutations from.
+      * @return the mutations created by all the generators in {@code generators}.
+      */
+     private Collection<Mutation> buildMutations(CFMetaData baseTableMetadata, List<ViewUpdateGenerator> generators)
+     {
+         // One view is probably common enough and we can optimize a bit easily
+         if (generators.size() == 1)
+         {
+             Collection<PartitionUpdate> updates = generators.get(0).generateViewUpdates();
+             List<Mutation> mutations = new ArrayList<>(updates.size());
+             for (PartitionUpdate update : updates)
+                 mutations.add(new Mutation(update));
+             return mutations;
+         }
+ 
+         Map<DecoratedKey, Mutation> mutations = new HashMap<>();
+         for (ViewUpdateGenerator generator : generators)
+         {
+             for (PartitionUpdate update : generator.generateViewUpdates())
+             {
+                 DecoratedKey key = update.partitionKey();
+                 Mutation mutation = mutations.get(key);
+                 if (mutation == null)
+                 {
+                     mutation = new Mutation(baseTableMetadata.ksName, key);
+                     mutations.put(key, mutation);
+                 }
+                 mutation.add(update);
+             }
+         }
+         return mutations.values();
+     }
+ 
+     /**
+      * A simple helper that tracks for a given {@code UnfilteredRowIterator} what is the current deletion at any time of the
+      * iteration. It will be the currently open range tombstone deletion if there is one and the partition deletion otherwise.
+      */
+     private static class DeletionTracker
+     {
+         private final DeletionTime partitionDeletion;
+         private DeletionTime deletion;
+ 
+         public DeletionTracker(DeletionTime partitionDeletion)
+         {
+             this.partitionDeletion = partitionDeletion;
+         }
+ 
+         public void update(Unfiltered marker)
+         {
+             assert marker instanceof RangeTombstoneMarker;
+             RangeTombstoneMarker rtm = (RangeTombstoneMarker)marker;
+             this.deletion = rtm.isOpen(false)
+                           ? rtm.openDeletionTime(false)
+                           : null;
+         }
+ 
+         public DeletionTime currentDeletion()
+         {
+             return deletion == null ? partitionDeletion : deletion;
+         }
+     }
+ }

http://git-wip-us.apache.org/repos/asf/cassandra/blob/a87fd715/src/java/org/apache/cassandra/db/view/ViewBuilder.java
----------------------------------------------------------------------
diff --cc src/java/org/apache/cassandra/db/view/ViewBuilder.java
index 8944122,b2b409b..65e26e2
--- a/src/java/org/apache/cassandra/db/view/ViewBuilder.java
+++ b/src/java/org/apache/cassandra/db/view/ViewBuilder.java
@@@ -77,28 -76,22 +77,22 @@@ public class ViewBuilder extends Compac
          if (!selectQuery.selectsKey(key))
              return;
  
-         QueryPager pager = view.getSelectStatement().internalReadForView(key, FBUtilities.nowInSeconds()).getPager(null, Server.CURRENT_VERSION);
+         int nowInSec = FBUtilities.nowInSeconds();
+         SinglePartitionReadCommand command = view.getSelectStatement().internalReadForView(key, nowInSec);
  
-         while (!pager.isExhausted())
+         // We're rebuilding everything from what's on disk, so we read everything, consider that as new updates
+         // and pretend that there is nothing pre-existing.
+         UnfilteredRowIterator empty = UnfilteredRowIterators.noRowsIterator(baseCfs.metadata, key, Rows.EMPTY_STATIC_ROW, DeletionTime.LIVE, false);
+ 
+         Collection<Mutation> mutations;
 -        try (ReadOrderGroup orderGroup = command.startOrderGroup();
++        try (ReadExecutionController orderGroup = command.executionController();
+              UnfilteredRowIterator data = UnfilteredPartitionIterators.getOnlyElement(command.executeLocally(orderGroup), command))
          {
-            try (ReadExecutionController executionController = pager.executionController();
-                 PartitionIterator partitionIterator = pager.fetchPageInternal(128, executionController))
-            {
-                if (!partitionIterator.hasNext())
-                    return;
- 
-                try (RowIterator rowIterator = partitionIterator.next())
-                {
-                    FilteredPartition partition = FilteredPartition.create(rowIterator);
-                    TemporalRow.Set temporalRows = view.getTemporalRowSet(partition, null, true);
- 
-                    Collection<Mutation> mutations = view.createMutations(partition, temporalRows, true);
- 
-                    if (mutations != null)
-                        StorageProxy.mutateMV(key.getKey(), mutations, true, noBase);
-                }
-            }
+             mutations = baseCfs.keyspace.viewManager.forTable(baseCfs.metadata).generateViewUpdates(Collections.singleton(view), data, empty, nowInSec);
          }
+ 
+         if (!mutations.isEmpty())
+             StorageProxy.mutateMV(key.getKey(), mutations, true, noBase);
      }
  
      public void run()

http://git-wip-us.apache.org/repos/asf/cassandra/blob/a87fd715/src/java/org/apache/cassandra/db/view/ViewManager.java
----------------------------------------------------------------------
diff --cc src/java/org/apache/cassandra/db/view/ViewManager.java
index 6b8fe0d,fd04b97..bd73733
--- a/src/java/org/apache/cassandra/db/view/ViewManager.java
+++ b/src/java/org/apache/cassandra/db/view/ViewManager.java
@@@ -30,24 -31,24 +31,25 @@@ import org.apache.cassandra.config.CFMe
  import org.apache.cassandra.config.DatabaseDescriptor;
  import org.apache.cassandra.config.ViewDefinition;
  import org.apache.cassandra.db.*;
- import org.apache.cassandra.db.commitlog.ReplayPosition;
- import org.apache.cassandra.db.partitions.PartitionUpdate;
- import org.apache.cassandra.dht.Token;
+ import org.apache.cassandra.db.rows.*;
+ import org.apache.cassandra.db.partitions.*;
 -
 +import org.apache.cassandra.repair.SystemDistributedKeyspace;
- import org.apache.cassandra.service.StorageProxy;
 +import org.apache.cassandra.service.StorageService;
  
- import org.slf4j.Logger;
- import org.slf4j.LoggerFactory;
- 
  /**
   * Manages {@link View}'s for a single {@link ColumnFamilyStore}. All of the views for that table are created when this
   * manager is initialized.
   *
   * The main purposes of the manager are to provide a single location for updates to be vetted to see whether they update
 - * any views {@link ViewManager#updatesAffectView(Collection, boolean)}, provide locks to prevent multiple
 - * updates from creating incoherent updates in the view {@link ViewManager#acquireLockFor(ByteBuffer)}, and
 + * any views {@link #updatesAffectView(Collection, boolean)}, provide locks to prevent multiple
 + * updates from creating incoherent updates in the view {@link #acquireLockFor(int)}, and
   * to affect change on the view.
+  *
+  * TODO: I think we can get rid of that class. For addition/removal of view by names, we could move it Keyspace. And we
+  * not sure it's even worth keeping viewsByName as none of the related operation are performance sensitive so we could
+  * find the view by iterating over the CFStore.viewManager directly.
+  * For the lock, it could move to Keyspace too, but I don't remmenber why it has to be at the keyspace level and if it
+  * can be at the table level, maybe that's where it should be.
   */
  public class ViewManager
  {
@@@ -250,9 -148,8 +163,9 @@@
          if (view == null)
              return;
  
-         forTable(view.getDefinition().baseTableId).removeView(name);
+         forTable(view.getDefinition().baseTableMetadata()).removeByName(name);
          SystemKeyspace.setViewRemoved(keyspace.getName(), view.name);
 +        SystemDistributedKeyspace.setViewRemoved(keyspace.getName(), view.name);
      }
  
      public void buildAllViews()
@@@ -261,22 -158,23 +174,23 @@@
              view.build();
      }
  
-     public ForStore forTable(UUID baseId)
+     public TableViews forTable(CFMetaData metadata)
      {
-         ForStore forStore = viewManagersByStore.get(baseId);
-         if (forStore == null)
+         UUID baseId = metadata.cfId;
+         TableViews views = viewsByBaseTable.get(baseId);
+         if (views == null)
          {
-             forStore = new ForStore();
-             ForStore previous = viewManagersByStore.put(baseId, forStore);
+             views = new TableViews(metadata);
+             TableViews previous = viewsByBaseTable.putIfAbsent(baseId, views);
              if (previous != null)
-                 forStore = previous;
+                 views = previous;
          }
-         return forStore;
+         return views;
      }
  
 -    public static Lock acquireLockFor(ByteBuffer key)
 +    public static Lock acquireLockFor(int keyAndCfidHash)
      {
 -        Lock lock = LOCKS.get(key);
 +        Lock lock = LOCKS.get(keyAndCfidHash);
  
          if (lock.tryLock())
              return lock;

http://git-wip-us.apache.org/repos/asf/cassandra/blob/a87fd715/src/java/org/apache/cassandra/db/view/ViewUpdateGenerator.java
----------------------------------------------------------------------
diff --cc src/java/org/apache/cassandra/db/view/ViewUpdateGenerator.java
index 0000000,af025cb..4c6dbb7
mode 000000,100644..100644
--- a/src/java/org/apache/cassandra/db/view/ViewUpdateGenerator.java
+++ b/src/java/org/apache/cassandra/db/view/ViewUpdateGenerator.java
@@@ -1,0 -1,549 +1,549 @@@
+ /*
+  * Licensed to the Apache Software Foundation (ASF) under one
+  * or more contributor license agreements.  See the NOTICE file
+  * distributed with this work for additional information
+  * regarding copyright ownership.  The ASF licenses this file
+  * to you under the Apache License, Version 2.0 (the
+  * "License"); you may not use this file except in compliance
+  * with the License.  You may obtain a copy of the License at
+  *
+  *     http://www.apache.org/licenses/LICENSE-2.0
+  *
+  * Unless required by applicable law or agreed to in writing, software
+  * distributed under the License is distributed on an "AS IS" BASIS,
+  * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+  * See the License for the specific language governing permissions and
+  * limitations under the License.
+  */
+ package org.apache.cassandra.db.view;
+ 
+ import java.nio.ByteBuffer;
+ import java.util.*;
+ 
+ import com.google.common.collect.Iterators;
+ import com.google.common.collect.PeekingIterator;
+ 
+ import org.apache.cassandra.config.CFMetaData;
+ import org.apache.cassandra.config.ColumnDefinition;
+ import org.apache.cassandra.config.ViewDefinition;
+ import org.apache.cassandra.db.*;
+ import org.apache.cassandra.db.rows.*;
+ import org.apache.cassandra.db.partitions.*;
+ import org.apache.cassandra.db.marshal.AbstractType;
+ import org.apache.cassandra.db.marshal.CompositeType;
+ 
+ /**
+  * Creates the updates to apply to a view given the existing rows in the base
+  * table and the updates that we're applying to them (this handles updates
+  * on a single partition only).
+  *
+  * This class is used by passing the updates made to the base table to
+  * {@link #addBaseTableUpdate} and calling {@link #generateViewUpdates} once all updates have
+  * been handled to get the resulting view mutations.
+  */
+ public class ViewUpdateGenerator
+ {
+     private final View view;
+     private final int nowInSec;
+ 
+     private final CFMetaData baseMetadata;
+     private final DecoratedKey baseDecoratedKey;
+     private final ByteBuffer[] basePartitionKey;
+ 
+     private final CFMetaData viewMetadata;
+ 
+     private final Map<DecoratedKey, PartitionUpdate> updates = new HashMap<>();
+ 
+     // Reused internally to build a new entry
+     private final ByteBuffer[] currentViewEntryPartitionKey;
+     private final Row.Builder currentViewEntryBuilder;
+ 
+     /**
+      * The type of type update action to perform to the view for a given base table
+      * update.
+      */
+     private enum UpdateAction
+     {
+         NONE,            // There was no view entry and none should be added
+         NEW_ENTRY,       // There was no entry but there is one post-update
+         DELETE_OLD,      // There was an entry but there is nothing after update
+         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}.
+      *
+      * @param view the view for which this will be building updates for.
+      * @param basePartitionKey the partition key for the base table partition for which
+      * we'll handle updates for.
+      * @param nowInSec the current time in seconds. Used to decide if data are live or not
+      * and as base reference for new deletions.
+      */
+     public ViewUpdateGenerator(View view, DecoratedKey basePartitionKey, int nowInSec)
+     {
+         this.view = view;
+         this.nowInSec = nowInSec;
+ 
+         this.baseMetadata = view.getDefinition().baseTableMetadata();
+         this.baseDecoratedKey = basePartitionKey;
+         this.basePartitionKey = extractKeyComponents(basePartitionKey, baseMetadata.getKeyValidator());
+ 
+         this.viewMetadata = view.getDefinition().metadata;
+ 
+         this.currentViewEntryPartitionKey = new ByteBuffer[viewMetadata.partitionKeyColumns().size()];
+         this.currentViewEntryBuilder = BTreeRow.sortedBuilder();
+     }
+ 
+     private static ByteBuffer[] extractKeyComponents(DecoratedKey partitionKey, AbstractType<?> type)
+     {
+         return type instanceof CompositeType
+              ? ((CompositeType)type).split(partitionKey.getKey())
+              : new ByteBuffer[]{ partitionKey.getKey() };
+     }
+ 
+     /**
+      * Adds to this generator the updates to be made to the view given a base table row
+      * before and after an update.
+      *
+      * @param existingBaseRow the base table row as it is before an update.
+      * @param mergedBaseRow the base table row after the update is applied (note that
+      * this is not just the new update, but rather the resulting row).
+      */
+     public void addBaseTableUpdate(Row existingBaseRow, Row mergedBaseRow)
+     {
+         switch (updateAction(existingBaseRow, mergedBaseRow))
+         {
+             case NONE:
+                 return;
+             case NEW_ENTRY:
+                 createEntry(mergedBaseRow);
+                 return;
+             case DELETE_OLD:
+                 deleteOldEntry(existingBaseRow);
+                 return;
+             case UPDATE_EXISTING:
+                 updateEntry(existingBaseRow, mergedBaseRow);
+                 return;
+             case SWITCH_ENTRY:
+                 createEntry(mergedBaseRow);
+                 deleteOldEntry(existingBaseRow);
+                 return;
+         }
+     }
+ 
+     /**
+      * Returns the updates that needs to be done to the view given the base table updates
+      * passed to {@link #generateViewMutations}.
+      *
+      * @return the updates to do to the view.
+      */
+     public Collection<PartitionUpdate> generateViewUpdates()
+     {
+         return updates.values();
+     }
+ 
+     /**
+      * Compute which type of action needs to be performed to the view for a base table row
+      * before and after an update.
+      */
+     private UpdateAction updateAction(Row existingBaseRow, Row mergedBaseRow)
+     {
+         // Having existing empty is useful, it just means we'll insert a brand new entry for mergedBaseRow,
+         // but if we have no update at all, we shouldn't get there.
+         assert !mergedBaseRow.isEmpty();
+ 
+         // Note that none of the base PK columns will differ since we're intrinsically dealing
+         // with the same base row. So we have to check 3 things:
+         //   1) that the clustering doesn't have a null, which can happen for compact tables. If that's the case,
+         //      there is no corresponding entries.
+         //   2) if there is a column not part of the base PK in the view PK, whether it is changed by the update.
+         //   3) whether mergedBaseRow actually match the view SELECT filter
+ 
+         if (baseMetadata.isCompactTable())
+         {
+             Clustering clustering = mergedBaseRow.clustering();
+             for (int i = 0; i < clustering.size(); i++)
+             {
+                 if (clustering.get(i) == null)
+                     return UpdateAction.NONE;
+             }
+         }
+ 
+         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.
+ 
+             // Note that we allow existingBaseRow to be null and treat it as empty (see MultiViewUpdateBuilder.generateViewsMutations).
+             boolean existingHasLiveData = existingBaseRow != null && existingBaseRow.hasLiveData(nowInSec);
+             boolean mergedHasLiveData = mergedBaseRow.hasLiveData(nowInSec);
+             return existingHasLiveData
+                  ? (mergedHasLiveData ? UpdateAction.UPDATE_EXISTING : UpdateAction.DELETE_OLD)
+                  : (mergedHasLiveData ? UpdateAction.NEW_ENTRY : UpdateAction.NONE);
+         }
+ 
+         ColumnDefinition baseColumn = view.baseNonPKColumnsInViewPK.get(0);
+         assert !baseColumn.isComplex() : "A complex column couldn't be part of the view PK";
+         Cell before = existingBaseRow == null ? null : existingBaseRow.getCell(baseColumn);
+         Cell after = mergedBaseRow.getCell(baseColumn);
+ 
+         // If the update didn't modified this column, the cells will be the same object so it's worth checking
+         if (before == after)
+             return before == null ? UpdateAction.NONE : UpdateAction.UPDATE_EXISTING;
+ 
+         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
+              : UpdateAction.SWITCH_ENTRY;
+     }
+ 
+     private boolean matchesViewFilter(Row baseRow)
+     {
+         return view.matchesViewFilter(baseDecoratedKey, baseRow, nowInSec);
+     }
+ 
+     private boolean isLive(Cell cell)
+     {
+         return cell != null && cell.isLive(nowInSec);
+     }
+ 
+     /**
+      * Creates a view entry corresponding to the provided base row.
+      * <p>
+      * This method checks that the base row does match the view filter before applying it.
+      */
+     private void createEntry(Row baseRow)
+     {
+         // Before create a new entry, make sure it matches the view filter
+         if (!matchesViewFilter(baseRow))
+             return;
+ 
+         startNewUpdate(baseRow);
+         currentViewEntryBuilder.addPrimaryKeyLivenessInfo(computeLivenessInfoForEntry(baseRow));
+         currentViewEntryBuilder.addRowDeletion(baseRow.deletion());
+ 
+         for (ColumnData data : baseRow)
+         {
+             ColumnDefinition viewColumn = view.getViewColumn(data.column());
+             // If that base table column is not denormalized in the view, we had nothing to do.
+             // Alose, if it's part of the view PK it's already been taken into account in the clustering.
+             if (viewColumn == null || viewColumn.isPrimaryKeyColumn())
+                 continue;
+ 
+             addColumnData(viewColumn, data);
+         }
+ 
+         submitUpdate();
+     }
+ 
+     /**
+      * Creates the updates to apply to the existing view entry given the base table row before
+      * and after the update, assuming that the update hasn't changed to which view entry the
+      * row correspond (that is, we know the columns composing the view PK haven't changed).
+      * <p>
+      * This method checks that the base row (before and after) does match the view filter before
+      * applying anything.
+      */
+     private void updateEntry(Row existingBaseRow, Row mergedBaseRow)
+     {
+         // While we know existingBaseRow and mergedBaseRow are corresponding to the same view entry,
+         // they may not match the view filter.
+         if (!matchesViewFilter(existingBaseRow))
+         {
+             createEntry(mergedBaseRow);
+             return;
+         }
+         if (!matchesViewFilter(mergedBaseRow))
+         {
+             deleteOldEntryInternal(existingBaseRow);
+             return;
+         }
+ 
+         startNewUpdate(mergedBaseRow);
+ 
+         // In theory, it may be the PK liveness and row deletion hasn't been change by the update
+         // and we could condition the 2 additions below. In practice though, it's as fast (if not
+         // faster) to compute those info than to check if they have changed so we keep it simple.
+         currentViewEntryBuilder.addPrimaryKeyLivenessInfo(computeLivenessInfoForEntry(mergedBaseRow));
+         currentViewEntryBuilder.addRowDeletion(mergedBaseRow.deletion());
+ 
+         // 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
+         // mergedBaseRow (note that including more cells than we strictly should isn't a problem
+         // for correction, so even if the code change and pointer equality don't work anymore, it'll
+         // only a slightly inefficiency which we can fix then).
+         // Note: we could alternatively use Rows.diff() for this, but because it is a bit more generic
+         // than what we need here, it's also a bit less efficient (it allocates more in particular),
+         // and this might be called a lot of time for view updates. So, given that this is not a whole
+         // lot of code anyway, it's probably doing the diff manually.
+         PeekingIterator<ColumnData> existingIter = Iterators.peekingIterator(existingBaseRow.iterator());
+         for (ColumnData mergedData : mergedBaseRow)
+         {
+             ColumnDefinition baseColumn = mergedData.column();
+             ColumnDefinition viewColumn = view.getViewColumn(baseColumn);
+             // If that base table column is not denormalized in the view, we had nothing to do.
+             // Alose, if it's part of the view PK it's already been taken into account in the clustering.
+             if (viewColumn == null || viewColumn.isPrimaryKeyColumn())
+                 continue;
+ 
+             ColumnData existingData = null;
+             // Find if there is data for that column in the existing row
+             while (existingIter.hasNext())
+             {
+                 int cmp = baseColumn.compareTo(existingIter.peek().column());
+                 if (cmp < 0)
+                     break;
+ 
+                 ColumnData next = existingIter.next();
+                 if (cmp == 0)
+                 {
+                     existingData = next;
+                     break;
+                 }
+             }
+ 
+             if (existingData == null)
+             {
+                 addColumnData(viewColumn, mergedData);
+                 continue;
+             }
+ 
+             if (mergedData == existingData)
+                 continue;
+ 
+             if (baseColumn.isComplex())
+             {
+                 ComplexColumnData mergedComplexData = (ComplexColumnData)mergedData;
+                 ComplexColumnData existingComplexData = (ComplexColumnData)existingData;
+                 if (mergedComplexData.complexDeletion().supersedes(existingComplexData.complexDeletion()))
+                     currentViewEntryBuilder.addComplexDeletion(viewColumn, mergedComplexData.complexDeletion());
+ 
+                 PeekingIterator<Cell> existingCells = Iterators.peekingIterator(existingComplexData.iterator());
+                 for (Cell mergedCell : mergedComplexData)
+                 {
+                     Cell existingCell = null;
+                     // Find if there is corresponding cell in the existing row
+                     while (existingCells.hasNext())
+                     {
+                         int cmp = baseColumn.cellPathComparator().compare(mergedCell.path(), existingCells.peek().path());
+                         if (cmp > 0)
+                             break;
+ 
+                         Cell next = existingCells.next();
+                         if (cmp == 0)
+                         {
+                             existingCell = next;
+                             break;
+                         }
+                     }
+ 
+                     if (mergedCell != existingCell)
+                         addCell(viewColumn, mergedCell);
+                 }
+             }
+             else
+             {
+                 // Note that we've already eliminated the case where merged == existing
+                 addCell(viewColumn, (Cell)mergedData);
+             }
+         }
+ 
+         submitUpdate();
+     }
+ 
+     /**
+      * Deletes the view entry corresponding to the provided base row.
+      * <p>
+      * This method checks that the base row does match the view filter before bothering.
+      */
+     private void deleteOldEntry(Row existingBaseRow)
+     {
+         // 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);
+     }
+ 
+     private void deleteOldEntryInternal(Row existingBaseRow)
+     {
+         startNewUpdate(existingBaseRow);
+         DeletionTime dt = new DeletionTime(computeTimestampForEntryDeletion(existingBaseRow), nowInSec);
+         currentViewEntryBuilder.addRowDeletion(Row.Deletion.shadowable(dt));
+         submitUpdate();
+     }
+ 
+     /**
+      * Computes the partition key and clustering for a new view entry, and setup the internal
+      * row builder for the new row.
+      *
+      * This assumes that there is corresponding entry, i.e. no values for the partition key and
+      * clustering are null (since we have eliminated that case through updateAction).
+      */
+     private void startNewUpdate(Row baseRow)
+     {
+         ByteBuffer[] clusteringValues = new ByteBuffer[viewMetadata.clusteringColumns().size()];
+         for (ColumnDefinition viewColumn : viewMetadata.primaryKeyColumns())
+         {
+             ColumnDefinition baseColumn = view.getBaseColumn(viewColumn);
+             ByteBuffer value = getValueForPK(baseColumn, baseRow);
+             if (viewColumn.isPartitionKey())
+                 currentViewEntryPartitionKey[viewColumn.position()] = value;
+             else
+                 clusteringValues[viewColumn.position()] = value;
+         }
+ 
 -        currentViewEntryBuilder.newRow(new Clustering(clusteringValues));
++        currentViewEntryBuilder.newRow(Clustering.make(clusteringValues));
+     }
+ 
+     private LivenessInfo computeLivenessInfoForEntry(Row baseRow)
+     {
+         /*
+          * We need to compute both the timestamp and expiration.
+          *
+          * 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())
+         {
+             int ttl = baseLiveness.ttl();
+             int expirationTime = baseLiveness.localExpirationTime();
+             for (Cell cell : baseRow.cells())
+             {
+                 if (cell.ttl() > ttl)
+                 {
+                     ttl = cell.ttl();
+                     expirationTime = cell.localDeletionTime();
+                 }
+             }
+             return ttl == baseLiveness.ttl()
+                  ? baseLiveness
 -                 : LivenessInfo.create(baseLiveness.timestamp(), ttl, expirationTime);
++                 : LivenessInfo.withExpirationTime(baseLiveness.timestamp(), ttl, expirationTime);
+         }
+ 
+         ColumnDefinition baseColumn = view.baseNonPKColumnsInViewPK.get(0);
+         Cell cell = baseRow.getCell(baseColumn);
+         assert isLive(cell) : "We shouldn't have got there is 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.withExpirationTime(timestamp, cell.ttl(), cell.localDeletionTime());
+     }
+ 
+     private long computeTimestampForEntryDeletion(Row baseRow)
+     {
+         // 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)
+         {
+             if (!view.getDefinition().includes(data.column().name))
+                 continue;
+ 
+             timestamp = Math.max(timestamp, data.maxTimestamp());
+         }
+         return timestamp;
+     }
+ 
+     private void addColumnData(ColumnDefinition viewColumn, ColumnData baseTableData)
+     {
+         assert viewColumn.isComplex() == baseTableData.column().isComplex();
+         if (!viewColumn.isComplex())
+         {
+             addCell(viewColumn, (Cell)baseTableData);
+             return;
+         }
+ 
+         ComplexColumnData complexData = (ComplexColumnData)baseTableData;
+         currentViewEntryBuilder.addComplexDeletion(viewColumn, complexData.complexDeletion());
+         for (Cell cell : complexData)
+             addCell(viewColumn, cell);
+     }
+ 
+     private void addCell(ColumnDefinition viewColumn, Cell baseTableCell)
+     {
+         assert !viewColumn.isPrimaryKeyColumn();
+         currentViewEntryBuilder.addCell(baseTableCell.withUpdatedColumn(viewColumn));
+     }
+ 
+     /**
+      * Finish building the currently updated view entry and add it to the other built
+      * updates.
+      */
+     private void submitUpdate()
+     {
+         Row row = currentViewEntryBuilder.build();
+         // I'm not sure we can reach there is there is nothing is updated, but adding an empty row breaks things
+         // and it costs us nothing to be prudent here.
+         if (row.isEmpty())
+             return;
+ 
+         DecoratedKey partitionKey = makeCurrentPartitionKey();
+         PartitionUpdate update = updates.get(partitionKey);
+         if (update == null)
+         {
+             // We can't really know which columns of the view will be updated nor how many row will be updated for this key
+             // so we rely on hopefully sane defaults.
+             update = new PartitionUpdate(viewMetadata, partitionKey, viewMetadata.partitionColumns(), 4);
+             updates.put(partitionKey, update);
+         }
+         update.add(row);
+     }
+ 
+     private DecoratedKey makeCurrentPartitionKey()
+     {
+         ByteBuffer rawKey = viewMetadata.partitionKeyColumns().size() == 1
+                           ? currentViewEntryPartitionKey[0]
+                           : CompositeType.build(currentViewEntryPartitionKey);
+ 
+         return viewMetadata.decorateKey(rawKey);
+     }
+ 
+     private ByteBuffer getValueForPK(ColumnDefinition column, Row row)
+     {
+         switch (column.kind)
+         {
+             case PARTITION_KEY:
+                 return basePartitionKey[column.position()];
+             case CLUSTERING:
+                 return row.clustering().get(column.position());
+             default:
+                 // This shouldn't NPE as we shouldn't get there if the value can be null (or there is a bug in updateAction())
+                 return row.getCell(column).value();
+         }
+     }
+ }

http://git-wip-us.apache.org/repos/asf/cassandra/blob/a87fd715/test/unit/org/apache/cassandra/db/rows/RowsTest.java
----------------------------------------------------------------------
diff --cc test/unit/org/apache/cassandra/db/rows/RowsTest.java
index 00ab6ca,b47bea2..ba03478
--- a/test/unit/org/apache/cassandra/db/rows/RowsTest.java
+++ b/test/unit/org/apache/cassandra/db/rows/RowsTest.java
@@@ -235,11 -235,12 +235,12 @@@ public class RowsTes
          originalBuilder.addPrimaryKeyLivenessInfo(liveness);
          DeletionTime complexDeletion = new DeletionTime(ts-1, now);
          originalBuilder.addComplexDeletion(m, complexDeletion);
 -        List<Cell> expectedCells = Lists.newArrayList(BufferCell.live(kcvm, v, secondToTs(now), BB1),
 -                                                      BufferCell.live(kcvm, m, secondToTs(now), BB1, CellPath.create(BB1)),
 -                                                      BufferCell.live(kcvm, m, secondToTs(now), BB2, CellPath.create(BB2)));
 +        List<Cell> expectedCells = Lists.newArrayList(BufferCell.live(v, secondToTs(now), BB1),
 +                                                      BufferCell.live(m, secondToTs(now), BB1, CellPath.create(BB1)),
 +                                                      BufferCell.live(m, secondToTs(now), BB2, CellPath.create(BB2)));
          expectedCells.forEach(originalBuilder::addCell);
-         Row.Deletion rowDeletion = new Row.Deletion(new DeletionTime(ts, now), false);
+         // We need to use ts-1 so the deletion doesn't shadow what we've created
+         Row.Deletion rowDeletion = new Row.Deletion(new DeletionTime(ts-1, now), false);
          originalBuilder.addRowDeletion(rowDeletion);
  
          RowBuilder builder = new RowBuilder();
@@@ -263,11 -264,12 +264,12 @@@
          builder.addPrimaryKeyLivenessInfo(liveness);
          DeletionTime complexDeletion = new DeletionTime(ts-1, now);
          builder.addComplexDeletion(m, complexDeletion);
 -        List<Cell> expectedCells = Lists.newArrayList(BufferCell.live(kcvm, v, ts, BB1),
 -                                                      BufferCell.live(kcvm, m, ts, BB1, CellPath.create(BB1)),
 -                                                      BufferCell.live(kcvm, m, ts, BB2, CellPath.create(BB2)));
 +        List<Cell> expectedCells = Lists.newArrayList(BufferCell.live(v, ts, BB1),
 +                                                      BufferCell.live(m, ts, BB1, CellPath.create(BB1)),
 +                                                      BufferCell.live(m, ts, BB2, CellPath.create(BB2)));
          expectedCells.forEach(builder::addCell);
-         Row.Deletion rowDeletion = new Row.Deletion(new DeletionTime(ts, now), false);
+         // We need to use ts-1 so the deletion doesn't shadow what we've created
+         Row.Deletion rowDeletion = new Row.Deletion(new DeletionTime(ts-1, now), false);
          builder.addRowDeletion(rowDeletion);
  
          StatsCollector collector = new StatsCollector();


[4/9] cassandra git commit: Refactor MV code

Posted by sl...@apache.org.
Refactor MV code

patch by slebresne; reviewed by carlyeks for CASSANDRA-11475


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

Branch: refs/heads/cassandra-3.7
Commit: 86ba227477b9f8595eb610ecaf950cfbc29dd36b
Parents: c19066e
Author: Sylvain Lebresne <sy...@datastax.com>
Authored: Fri Mar 11 14:19:38 2016 +0100
Committer: Sylvain Lebresne <sy...@datastax.com>
Committed: Fri May 6 13:41:41 2016 +0200

----------------------------------------------------------------------
 CHANGES.txt                                     |   3 +
 .../org/apache/cassandra/config/CFMetaData.java |   6 +
 .../apache/cassandra/config/ViewDefinition.java |   1 -
 .../cql3/statements/CreateViewStatement.java    |   4 +-
 .../cql3/statements/SelectStatement.java        |  41 +-
 .../apache/cassandra/db/ColumnFamilyStore.java  |   6 +-
 src/java/org/apache/cassandra/db/Keyspace.java  |   2 +-
 .../db/SinglePartitionReadCommand.java          |  33 +
 src/java/org/apache/cassandra/db/Slices.java    |   7 +
 .../apache/cassandra/db/filter/RowFilter.java   |  24 +
 .../SingletonUnfilteredPartitionIterator.java   |   3 +-
 .../apache/cassandra/db/rows/AbstractCell.java  |   5 +
 .../org/apache/cassandra/db/rows/BTreeRow.java  |  34 +-
 .../apache/cassandra/db/rows/BufferCell.java    |   5 +
 src/java/org/apache/cassandra/db/rows/Cell.java |   2 +
 .../apache/cassandra/db/rows/ColumnData.java    |   2 +
 .../cassandra/db/rows/ComplexColumnData.java    |   8 +
 src/java/org/apache/cassandra/db/rows/Row.java  |  35 +-
 .../cassandra/db/rows/RowDiffListener.java      |   2 +-
 .../db/rows/UnfilteredRowIterators.java         |   2 +-
 .../apache/cassandra/db/view/TableViews.java    | 481 ++++++++++++++
 .../apache/cassandra/db/view/TemporalRow.java   | 610 ------------------
 src/java/org/apache/cassandra/db/view/View.java | 629 ++-----------------
 .../apache/cassandra/db/view/ViewBuilder.java   |  38 +-
 .../apache/cassandra/db/view/ViewManager.java   | 146 +----
 .../cassandra/db/view/ViewUpdateGenerator.java  | 549 ++++++++++++++++
 .../org/apache/cassandra/cql3/ViewTest.java     |  52 +-
 .../org/apache/cassandra/db/rows/RowsTest.java  |   6 +-
 28 files changed, 1399 insertions(+), 1337 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/cassandra/blob/86ba2274/CHANGES.txt
----------------------------------------------------------------------
diff --git a/CHANGES.txt b/CHANGES.txt
index 0679e11..3a49f6a 100644
--- a/CHANGES.txt
+++ b/CHANGES.txt
@@ -1,3 +1,6 @@
+3.0.7
+ * Refactor Materialized View code (CASSANDRA-11475)
+
 3.0.6
  * Disallow creating view with a static column (CASSANDRA-11602)
  * Reduce the amount of object allocations caused by the getFunctions methods (CASSANDRA-11593)

http://git-wip-us.apache.org/repos/asf/cassandra/blob/86ba2274/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 79cd779..e263697 100644
--- a/src/java/org/apache/cassandra/config/CFMetaData.java
+++ b/src/java/org/apache/cassandra/config/CFMetaData.java
@@ -31,6 +31,7 @@ import com.google.common.annotations.VisibleForTesting;
 import com.google.common.base.MoreObjects;
 import com.google.common.base.Objects;
 import com.google.common.collect.ImmutableSet;
+import com.google.common.collect.Iterables;
 import com.google.common.collect.Sets;
 import org.apache.commons.lang3.ArrayUtils;
 import org.apache.commons.lang3.builder.HashCodeBuilder;
@@ -612,6 +613,11 @@ public final class CFMetaData
         };
     }
 
+    public Iterable<ColumnDefinition> primaryKeyColumns()
+    {
+        return Iterables.concat(partitionKeyColumns, clusteringColumns);
+    }
+
     public List<ColumnDefinition> partitionKeyColumns()
     {
         return partitionKeyColumns;

http://git-wip-us.apache.org/repos/asf/cassandra/blob/86ba2274/src/java/org/apache/cassandra/config/ViewDefinition.java
----------------------------------------------------------------------
diff --git a/src/java/org/apache/cassandra/config/ViewDefinition.java b/src/java/org/apache/cassandra/config/ViewDefinition.java
index b29a8f9..5300f56 100644
--- a/src/java/org/apache/cassandra/config/ViewDefinition.java
+++ b/src/java/org/apache/cassandra/config/ViewDefinition.java
@@ -37,7 +37,6 @@ public class ViewDefinition
     public final UUID baseTableId;
     public final String baseTableName;
     public final boolean includeAllColumns;
-    // The order of partititon columns and clustering columns is important, so we cannot switch these two to sets
     public final CFMetaData metadata;
 
     public SelectStatement.RawStatement select;

http://git-wip-us.apache.org/repos/asf/cassandra/blob/86ba2274/src/java/org/apache/cassandra/cql3/statements/CreateViewStatement.java
----------------------------------------------------------------------
diff --git a/src/java/org/apache/cassandra/cql3/statements/CreateViewStatement.java b/src/java/org/apache/cassandra/cql3/statements/CreateViewStatement.java
index 45231b7..6446602 100644
--- a/src/java/org/apache/cassandra/cql3/statements/CreateViewStatement.java
+++ b/src/java/org/apache/cassandra/cql3/statements/CreateViewStatement.java
@@ -227,10 +227,10 @@ public class CreateViewStatement extends SchemaAlteringStatement
         // This is only used as an intermediate state; this is to catch whether multiple non-PK columns are used
         boolean hasNonPKColumn = false;
         for (ColumnIdentifier.Raw raw : partitionKeys)
-            hasNonPKColumn = getColumnIdentifier(cfm, basePrimaryKeyCols, hasNonPKColumn, raw, targetPartitionKeys, restrictions);
+            hasNonPKColumn |= getColumnIdentifier(cfm, basePrimaryKeyCols, hasNonPKColumn, raw, targetPartitionKeys, restrictions);
 
         for (ColumnIdentifier.Raw raw : clusteringKeys)
-            hasNonPKColumn = getColumnIdentifier(cfm, basePrimaryKeyCols, hasNonPKColumn, raw, targetClusteringColumns, restrictions);
+            hasNonPKColumn |= getColumnIdentifier(cfm, basePrimaryKeyCols, hasNonPKColumn, raw, targetClusteringColumns, restrictions);
 
         // We need to include all of the primary key columns from the base table in order to make sure that we do not
         // overwrite values in the view. We cannot support "collapsing" the base table into a smaller number of rows in

http://git-wip-us.apache.org/repos/asf/cassandra/blob/86ba2274/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 c3e13f4..0e33475 100644
--- a/src/java/org/apache/cassandra/cql3/statements/SelectStatement.java
+++ b/src/java/org/apache/cassandra/cql3/statements/SelectStatement.java
@@ -151,6 +151,15 @@ public class SelectStatement implements CQLStatement
         return builder.build();
     }
 
+    /**
+     * The columns to fetch internally for this SELECT statement (which can be more than the one selected by the
+     * user as it also include any restricted column in particular).
+     */
+    public ColumnFilter queriedColumns()
+    {
+        return queriedColumns;
+    }
+
     // Creates a simple select based on the given selection.
     // Note that the results select statement should not be used for actual queries, but only for processing already
     // queried data through processColumnFamily.
@@ -473,7 +482,29 @@ public class SelectStatement implements CQLStatement
     }
 
     /**
-     * Returns a read command that can be used internally to filter individual rows for materialized views.
+     * Returns the slices fetched by this SELECT, assuming an internal call (no bound values in particular).
+     * <p>
+     * Note that if the SELECT intrinsically selects rows by names, we convert them into equivalent slices for
+     * the purpose of this method. This is used for MVs to restrict what needs to be read when we want to read
+     * everything that could be affected by a given view (and so, if the view SELECT statement has restrictions
+     * on the clustering columns, we can restrict what we read).
+     */
+    public Slices clusteringIndexFilterAsSlices()
+    {
+        QueryOptions options = QueryOptions.forInternalCalls(Collections.emptyList());
+        ClusteringIndexFilter filter = makeClusteringIndexFilter(options);
+        if (filter instanceof ClusteringIndexSliceFilter)
+            return ((ClusteringIndexSliceFilter)filter).requestedSlices();
+
+        Slices.Builder builder = new Slices.Builder(cfm.comparator);
+        for (Clustering clustering: ((ClusteringIndexNamesFilter)filter).requestedRows())
+            builder.add(Slice.make(clustering));
+        return builder.build();
+    }
+
+    /**
+     * Returns a read command that can be used internally to query all the rows queried by this SELECT for a
+     * give key (used for materialized views).
      */
     public SinglePartitionReadCommand internalReadForView(DecoratedKey key, int nowInSec)
     {
@@ -483,6 +514,14 @@ public class SelectStatement implements CQLStatement
         return SinglePartitionReadCommand.create(cfm, nowInSec, queriedColumns, rowFilter, DataLimits.NONE, key, filter);
     }
 
+    /**
+     * The {@code RowFilter} for this SELECT, assuming an internal call (no bound values in particular).
+     */
+    public RowFilter rowFilterForInternalCalls()
+    {
+        return getRowFilter(QueryOptions.forInternalCalls(Collections.emptyList()));
+    }
+
     private ReadQuery getRangeCommand(QueryOptions options, DataLimits limit, int nowInSec) throws RequestValidationException
     {
         ClusteringIndexFilter clusteringIndexFilter = makeClusteringIndexFilter(options);

http://git-wip-us.apache.org/repos/asf/cassandra/blob/86ba2274/src/java/org/apache/cassandra/db/ColumnFamilyStore.java
----------------------------------------------------------------------
diff --git a/src/java/org/apache/cassandra/db/ColumnFamilyStore.java b/src/java/org/apache/cassandra/db/ColumnFamilyStore.java
index 919fed6..5b8de8f 100644
--- a/src/java/org/apache/cassandra/db/ColumnFamilyStore.java
+++ b/src/java/org/apache/cassandra/db/ColumnFamilyStore.java
@@ -48,7 +48,7 @@ import org.apache.cassandra.db.commitlog.ReplayPosition;
 import org.apache.cassandra.db.compaction.*;
 import org.apache.cassandra.db.filter.ClusteringIndexFilter;
 import org.apache.cassandra.db.filter.DataLimits;
-import org.apache.cassandra.db.view.ViewManager;
+import org.apache.cassandra.db.view.TableViews;
 import org.apache.cassandra.db.lifecycle.*;
 import org.apache.cassandra.db.partitions.CachedPartition;
 import org.apache.cassandra.db.partitions.PartitionUpdate;
@@ -198,7 +198,7 @@ public class ColumnFamilyStore implements ColumnFamilyStoreMBean
     private final AtomicInteger fileIndexGenerator = new AtomicInteger(0);
 
     public final SecondaryIndexManager indexManager;
-    public final ViewManager.ForStore viewManager;
+    public final TableViews viewManager;
 
     /* These are locally held copies to be changed from the config during runtime */
     private volatile DefaultValue<Integer> minCompactionThreshold;
@@ -373,7 +373,7 @@ public class ColumnFamilyStore implements ColumnFamilyStoreMBean
         maxCompactionThreshold = new DefaultValue<>(metadata.params.compaction.maxCompactionThreshold());
         crcCheckChance = new DefaultValue<>(metadata.params.crcCheckChance);
         indexManager = new SecondaryIndexManager(this);
-        viewManager = keyspace.viewManager.forTable(metadata.cfId);
+        viewManager = keyspace.viewManager.forTable(metadata);
         metric = new TableMetrics(this);
         fileIndexGenerator.set(generation);
         sampleLatencyNanos = DatabaseDescriptor.getReadRpcTimeout() / 2;

http://git-wip-us.apache.org/repos/asf/cassandra/blob/86ba2274/src/java/org/apache/cassandra/db/Keyspace.java
----------------------------------------------------------------------
diff --git a/src/java/org/apache/cassandra/db/Keyspace.java b/src/java/org/apache/cassandra/db/Keyspace.java
index 5783b41..273946e 100644
--- a/src/java/org/apache/cassandra/db/Keyspace.java
+++ b/src/java/org/apache/cassandra/db/Keyspace.java
@@ -484,7 +484,7 @@ public class Keyspace
                     try
                     {
                         Tracing.trace("Creating materialized view mutations from base table replica");
-                        viewManager.pushViewReplicaUpdates(upd, !isClReplay, baseComplete);
+                        viewManager.forTable(upd.metadata()).pushViewReplicaUpdates(upd, !isClReplay, baseComplete);
                     }
                     catch (Throwable t)
                     {

http://git-wip-us.apache.org/repos/asf/cassandra/blob/86ba2274/src/java/org/apache/cassandra/db/SinglePartitionReadCommand.java
----------------------------------------------------------------------
diff --git a/src/java/org/apache/cassandra/db/SinglePartitionReadCommand.java b/src/java/org/apache/cassandra/db/SinglePartitionReadCommand.java
index 14923b9..d5f2dc4 100644
--- a/src/java/org/apache/cassandra/db/SinglePartitionReadCommand.java
+++ b/src/java/org/apache/cassandra/db/SinglePartitionReadCommand.java
@@ -229,6 +229,39 @@ public class SinglePartitionReadCommand extends ReadCommand
         return create(metadata, nowInSec, metadata.decorateKey(key), slices);
     }
 
+    /**
+     * Creates a new single partition name command for the provided rows.
+     *
+     * @param metadata the table to query.
+     * @param nowInSec the time in seconds to use are "now" for this query.
+     * @param key the partition key for the partition to query.
+     * @param names the clustering for the rows to query.
+     *
+     * @return a newly created read command that queries the {@code names} in {@code key}. The returned query will
+     * query every columns (without limit or row filtering) and be in forward order.
+     */
+    public static SinglePartitionReadCommand create(CFMetaData metadata, int nowInSec, DecoratedKey key, NavigableSet<Clustering> names)
+    {
+        ClusteringIndexNamesFilter filter = new ClusteringIndexNamesFilter(names, false);
+        return SinglePartitionReadCommand.create(metadata, nowInSec, ColumnFilter.all(metadata), RowFilter.NONE, DataLimits.NONE, key, filter);
+    }
+
+    /**
+     * Creates a new single partition name command for the provided row.
+     *
+     * @param metadata the table to query.
+     * @param nowInSec the time in seconds to use are "now" for this query.
+     * @param key the partition key for the partition to query.
+     * @param name the clustering for the row to query.
+     *
+     * @return a newly created read command that queries {@code name} in {@code key}. The returned query will
+     * query every columns (without limit or row filtering).
+     */
+    public static SinglePartitionReadCommand create(CFMetaData metadata, int nowInSec, DecoratedKey key, Clustering name)
+    {
+        return create(metadata, nowInSec, key, FBUtilities.singleton(name, metadata.comparator));
+    }
+
     public SinglePartitionReadCommand copy()
     {
         return new SinglePartitionReadCommand(isDigestQuery(), digestVersion(), isForThrift(), metadata(), nowInSec(), columnFilter(), rowFilter(), limits(), partitionKey(), clusteringIndexFilter());

http://git-wip-us.apache.org/repos/asf/cassandra/blob/86ba2274/src/java/org/apache/cassandra/db/Slices.java
----------------------------------------------------------------------
diff --git a/src/java/org/apache/cassandra/db/Slices.java b/src/java/org/apache/cassandra/db/Slices.java
index 8fa9337..bb354a1 100644
--- a/src/java/org/apache/cassandra/db/Slices.java
+++ b/src/java/org/apache/cassandra/db/Slices.java
@@ -210,6 +210,13 @@ public abstract class Slices implements Iterable<Slice>
             return this;
         }
 
+        public Builder addAll(Slices slices)
+        {
+            for (Slice slice : slices)
+                add(slice);
+            return this;
+        }
+
         public int size()
         {
             return slices.size();

http://git-wip-us.apache.org/repos/asf/cassandra/blob/86ba2274/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 8060f23..11cfb87 100644
--- a/src/java/org/apache/cassandra/db/filter/RowFilter.java
+++ b/src/java/org/apache/cassandra/db/filter/RowFilter.java
@@ -122,6 +122,30 @@ public abstract class RowFilter implements Iterable<RowFilter.Expression>
     public abstract UnfilteredPartitionIterator filter(UnfilteredPartitionIterator iter, int nowInSec);
 
     /**
+     * Whether the provided row in the provided partition satisfies this filter.
+     *
+     * @param metadata the table metadata.
+     * @param partitionKey the partition key for partition to test.
+     * @param row the row to test.
+     * @param nowInSec the current time in seconds (to know what is live and what isn't).
+     * @return {@code true} if {@code row} in partition {@code partitionKey} satisfies this row filter.
+     */
+    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);
+        if (purged == null)
+            return expressions.isEmpty();
+
+        for (Expression e : expressions)
+        {
+            if (!e.isSatisfiedBy(metadata, partitionKey, purged))
+                return false;
+        }
+        return true;
+    }
+
+    /**
      * Returns true if all of the expressions within this filter that apply to the partition key are satisfied by
      * the given key, false otherwise.
      */

http://git-wip-us.apache.org/repos/asf/cassandra/blob/86ba2274/src/java/org/apache/cassandra/db/partitions/SingletonUnfilteredPartitionIterator.java
----------------------------------------------------------------------
diff --git a/src/java/org/apache/cassandra/db/partitions/SingletonUnfilteredPartitionIterator.java b/src/java/org/apache/cassandra/db/partitions/SingletonUnfilteredPartitionIterator.java
index bfa6690..1f966db 100644
--- a/src/java/org/apache/cassandra/db/partitions/SingletonUnfilteredPartitionIterator.java
+++ b/src/java/org/apache/cassandra/db/partitions/SingletonUnfilteredPartitionIterator.java
@@ -65,6 +65,7 @@ public class SingletonUnfilteredPartitionIterator implements UnfilteredPartition
 
     public void close()
     {
-        iter.close();
+        if (!returned)
+            iter.close();
     }
 }

http://git-wip-us.apache.org/repos/asf/cassandra/blob/86ba2274/src/java/org/apache/cassandra/db/rows/AbstractCell.java
----------------------------------------------------------------------
diff --git a/src/java/org/apache/cassandra/db/rows/AbstractCell.java b/src/java/org/apache/cassandra/db/rows/AbstractCell.java
index 00fc286..7e93c2e 100644
--- a/src/java/org/apache/cassandra/db/rows/AbstractCell.java
+++ b/src/java/org/apache/cassandra/db/rows/AbstractCell.java
@@ -74,6 +74,11 @@ public abstract class AbstractCell extends Cell
             column().validateCellPath(path());
     }
 
+    public long maxTimestamp()
+    {
+        return timestamp();
+    }
+
     @Override
     public boolean equals(Object other)
     {

http://git-wip-us.apache.org/repos/asf/cassandra/blob/86ba2274/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 e8667e0..ea1d9e0 100644
--- a/src/java/org/apache/cassandra/db/rows/BTreeRow.java
+++ b/src/java/org/apache/cassandra/db/rows/BTreeRow.java
@@ -322,6 +322,18 @@ public class BTreeRow extends AbstractRow
         return transformAndFilter(newInfo, newDeletion, (cd) -> cd.updateAllTimestamp(newTimestamp));
     }
 
+    public Row withRowDeletion(DeletionTime newDeletion)
+    {
+        // Note that:
+        //  - it is a contract with the caller that the new deletion shouldn't shadow anything in
+        //    the row, and so in particular it can't shadow the row deletion. So if there is a
+        //    already a row deletion we have nothing to do.
+        //  - we set the minLocalDeletionTime to MIN_VALUE because we know the deletion is live
+        return newDeletion.isLive() || !deletion.isLive()
+             ? this
+             : new BTreeRow(clustering, primaryKeyLivenessInfo, Deletion.regular(newDeletion), btree, Integer.MIN_VALUE);
+    }
+
     public Row purge(DeletionPurger purger, int nowInSec)
     {
         if (!hasDeletion(nowInSec))
@@ -566,6 +578,17 @@ public class BTreeRow extends AbstractRow
                 }
 
                 List<Object> buildFrom = Arrays.asList(cells).subList(lb, ub);
+                if (deletion != DeletionTime.LIVE)
+                {
+                    // Make sure we don't include any shadowed cells
+                    List<Object> filtered = new ArrayList<>(buildFrom.size());
+                    for (Object c : buildFrom)
+                    {
+                        if (((Cell)c).timestamp() >= deletion.markedForDeleteAt())
+                            filtered.add(c);
+                    }
+                    buildFrom = filtered;
+                }
                 Object[] btree = BTree.build(buildFrom, UpdateFunction.noOp());
                 return new ComplexColumnData(column, btree, deletion);
             }
@@ -621,17 +644,26 @@ public class BTreeRow extends AbstractRow
 
         public void addPrimaryKeyLivenessInfo(LivenessInfo info)
         {
-            this.primaryKeyLivenessInfo = info;
+            // The check is only required for unsorted builders, but it's worth the extra safety to have it unconditional
+            if (!deletion.deletes(info))
+                this.primaryKeyLivenessInfo = info;
         }
 
         public void addRowDeletion(Deletion deletion)
         {
             this.deletion = deletion;
+            // The check is only required for unsorted builders, but it's worth the extra safety to have it unconditional
+            if (deletion.deletes(primaryKeyLivenessInfo))
+                this.primaryKeyLivenessInfo = LivenessInfo.EMPTY;
         }
 
         public void addCell(Cell cell)
         {
             assert cell.column().isStatic() == (clustering == Clustering.STATIC_CLUSTERING) : "Column is " + cell.column() + ", clustering = " + clustering;
+            // In practice, only unsorted builder have to deal with shadowed cells, but it doesn't cost us much to deal with it unconditionally in this case
+            if (deletion.deletes(cell))
+                return;
+
             cells.add(cell);
             hasComplex |= cell.column.isComplex();
         }

http://git-wip-us.apache.org/repos/asf/cassandra/blob/86ba2274/src/java/org/apache/cassandra/db/rows/BufferCell.java
----------------------------------------------------------------------
diff --git a/src/java/org/apache/cassandra/db/rows/BufferCell.java b/src/java/org/apache/cassandra/db/rows/BufferCell.java
index 8912f59..0a2c528 100644
--- a/src/java/org/apache/cassandra/db/rows/BufferCell.java
+++ b/src/java/org/apache/cassandra/db/rows/BufferCell.java
@@ -133,6 +133,11 @@ public class BufferCell extends AbstractCell
         return path;
     }
 
+    public Cell withUpdatedColumn(ColumnDefinition newColumn)
+    {
+        return new BufferCell(newColumn, timestamp, ttl, localDeletionTime, value, path);
+    }
+
     public Cell withUpdatedValue(ByteBuffer newValue)
     {
         return new BufferCell(column, timestamp, ttl, localDeletionTime, newValue, path);

http://git-wip-us.apache.org/repos/asf/cassandra/blob/86ba2274/src/java/org/apache/cassandra/db/rows/Cell.java
----------------------------------------------------------------------
diff --git a/src/java/org/apache/cassandra/db/rows/Cell.java b/src/java/org/apache/cassandra/db/rows/Cell.java
index 73d9e44..b10ce06 100644
--- a/src/java/org/apache/cassandra/db/rows/Cell.java
+++ b/src/java/org/apache/cassandra/db/rows/Cell.java
@@ -129,6 +129,8 @@ public abstract class Cell extends ColumnData
      */
     public abstract CellPath path();
 
+    public abstract Cell withUpdatedColumn(ColumnDefinition newColumn);
+
     public abstract Cell withUpdatedValue(ByteBuffer newValue);
 
     public abstract Cell copy(AbstractAllocator allocator);

http://git-wip-us.apache.org/repos/asf/cassandra/blob/86ba2274/src/java/org/apache/cassandra/db/rows/ColumnData.java
----------------------------------------------------------------------
diff --git a/src/java/org/apache/cassandra/db/rows/ColumnData.java b/src/java/org/apache/cassandra/db/rows/ColumnData.java
index 84763e5..933da6a 100644
--- a/src/java/org/apache/cassandra/db/rows/ColumnData.java
+++ b/src/java/org/apache/cassandra/db/rows/ColumnData.java
@@ -82,4 +82,6 @@ public abstract class ColumnData
     public abstract ColumnData markCounterLocalToBeCleared();
 
     public abstract ColumnData purge(DeletionPurger purger, int nowInSec);
+
+    public abstract long maxTimestamp();
 }

http://git-wip-us.apache.org/repos/asf/cassandra/blob/86ba2274/src/java/org/apache/cassandra/db/rows/ComplexColumnData.java
----------------------------------------------------------------------
diff --git a/src/java/org/apache/cassandra/db/rows/ComplexColumnData.java b/src/java/org/apache/cassandra/db/rows/ComplexColumnData.java
index fab529b..d67d079 100644
--- a/src/java/org/apache/cassandra/db/rows/ComplexColumnData.java
+++ b/src/java/org/apache/cassandra/db/rows/ComplexColumnData.java
@@ -184,6 +184,14 @@ public class ComplexColumnData extends ColumnData implements Iterable<Cell>
         return transformAndFilter(newDeletion, (cell) -> (Cell) cell.updateAllTimestamp(newTimestamp));
     }
 
+    public long maxTimestamp()
+    {
+        long timestamp = complexDeletion.markedForDeleteAt();
+        for (Cell cell : this)
+            timestamp = Math.max(timestamp, cell.timestamp());
+        return timestamp;
+    }
+
     // This is the partner in crime of ArrayBackedRow.setValue. The exact warning apply. The short
     // version is: "don't use that method".
     void setValue(CellPath path, ByteBuffer value)

http://git-wip-us.apache.org/repos/asf/cassandra/blob/86ba2274/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 5f79a66..82c07a7 100644
--- a/src/java/org/apache/cassandra/db/rows/Row.java
+++ b/src/java/org/apache/cassandra/db/rows/Row.java
@@ -215,6 +215,18 @@ public interface Row extends Unfiltered, Collection<ColumnData>
      */
     public Row updateAllTimestamp(long newTimestamp);
 
+    /**
+     * Returns a copy of this row with the new deletion as row deletion if it is more recent
+     * than the current row deletion.
+     * <p>
+     * WARNING: this method <b>does not</b> check that nothing in the row is shadowed by the provided
+     * deletion and if that is the case, the created row will be <b>invalid</b>. It is thus up to the
+     * caller to verify that this is not the case and the only reasonable use case of this is probably
+     * when the row and the deletion comes from the same {@code UnfilteredRowIterator} since that gives
+     * use this guarantee.
+     */
+    public Row withRowDeletion(DeletionTime deletion);
+
     public int dataSize();
 
     public long unsharedHeapSizeExcludingData();
@@ -227,12 +239,15 @@ public interface Row extends Unfiltered, Collection<ColumnData>
      * A row deletion mostly consists of the time of said deletion, but there is 2 variants: shadowable
      * and regular row deletion.
      * <p>
-     * A shadowable row deletion only exists if the row timestamp ({@code primaryKeyLivenessInfo().timestamp()})
-     * is lower than the deletion timestamp. That is, if a row has a shadowable deletion with timestamp A and an update is made
-     * to that row with a timestamp B such that B > A, then the shadowable deletion is 'shadowed' by that update. A concrete
-     * consequence is that if said update has cells with timestamp lower than A, then those cells are preserved
-     * (since the deletion is removed), and this contrarily to a normal (regular) deletion where the deletion is preserved
-     * and such cells are removed.
+     * A shadowable row deletion only exists if the row has no timestamp. In other words, the deletion is only
+     * valid as long as no newer insert is done (thus setting a row timestap; note that if the row timestamp set
+     * is lower than the deletion, it is shadowed (and thus ignored) as usual).
+     * <p>
+     * That is, if a row has a shadowable deletion with timestamp A and an update is madeto that row with a
+     * timestamp B such that B > A (and that update sets the row timestamp), then the shadowable deletion is 'shadowed'
+     * by that update. A concrete consequence is that if said update has cells with timestamp lower than A, then those
+     * cells are preserved(since the deletion is removed), and this contrarily to a normal (regular) deletion where the
+     * deletion is preserved and such cells are removed.
      * <p>
      * Currently, the only use of shadowable row deletions is Materialized Views, see CASSANDRA-10261.
      */
@@ -312,6 +327,11 @@ public interface Row extends Unfiltered, Collection<ColumnData>
             return time.deletes(info);
         }
 
+        public boolean deletes(Cell cell)
+        {
+            return time.deletes(cell);
+        }
+
         public void digest(MessageDigest digest)
         {
             time.digest(digest);
@@ -361,6 +381,9 @@ public interface Row extends Unfiltered, Collection<ColumnData>
      *      any column before {@code c} and before any call for any column after {@code c}.
      *   5) Calls to {@link #addCell} are further done in strictly increasing cell order (the one defined by
      *      {@link Cell#comparator}. That is, for a give column, cells are passed in {@code CellPath} order.
+     *   6) No shadowed data should be added. Concretely, this means that if a a row deletion is added, it doesn't
+     *      deletes the row timestamp or any cell added later, and similarly no cell added is deleted by the complex
+     *      deletion of the column this is a cell of.
      *
      * An unsorted builder will not expect those last rules however: {@link #addCell} and {@link #addComplexDeletion}
      * can be done in any order. And in particular unsorted builder allows multiple calls for the same column/cell. In

http://git-wip-us.apache.org/repos/asf/cassandra/blob/86ba2274/src/java/org/apache/cassandra/db/rows/RowDiffListener.java
----------------------------------------------------------------------
diff --git a/src/java/org/apache/cassandra/db/rows/RowDiffListener.java b/src/java/org/apache/cassandra/db/rows/RowDiffListener.java
index f209bfc..ec848a0 100644
--- a/src/java/org/apache/cassandra/db/rows/RowDiffListener.java
+++ b/src/java/org/apache/cassandra/db/rows/RowDiffListener.java
@@ -71,7 +71,7 @@ public interface RowDiffListener
      * @param i the input row from which {@code original} is from.
      * @param clustering the clustering for the row that is merged.
      * @param merged the cell of the merged row. Will be {@code null} if input {@code i} had a cell but that cell is no present
-     * in the mergd result (it has been deleted/shadowed).
+     * in the merged result (it has been deleted/shadowed).
      * @param original the cell of input {@code i}. May be {@code null} if input {@code i} had cell corresponding to {@code merged}.
      */
     public void onCell(int i, Clustering clustering, Cell merged, Cell original);

http://git-wip-us.apache.org/repos/asf/cassandra/blob/86ba2274/src/java/org/apache/cassandra/db/rows/UnfilteredRowIterators.java
----------------------------------------------------------------------
diff --git a/src/java/org/apache/cassandra/db/rows/UnfilteredRowIterators.java b/src/java/org/apache/cassandra/db/rows/UnfilteredRowIterators.java
index 9416896..ce5fffe 100644
--- a/src/java/org/apache/cassandra/db/rows/UnfilteredRowIterators.java
+++ b/src/java/org/apache/cassandra/db/rows/UnfilteredRowIterators.java
@@ -92,7 +92,7 @@ public abstract class UnfilteredRowIterators
     }
 
     /**
-     * Returns an empty atom iterator for a given partition.
+     * Returns an empty unfiltered iterator for a given partition.
      */
     public static UnfilteredRowIterator noRowsIterator(final CFMetaData cfm, final DecoratedKey partitionKey, final Row staticRow, final DeletionTime partitionDeletion, final boolean isReverseOrder)
     {

http://git-wip-us.apache.org/repos/asf/cassandra/blob/86ba2274/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
new file mode 100644
index 0000000..893bdd5
--- /dev/null
+++ b/src/java/org/apache/cassandra/db/view/TableViews.java
@@ -0,0 +1,481 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.cassandra.db.view;
+
+import java.util.*;
+import java.util.concurrent.CopyOnWriteArrayList;
+import java.util.concurrent.TimeUnit;
+import java.util.concurrent.atomic.AtomicLong;
+
+import com.google.common.collect.Iterables;
+import com.google.common.collect.Iterators;
+import com.google.common.collect.PeekingIterator;
+
+import org.apache.cassandra.config.CFMetaData;
+import org.apache.cassandra.db.*;
+import org.apache.cassandra.db.commitlog.ReplayPosition;
+import org.apache.cassandra.db.filter.*;
+import org.apache.cassandra.db.rows.*;
+import org.apache.cassandra.db.partitions.*;
+import org.apache.cassandra.dht.Token;
+import org.apache.cassandra.service.StorageProxy;
+import org.apache.cassandra.utils.FBUtilities;
+import org.apache.cassandra.utils.btree.BTreeSet;
+
+
+/**
+ * Groups all the views for a given table.
+ */
+public class TableViews extends AbstractCollection<View>
+{
+    private final CFMetaData baseTableMetadata;
+
+    // We need this to be thread-safe, but the number of times this is changed (when a view is created in the keyspace)
+    // massively exceeds the number of time it's read (for every mutation on the keyspace), so a copy-on-write list is the best option.
+    private final List<View> views = new CopyOnWriteArrayList();
+
+    public TableViews(CFMetaData baseTableMetadata)
+    {
+        this.baseTableMetadata = baseTableMetadata;
+    }
+
+    public int size()
+    {
+        return views.size();
+    }
+
+    public Iterator<View> iterator()
+    {
+        return views.iterator();
+    }
+
+    public boolean contains(String viewName)
+    {
+        return Iterables.any(views, view -> view.name.equals(viewName));
+    }
+
+    public boolean add(View view)
+    {
+        // We should have validated that there is no existing view with this name at this point
+        assert !contains(view.name);
+        return views.add(view);
+    }
+
+    public Iterable<ColumnFamilyStore> allViewsCfs()
+    {
+        Keyspace keyspace = Keyspace.open(baseTableMetadata.ksName);
+        return Iterables.transform(views, view -> keyspace.getColumnFamilyStore(view.getDefinition().viewName));
+    }
+
+    public void forceBlockingFlush()
+    {
+        for (ColumnFamilyStore viewCfs : allViewsCfs())
+            viewCfs.forceBlockingFlush();
+    }
+
+    public void dumpMemtables()
+    {
+        for (ColumnFamilyStore viewCfs : allViewsCfs())
+            viewCfs.dumpMemtable();
+    }
+
+    public void truncateBlocking(long truncatedAt)
+    {
+        for (ColumnFamilyStore viewCfs : allViewsCfs())
+        {
+            ReplayPosition replayAfter = viewCfs.discardSSTables(truncatedAt);
+            SystemKeyspace.saveTruncationRecord(viewCfs, truncatedAt, replayAfter);
+        }
+    }
+
+    public void removeByName(String viewName)
+    {
+        views.removeIf(v -> v.name.equals(viewName));
+    }
+
+    /**
+     * Calculates and pushes updates to the views replicas. The replicas are determined by
+     * {@link ViewUtils#getViewNaturalEndpoint(String, Token, Token)}.
+     *
+     * @param update an update on the base table represented by this object.
+     * @param writeCommitLog whether we should write the commit log for the view updates.
+     * @param baseComplete time from epoch in ms that the local base mutation was (or will be) completed
+     */
+    public void pushViewReplicaUpdates(PartitionUpdate update, boolean writeCommitLog, AtomicLong baseComplete)
+    {
+        assert update.metadata().cfId.equals(baseTableMetadata.cfId);
+
+        Collection<View> views = updatedViews(update);
+        if (views.isEmpty())
+            return;
+
+        // Read modified rows
+        int nowInSec = FBUtilities.nowInSeconds();
+        SinglePartitionReadCommand command = readExistingRowsCommand(update, views, nowInSec);
+        if (command == null)
+            return;
+
+        ColumnFamilyStore cfs = Keyspace.openAndGetStore(update.metadata());
+        long start = System.nanoTime();
+        Collection<Mutation> mutations;
+        try (ReadOrderGroup orderGroup = command.startOrderGroup();
+             UnfilteredRowIterator existings = UnfilteredPartitionIterators.getOnlyElement(command.executeLocally(orderGroup), command);
+             UnfilteredRowIterator updates = update.unfilteredIterator())
+        {
+            mutations = generateViewUpdates(views, updates, existings, nowInSec);
+        }
+        Keyspace.openAndGetStore(update.metadata()).metric.viewReadTime.update(System.nanoTime() - start, TimeUnit.NANOSECONDS);
+
+        if (!mutations.isEmpty())
+            StorageProxy.mutateMV(update.partitionKey().getKey(), mutations, writeCommitLog, baseComplete);
+    }
+
+    /**
+     * Given some updates on the base table of this object and the existing values for the rows affected by that update, generates the
+     * mutation to be applied to the provided views.
+     *
+     * @param views the views potentially affected by {@code updates}.
+     * @param updates the base table updates being applied.
+     * @param existings the existing values for the rows affected by {@code updates}. This is used to decide if a view is
+     * obsoleted by the update and should be removed, gather the values for columns that may not be part of the update if
+     * a new view entry needs to be created, and compute the minimal updates to be applied if the view entry isn't changed
+     * 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.
+     * @return the mutations to apply to the {@code views}. This can be empty.
+     */
+    public Collection<Mutation> generateViewUpdates(Collection<View> views, UnfilteredRowIterator updates, UnfilteredRowIterator existings, int nowInSec)
+    {
+        assert updates.metadata().cfId.equals(baseTableMetadata.cfId);
+
+        List<ViewUpdateGenerator> generators = new ArrayList<>(views.size());
+        for (View view : views)
+            generators.add(new ViewUpdateGenerator(view, updates.partitionKey(), nowInSec));
+
+        DeletionTracker existingsDeletion = new DeletionTracker(existings.partitionLevelDeletion());
+        DeletionTracker updatesDeletion = new DeletionTracker(updates.partitionLevelDeletion());
+
+        /*
+         * We iterate through the updates and the existing rows in parallel. This allows us to know the consequence
+         * on the view of each update.
+         */
+        PeekingIterator<Unfiltered> existingsIter = Iterators.peekingIterator(existings);
+        PeekingIterator<Unfiltered> updatesIter = Iterators.peekingIterator(updates);
+
+        while (existingsIter.hasNext() && updatesIter.hasNext())
+        {
+            Unfiltered existing = existingsIter.peek();
+            Unfiltered update = updatesIter.peek();
+
+            Row existingRow;
+            Row updateRow;
+            int cmp = baseTableMetadata.comparator.compare(update, existing);
+            if (cmp < 0)
+            {
+                // We have an update where there was nothing before
+                if (update.isRangeTombstoneMarker())
+                {
+                    updatesDeletion.update(updatesIter.next());
+                    continue;
+                }
+
+                updateRow = ((Row)updatesIter.next()).withRowDeletion(updatesDeletion.currentDeletion());
+                existingRow = emptyRow(updateRow.clustering(), existingsDeletion.currentDeletion());
+            }
+            else if (cmp > 0)
+            {
+                // We have something existing but no update (which will happen either because it's a range tombstone marker in
+                // existing, or because we've fetched the existing row due to some partition/range deletion in the updates)
+                if (existing.isRangeTombstoneMarker())
+                {
+                    existingsDeletion.update(existingsIter.next());
+                    continue;
+                }
+
+                existingRow = ((Row)existingsIter.next()).withRowDeletion(existingsDeletion.currentDeletion());
+                updateRow = emptyRow(existingRow.clustering(), updatesDeletion.currentDeletion());
+
+                // The way we build the read command used for existing rows, we should always have updatesDeletion.currentDeletion()
+                // that is not live, since we wouldn't have read the existing row otherwise. And we could assert that, but if we ever
+                // change the read method so that it can slightly over-read in some case, that would be an easily avoiding bug lurking,
+                // so we just handle the case.
+                if (updateRow == null)
+                    continue;
+            }
+            else
+            {
+                // We're updating a row that had pre-existing data
+                if (update.isRangeTombstoneMarker())
+                {
+                    assert existing.isRangeTombstoneMarker();
+                    updatesDeletion.update(updatesIter.next());
+                    existingsDeletion.update(existingsIter.next());
+                    continue;
+                }
+
+                assert !existing.isRangeTombstoneMarker();
+                existingRow = ((Row)existingsIter.next()).withRowDeletion(existingsDeletion.currentDeletion());
+                updateRow = ((Row)updatesIter.next()).withRowDeletion(updatesDeletion.currentDeletion());
+            }
+
+            addToViewUpdateGenerators(existingRow, updateRow, generators, nowInSec);
+        }
+
+        // We only care about more existing rows if the update deletion isn't live, i.e. if we had a partition deletion
+        if (!updatesDeletion.currentDeletion().isLive())
+        {
+            while (existingsIter.hasNext())
+            {
+                Unfiltered existing = existingsIter.next();
+                // If it's a range tombstone, we don't care, we're only looking for existing entry that gets deleted by
+                // the new partition deletion
+                if (existing.isRangeTombstoneMarker())
+                    continue;
+
+                Row existingRow = (Row)existing;
+                addToViewUpdateGenerators(existingRow, emptyRow(existingRow.clustering(), updatesDeletion.currentDeletion()), generators, nowInSec);
+            }
+        }
+        while (updatesIter.hasNext())
+        {
+            Unfiltered update = updatesIter.next();
+            // If it's a range tombstone, it removes nothing pre-exisiting, so we can ignore it for view updates
+            if (update.isRangeTombstoneMarker())
+                continue;
+
+            Row updateRow = (Row)update;
+            addToViewUpdateGenerators(emptyRow(updateRow.clustering(), DeletionTime.LIVE), updateRow, generators, nowInSec);
+        }
+
+        return buildMutations(baseTableMetadata, generators);
+    }
+
+    /**
+     * Return the views that are potentially updated by the provided updates.
+     *
+     * @param updates the updates applied to the base table.
+     * @return the views affected by {@code updates}.
+     */
+    public Collection<View> updatedViews(PartitionUpdate updates)
+    {
+        List<View> matchingViews = new ArrayList<>(views.size());
+
+        for (View view : views)
+        {
+            ReadQuery selectQuery = view.getReadQuery();
+            if (!selectQuery.selectsKey(updates.partitionKey()))
+                continue;
+
+            matchingViews.add(view);
+        }
+        return matchingViews;
+    }
+
+    /**
+     * Returns the command to use to read the existing rows required to generate view updates for the provided base
+     * base updates.
+     *
+     * @param updates the base table updates being applied.
+     * @param views the views potentially affected by {@code updates}.
+     * @param nowInSec the current time in seconds.
+     * @return the command to use to read the base table rows required to generate view updates for {@code updates}.
+     */
+    private SinglePartitionReadCommand readExistingRowsCommand(PartitionUpdate updates, Collection<View> views, int nowInSec)
+    {
+        Slices.Builder sliceBuilder = null;
+        DeletionInfo deletionInfo = updates.deletionInfo();
+        CFMetaData metadata = updates.metadata();
+        DecoratedKey key = updates.partitionKey();
+        // TODO: This is subtle: we need to gather all the slices that we have to fetch between partition del, range tombstones and rows.
+        if (!deletionInfo.isLive())
+        {
+            sliceBuilder = new Slices.Builder(metadata.comparator);
+            // Everything covered by a deletion might invalidate an existing view entry, which means we must read it to know. In practice
+            // though, the views involved might filter some base table clustering columns, in which case we can restrict what we read
+            // using those restrictions.
+            // If there is a partition deletion, then we can simply take each slices from each view select filter. They may overlap but
+            // the Slices.Builder handles that for us. Note that in many case this will just involve reading everything (as soon as any
+            // view involved has no clustering restrictions for instance).
+            // For range tombstone, we should theoretically take the difference between the range tombstoned and the slices selected
+            // by every views, but as we don't an easy way to compute that right now, we keep it simple and just use the tombstoned
+            // range.
+            // TODO: we should improve that latter part.
+            if (!deletionInfo.getPartitionDeletion().isLive())
+            {
+                for (View view : views)
+                    sliceBuilder.addAll(view.getSelectStatement().clusteringIndexFilterAsSlices());
+            }
+            else
+            {
+                assert deletionInfo.hasRanges();
+                Iterator<RangeTombstone> iter = deletionInfo.rangeIterator(false);
+                while (iter.hasNext())
+                    sliceBuilder.add(iter.next().deletedSlice());
+            }
+        }
+
+        // We need to read every row that is updated, unless we can prove that it has no impact on any view entries.
+
+        // If we had some slices from the deletions above, we'll continue using that. Otherwise, it's more efficient to build
+        // a names query.
+        BTreeSet.Builder<Clustering> namesBuilder = sliceBuilder == null ? BTreeSet.builder(metadata.comparator) : null;
+        for (Row row : updates)
+        {
+            // Don't read the existing state if we can prove the update won't affect any views
+            if (!affectsAnyViews(key, row, views))
+                continue;
+
+            if (namesBuilder == null)
+                sliceBuilder.add(Slice.make(row.clustering()));
+            else
+                namesBuilder.add(row.clustering());
+        }
+
+        NavigableSet<Clustering> names = namesBuilder == null ? null : namesBuilder.build();
+        // If we have a slice builder, it means we had some deletions and we have to read. But if we had
+        // only row updates, it's possible none of them affected the views, in which case we have nothing
+        // to do.
+        if (names != null && names.isEmpty())
+            return null;
+
+        ClusteringIndexFilter clusteringFilter = names == null
+                                               ? new ClusteringIndexSliceFilter(sliceBuilder.build(), false)
+                                               : new ClusteringIndexNamesFilter(names, false);
+        // 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);
+        // 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.
+        // TODO: we could still make sense to special case for when there is a single view and a small number of updates (and
+        // no deletions). Indeed, in that case we could check whether any of the update modify any of the restricted regular
+        // column, and if that's not the case we could use view filter. We keep it simple for now though.
+        RowFilter rowFilter = RowFilter.NONE;
+        return SinglePartitionReadCommand.create(metadata, nowInSec, queriedColumns, rowFilter, DataLimits.NONE, key, clusteringFilter);
+    }
+
+    private boolean affectsAnyViews(DecoratedKey partitionKey, Row update, Collection<View> views)
+    {
+        for (View view : views)
+        {
+            if (view.mayBeAffectedBy(partitionKey, update))
+                return true;
+        }
+        return false;
+    }
+
+    /**
+     * Given an existing base row and the update that we're going to apply to this row, generate the modifications
+     * to apply to MVs using the provided {@code ViewUpdateGenerator}s.
+     *
+     * @param existingBaseRow the base table row as it is before an update.
+     * @param updateBaseRow the newly updates made to {@code existingBaseRow}.
+     * @param generators the view update generators to add the new changes to.
+     * @param nowInSec the current time in seconds. Used to decide if data is live or not.
+     */
+    private static void addToViewUpdateGenerators(Row existingBaseRow, Row updateBaseRow, Collection<ViewUpdateGenerator> generators, int nowInSec)
+    {
+        // Having existing empty is useful, it just means we'll insert a brand new entry for updateBaseRow,
+        // but if we have no update at all, we shouldn't get there.
+        assert !updateBaseRow.isEmpty();
+
+        // We allow existingBaseRow to be null, which we treat the same as being empty as an small optimization
+        // to avoid allocating empty row objects when we know there was nothing existing.
+        Row mergedBaseRow = existingBaseRow == null ? updateBaseRow : Rows.merge(existingBaseRow, updateBaseRow, nowInSec);
+        for (ViewUpdateGenerator generator : generators)
+            generator.addBaseTableUpdate(existingBaseRow, mergedBaseRow);
+    }
+
+    private static Row emptyRow(Clustering clustering, DeletionTime deletion)
+    {
+        // Returning null for an empty row is slightly ugly, but the case where there is no pre-existing row is fairly common
+        // (especially when building the view), so we want to avoid a dummy allocation of an empty row every time.
+        // And MultiViewUpdateBuilder knows how to deal with that.
+        return deletion.isLive() ? null : BTreeRow.emptyDeletedRow(clustering, Row.Deletion.regular(deletion));
+    }
+
+    /**
+     * Extracts (and potentially groups) the mutations generated by the provided view update generator.
+     * Returns the mutation that needs to be done to the views given the base table updates
+     * passed to {@link #addBaseTableUpdate}.
+     *
+     * @param baseTableMetadata the metadata for the base table being updated.
+     * @param generators the generators from which to extract the view mutations from.
+     * @return the mutations created by all the generators in {@code generators}.
+     */
+    private Collection<Mutation> buildMutations(CFMetaData baseTableMetadata, List<ViewUpdateGenerator> generators)
+    {
+        // One view is probably common enough and we can optimize a bit easily
+        if (generators.size() == 1)
+        {
+            Collection<PartitionUpdate> updates = generators.get(0).generateViewUpdates();
+            List<Mutation> mutations = new ArrayList<>(updates.size());
+            for (PartitionUpdate update : updates)
+                mutations.add(new Mutation(update));
+            return mutations;
+        }
+
+        Map<DecoratedKey, Mutation> mutations = new HashMap<>();
+        for (ViewUpdateGenerator generator : generators)
+        {
+            for (PartitionUpdate update : generator.generateViewUpdates())
+            {
+                DecoratedKey key = update.partitionKey();
+                Mutation mutation = mutations.get(key);
+                if (mutation == null)
+                {
+                    mutation = new Mutation(baseTableMetadata.ksName, key);
+                    mutations.put(key, mutation);
+                }
+                mutation.add(update);
+            }
+        }
+        return mutations.values();
+    }
+
+    /**
+     * A simple helper that tracks for a given {@code UnfilteredRowIterator} what is the current deletion at any time of the
+     * iteration. It will be the currently open range tombstone deletion if there is one and the partition deletion otherwise.
+     */
+    private static class DeletionTracker
+    {
+        private final DeletionTime partitionDeletion;
+        private DeletionTime deletion;
+
+        public DeletionTracker(DeletionTime partitionDeletion)
+        {
+            this.partitionDeletion = partitionDeletion;
+        }
+
+        public void update(Unfiltered marker)
+        {
+            assert marker instanceof RangeTombstoneMarker;
+            RangeTombstoneMarker rtm = (RangeTombstoneMarker)marker;
+            this.deletion = rtm.isOpen(false)
+                          ? rtm.openDeletionTime(false)
+                          : null;
+        }
+
+        public DeletionTime currentDeletion()
+        {
+            return deletion == null ? partitionDeletion : deletion;
+        }
+    }
+}

http://git-wip-us.apache.org/repos/asf/cassandra/blob/86ba2274/src/java/org/apache/cassandra/db/view/TemporalRow.java
----------------------------------------------------------------------
diff --git a/src/java/org/apache/cassandra/db/view/TemporalRow.java b/src/java/org/apache/cassandra/db/view/TemporalRow.java
deleted file mode 100644
index 23705b9..0000000
--- a/src/java/org/apache/cassandra/db/view/TemporalRow.java
+++ /dev/null
@@ -1,610 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements.  See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership.  The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License.  You may obtain a copy of the License at
- *
- *     http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-
-package org.apache.cassandra.db.view;
-
-import static java.util.Comparator.naturalOrder;
-import static java.util.Comparator.reverseOrder;
-
-import java.nio.ByteBuffer;
-import java.util.ArrayList;
-import java.util.Collection;
-import java.util.Collections;
-import java.util.Comparator;
-import java.util.HashMap;
-import java.util.Iterator;
-import java.util.List;
-import java.util.Map;
-
-import com.google.common.base.MoreObjects;
-import com.google.common.collect.Iterables;
-
-import org.apache.cassandra.config.CFMetaData;
-import org.apache.cassandra.config.ColumnDefinition;
-import org.apache.cassandra.cql3.ColumnIdentifier;
-import org.apache.cassandra.db.CBuilder;
-import org.apache.cassandra.db.Clustering;
-import org.apache.cassandra.db.ColumnFamilyStore;
-import org.apache.cassandra.db.Conflicts;
-import org.apache.cassandra.db.DecoratedKey;
-import org.apache.cassandra.db.DeletionInfo;
-import org.apache.cassandra.db.DeletionTime;
-import org.apache.cassandra.db.LivenessInfo;
-import org.apache.cassandra.db.RangeTombstone;
-import org.apache.cassandra.db.Slice;
-import org.apache.cassandra.db.marshal.CompositeType;
-import org.apache.cassandra.db.partitions.AbstractBTreePartition;
-import org.apache.cassandra.db.rows.BufferCell;
-import org.apache.cassandra.db.rows.Cell;
-import org.apache.cassandra.db.rows.CellPath;
-import org.apache.cassandra.db.rows.Row;
-import org.apache.cassandra.utils.ByteBufferUtil;
-import org.apache.cassandra.utils.FBUtilities;
-
-/**
- * Represents a single CQL Row in a base table, with both the currently persisted value and the update's value. The
- * values are stored in timestamp order, but also indicate whether they are from the currently persisted, allowing a
- * {@link TemporalRow.Resolver} to resolve if the value is an old value that has been updated; if it sorts after the
- * update's value, then it does not qualify.
- */
-public class TemporalRow
-{
-    private static final int NO_TTL = LivenessInfo.NO_TTL;
-    private static final long NO_TIMESTAMP = LivenessInfo.NO_TIMESTAMP;
-    private static final int NO_DELETION_TIME = DeletionTime.LIVE.localDeletionTime();
-
-    public interface Resolver
-    {
-        /**
-         * @param cellVersions  all cells for a certain TemporalRow's Cell
-         * @return      A single TemporalCell from the iterable which satisfies the resolution criteria, or null if
-         *              there is no cell which qualifies
-         */
-        TemporalCell resolve(TemporalCell.Versions cellVersions);
-    }
-
-    public static final Resolver oldValueIfUpdated = TemporalCell.Versions::getOldCellIfUpdated;
-    public static final Resolver earliest = TemporalCell.Versions::getEarliestCell;
-    public static final Resolver latest = TemporalCell.Versions::getLatestCell;
-
-    private static class TemporalCell
-    {
-        public final ByteBuffer value;
-        public final long timestamp;
-        public final int ttl;
-        public final int localDeletionTime;
-        public final boolean isNew;
-
-        private TemporalCell(ByteBuffer value, long timestamp, int ttl, int localDeletionTime, boolean isNew)
-        {
-            this.value = value;
-            this.timestamp = timestamp;
-            this.ttl = ttl;
-            this.localDeletionTime = localDeletionTime;
-            this.isNew = isNew;
-        }
-
-        @Override
-        public String toString()
-        {
-            return MoreObjects.toStringHelper(this)
-                    .add("value", value == null ? "null" : ByteBufferUtil.bytesToHex(value))
-                    .add("timestamp", timestamp)
-                    .add("ttl", ttl)
-                    .add("localDeletionTime", localDeletionTime)
-                    .add("isNew", isNew)
-                    .toString();
-        }
-
-        public TemporalCell reconcile(TemporalCell that)
-        {
-            int now = FBUtilities.nowInSeconds();
-            Conflicts.Resolution resolution = Conflicts.resolveRegular(that.timestamp,
-                                                                       that.isLive(now),
-                                                                       that.localDeletionTime,
-                                                                       that.value,
-                                                                       this.timestamp,
-                                                                       this.isLive(now),
-                                                                       this.localDeletionTime,
-                                                                       this.value);
-            assert resolution != Conflicts.Resolution.MERGE;
-            if (resolution == Conflicts.Resolution.LEFT_WINS)
-                return that;
-            return this;
-        }
-
-        private boolean isLive(int now)
-        {
-            return localDeletionTime == NO_DELETION_TIME || (ttl != NO_TTL && now < localDeletionTime);
-        }
-
-        public Cell cell(ColumnDefinition definition, CellPath cellPath)
-        {
-            return new BufferCell(definition, timestamp, ttl, localDeletionTime, value, cellPath);
-        }
-
-        public boolean equals(Object o)
-        {
-            if (this == o) return true;
-            if (o == null || getClass() != o.getClass()) return false;
-
-            TemporalCell that = (TemporalCell) o;
-
-            if (timestamp != that.timestamp) return false;
-            if (ttl != that.ttl) return false;
-            if (localDeletionTime != that.localDeletionTime) return false;
-            if (isNew != that.isNew) return false;
-            return !(value != null ? !value.equals(that.value) : that.value != null);
-        }
-
-        public int hashCode()
-        {
-            int result = value != null ? value.hashCode() : 0;
-            result = 31 * result + (int) (timestamp ^ (timestamp >>> 32));
-            result = 31 * result + ttl;
-            result = 31 * result + localDeletionTime;
-            result = 31 * result + (isNew ? 1 : 0);
-            return result;
-        }
-
-        /**
-         * Tracks the versions of a cell for a given TemporalRow.
-         * There are only two possible versions, existing and new.
-         *
-         */
-        static class Versions
-        {
-            private TemporalCell existingCell = null;
-            private TemporalCell newCell = null;
-            private int numSet = 0;
-
-
-            /**
-             * @return the cell that is earliest
-             * Or would be overwritten in the case of a timestamp conflict
-             */
-            public TemporalCell getEarliestCell()
-            {
-                assert numSet > 0;
-
-                if (numSet == 1)
-                    return existingCell == null ? newCell : existingCell;
-
-                TemporalCell latest = existingCell.reconcile(newCell);
-
-                return latest == newCell ? existingCell : newCell;
-            }
-
-            /**
-             * @return the cell that is latest
-             * Or would be the winner in the case of a timestamp conflict
-             */
-            public TemporalCell getLatestCell()
-            {
-                assert numSet > 0;
-
-                if (numSet == 1)
-                    return existingCell == null ? newCell : existingCell;
-
-                return existingCell.reconcile(newCell);
-            }
-
-            /**
-             * @return the new cell if it updates the existing cell
-             */
-            public TemporalCell getOldCellIfUpdated()
-            {
-                assert numSet > 0;
-
-                if (numSet == 1)
-                   return null;
-
-                TemporalCell value = existingCell.reconcile(newCell);
-
-                return ByteBufferUtil.compareUnsigned(existingCell.value, value.value) != 0 ? existingCell : null;
-            }
-
-            void setVersion(TemporalCell cell)
-            {
-                assert cell != null;
-
-                if (cell.isNew)
-                {
-                    assert newCell == null || newCell.equals(cell) : "Only one cell version can be marked New; newCell: " + newCell + ", cell: " + cell;
-                    newCell = cell;
-                    numSet = existingCell == null ? 1 : 2;
-                }
-                else
-                {
-                    assert existingCell == null || existingCell.equals(cell) : "Only one cell version can be marked Existing; existingCell: " + existingCell + ", cell: " + cell;
-                    existingCell = cell;
-                    numSet = newCell == null ? 1 : 2;
-                }
-            }
-
-            public void addToRow(TemporalRow row, ColumnIdentifier column, CellPath path)
-            {
-                if (existingCell != null)
-                    row.addColumnValue(column, path, existingCell.timestamp, existingCell.ttl,
-                                       existingCell.localDeletionTime, existingCell.value, existingCell.isNew);
-
-                if (newCell != null)
-                    row.addColumnValue(column, path, newCell.timestamp, newCell.ttl,
-                                       newCell.localDeletionTime, newCell.value, newCell.isNew);
-            }
-
-            @Override
-            public String toString()
-            {
-                return MoreObjects.toStringHelper(this)
-                        .add("numSet", numSet)
-                        .add("existingCell", existingCell)
-                        .add("newCell", newCell)
-                        .toString();
-            }
-        }
-    }
-
-    private final ColumnFamilyStore baseCfs;
-    private final java.util.Set<ColumnIdentifier> viewPrimaryKey;
-    private final ByteBuffer basePartitionKey;
-    private final Map<ColumnIdentifier, ByteBuffer> clusteringColumns;
-    private final Row startRow;
-    private final boolean startIsNew;
-
-    public final int nowInSec;
-    private final Map<ColumnIdentifier, Map<CellPath, TemporalCell.Versions>> columnValues = new HashMap<>();
-    private int viewClusteringTtl = NO_TTL;
-    private long viewClusteringTimestamp = NO_TIMESTAMP;
-    private int viewClusteringLocalDeletionTime = NO_DELETION_TIME;
-
-    TemporalRow(ColumnFamilyStore baseCfs, java.util.Set<ColumnIdentifier> viewPrimaryKey, ByteBuffer key, Row row, int nowInSec, boolean isNew)
-    {
-        this.baseCfs = baseCfs;
-        this.viewPrimaryKey = viewPrimaryKey;
-        this.basePartitionKey = key;
-        this.startRow = row;
-        this.startIsNew = isNew;
-        this.nowInSec = nowInSec;
-
-        LivenessInfo liveness = row.primaryKeyLivenessInfo();
-        updateLiveness(liveness.ttl(), liveness.timestamp(), row.deletion().time().localDeletionTime());
-
-        List<ColumnDefinition> clusteringDefs = baseCfs.metadata.clusteringColumns();
-        clusteringColumns = new HashMap<>();
-
-        for (int i = 0; i < clusteringDefs.size(); i++)
-        {
-            ColumnDefinition cdef = clusteringDefs.get(i);
-            clusteringColumns.put(cdef.name, row.clustering().get(i));
-
-            addColumnValue(cdef.name, null, NO_TIMESTAMP, NO_TTL, NO_DELETION_TIME, row.clustering().get(i), isNew);
-        }
-    }
-
-    /*
-     * PK ts:5, ttl:1, deletion: 2
-     * Col ts:4, ttl:2, deletion: 3
-     *
-     * TTL use min, since it expires at the lowest time which we are expiring. If we have the above values, we
-     * would want to return 1, since the base row expires in 1 second.
-     *
-     * Timestamp uses max, as this is the time that the row has been written to the view. See CASSANDRA-10910.
-     *
-     * Local Deletion Time should use max, as this deletion will cover all previous values written.
-     */
-    private void updateLiveness(int ttl, long timestamp, int localDeletionTime)
-    {
-        // We are returning whichever is higher from valueIfSet
-        // Natural order will return the max: 1.compareTo(2) < 0, so 2 is returned
-        // Reverse order will return the min: 1.compareTo(2) > 0, so 1 is returned
-        this.viewClusteringTtl = valueIfSet(viewClusteringTtl, ttl, NO_TTL, reverseOrder());
-        this.viewClusteringTimestamp = valueIfSet(viewClusteringTimestamp, timestamp, NO_TIMESTAMP, naturalOrder());
-        this.viewClusteringLocalDeletionTime = valueIfSet(viewClusteringLocalDeletionTime, localDeletionTime, NO_DELETION_TIME, naturalOrder());
-    }
-
-    @Override
-    public String toString()
-    {
-        return MoreObjects.toStringHelper(this)
-                .add("table", baseCfs.keyspace.getName() + "." + baseCfs.getTableName())
-                .add("basePartitionKey", ByteBufferUtil.bytesToHex(basePartitionKey))
-                .add("startRow", startRow.toString(baseCfs.metadata))
-                .add("startIsNew", startIsNew)
-                .add("nowInSec", nowInSec)
-                .add("viewClusteringTtl", viewClusteringTtl)
-                .add("viewClusteringTimestamp", viewClusteringTimestamp)
-                .add("viewClusteringLocalDeletionTime", viewClusteringLocalDeletionTime)
-                .add("columnValues", columnValues)
-                .toString();
-    }
-
-    @Override
-    public boolean equals(Object o)
-    {
-        if (this == o) return true;
-        if (o == null || getClass() != o.getClass()) return false;
-
-        TemporalRow that = (TemporalRow) o;
-
-        if (!clusteringColumns.equals(that.clusteringColumns)) return false;
-        if (!basePartitionKey.equals(that.basePartitionKey)) return false;
-
-        return true;
-    }
-
-    @Override
-    public int hashCode()
-    {
-        int result = basePartitionKey.hashCode();
-        result = 31 * result + clusteringColumns.hashCode();
-        return result;
-    }
-
-    public void addColumnValue(ColumnIdentifier identifier,
-                               CellPath cellPath,
-                               long timestamp,
-                               int ttl,
-                               int localDeletionTime,
-                               ByteBuffer value,  boolean isNew)
-    {
-        if (!columnValues.containsKey(identifier))
-            columnValues.put(identifier, new HashMap<>());
-
-        Map<CellPath, TemporalCell.Versions> innerMap = columnValues.get(identifier);
-
-        if (!innerMap.containsKey(cellPath))
-            innerMap.put(cellPath, new TemporalCell.Versions());
-
-        // If this column is part of the view's primary keys
-        if (viewPrimaryKey.contains(identifier))
-        {
-            updateLiveness(ttl, timestamp, localDeletionTime);
-        }
-
-        innerMap.get(cellPath).setVersion(new TemporalCell(value, timestamp, ttl, localDeletionTime, isNew));
-    }
-
-    /**
-     * @return
-     * <ul>
-     *     <li>
-     *         If both existing and update are defaultValue, return defaultValue
-     *     </li>
-     *     <li>
-     *         If only one of existing or existing are defaultValue, return the one which is not
-     *     </li>
-     *     <li>
-     *         If both existing and update are not defaultValue, compare using comparator and return the higher one.
-     *     </li>
-     * </ul>
-     */
-    private static <T> T valueIfSet(T existing, T update, T defaultValue, Comparator<T> comparator)
-    {
-        if (existing.equals(defaultValue))
-            return update;
-        if (update.equals(defaultValue))
-            return existing;
-        return comparator.compare(existing, update) > 0 ? existing : update;
-    }
-
-    public int viewClusteringTtl()
-    {
-        return viewClusteringTtl;
-    }
-
-    public long viewClusteringTimestamp()
-    {
-        return viewClusteringTimestamp;
-    }
-
-    public int viewClusteringLocalDeletionTime()
-    {
-        return viewClusteringLocalDeletionTime;
-    }
-
-    public void addCell(Cell cell, boolean isNew)
-    {
-        addColumnValue(cell.column().name, cell.path(), cell.timestamp(), cell.ttl(), cell.localDeletionTime(), cell.value(), isNew);
-    }
-
-    // The Definition here is actually the *base table* definition
-    public ByteBuffer clusteringValue(ColumnDefinition definition, Resolver resolver)
-    {
-        ColumnDefinition baseDefinition = definition.cfName.equals(baseCfs.name)
-                                          ? definition
-                                          : baseCfs.metadata.getColumnDefinition(definition.name);
-
-        if (baseDefinition.isPartitionKey())
-        {
-            if (baseCfs.metadata.getKeyValidator() instanceof CompositeType)
-            {
-                CompositeType keyComparator = (CompositeType) baseCfs.metadata.getKeyValidator();
-                ByteBuffer[] components = keyComparator.split(basePartitionKey);
-                return components[baseDefinition.position()];
-            }
-            else
-            {
-                return basePartitionKey;
-            }
-        }
-        else
-        {
-            ColumnIdentifier columnIdentifier = baseDefinition.name;
-
-            if (clusteringColumns.containsKey(columnIdentifier))
-                return clusteringColumns.get(columnIdentifier);
-
-            Collection<org.apache.cassandra.db.rows.Cell> val = values(definition, resolver);
-            if (val != null && val.size() == 1)
-            {
-                org.apache.cassandra.db.rows.Cell cell = Iterables.getOnlyElement(val);
-                // handle single-column deletions correctly
-                return cell.isTombstone() ? null : cell.value();
-            }
-        }
-        return null;
-    }
-
-    public DeletionTime deletionTime(AbstractBTreePartition partition)
-    {
-        DeletionInfo deletionInfo = partition.deletionInfo();
-        if (!deletionInfo.getPartitionDeletion().isLive())
-            return deletionInfo.getPartitionDeletion();
-
-        Clustering baseClustering = baseClusteringBuilder().build();
-        RangeTombstone clusterTombstone = deletionInfo.rangeCovering(baseClustering);
-        if (clusterTombstone != null)
-            return clusterTombstone.deletionTime();
-
-        Row row = partition.getRow(baseClustering);
-        return row == null || row.deletion().isLive() ? DeletionTime.LIVE : row.deletion().time();
-    }
-
-    public Collection<org.apache.cassandra.db.rows.Cell> values(ColumnDefinition definition, Resolver resolver)
-    {
-        Map<CellPath, TemporalCell.Versions> innerMap = columnValues.get(definition.name);
-        if (innerMap == null)
-        {
-            return Collections.emptyList();
-        }
-
-        Collection<org.apache.cassandra.db.rows.Cell> value = new ArrayList<>();
-        for (Map.Entry<CellPath, TemporalCell.Versions> pathAndCells : innerMap.entrySet())
-        {
-            TemporalCell cell = resolver.resolve(pathAndCells.getValue());
-
-            if (cell != null)
-                value.add(cell.cell(definition, pathAndCells.getKey()));
-        }
-        return value;
-    }
-
-    public Slice baseSlice()
-    {
-        return baseClusteringBuilder().buildSlice();
-    }
-
-    private CBuilder baseClusteringBuilder()
-    {
-        CFMetaData metadata = baseCfs.metadata;
-        CBuilder builder = CBuilder.create(metadata.comparator);
-
-        ByteBuffer[] buffers = new ByteBuffer[clusteringColumns.size()];
-        for (Map.Entry<ColumnIdentifier, ByteBuffer> buffer : clusteringColumns.entrySet())
-            buffers[metadata.getColumnDefinition(buffer.getKey()).position()] = buffer.getValue();
-
-        for (ByteBuffer byteBuffer : buffers)
-            builder = builder.add(byteBuffer);
-
-        return builder;
-    }
-
-    public Clustering baseClustering()
-    {
-        return startRow.clustering();
-    }
-
-    static class Set implements Iterable<TemporalRow>
-    {
-        private final ColumnFamilyStore baseCfs;
-        private final java.util.Set<ColumnIdentifier> viewPrimaryKey;
-        private final ByteBuffer key;
-        public final DecoratedKey dk;
-        private final Map<Clustering, TemporalRow> clusteringToRow;
-        final int nowInSec = FBUtilities.nowInSeconds();
-        private boolean hasTombstonedExisting = false;
-
-        Set(ColumnFamilyStore baseCfs, java.util.Set<ColumnIdentifier> viewPrimaryKey, ByteBuffer key)
-        {
-            this.baseCfs = baseCfs;
-            this.viewPrimaryKey = viewPrimaryKey;
-            this.key = key;
-            this.dk = baseCfs.decorateKey(key);
-            this.clusteringToRow = new HashMap<>();
-        }
-
-        public Iterator<TemporalRow> iterator()
-        {
-            return clusteringToRow.values().iterator();
-        }
-
-        public TemporalRow getClustering(Clustering clustering)
-        {
-            return clusteringToRow.get(clustering);
-        }
-
-        public void addRow(Row row, boolean isNew)
-        {
-            TemporalRow temporalRow = clusteringToRow.get(row.clustering());
-            if (temporalRow == null)
-            {
-                temporalRow = new TemporalRow(baseCfs, viewPrimaryKey, key, row, nowInSec, isNew);
-                clusteringToRow.put(row.clustering(), temporalRow);
-            }
-
-            for (Cell cell : row.cells())
-            {
-                temporalRow.addCell(cell, isNew);
-            }
-        }
-
-        private void addRow(TemporalRow row)
-        {
-            TemporalRow newRow = new TemporalRow(baseCfs, viewPrimaryKey, key, row.startRow, nowInSec, row.startIsNew);
-
-            TemporalRow existing = clusteringToRow.put(row.startRow.clustering(), newRow);
-            assert existing == null;
-
-            for (Map.Entry<ColumnIdentifier, Map<CellPath, TemporalCell.Versions>> entry : row.columnValues.entrySet())
-            {
-                for (Map.Entry<CellPath, TemporalCell.Versions> cellPathEntry : entry.getValue().entrySet())
-                {
-                    TemporalCell.Versions cellVersions = cellPathEntry.getValue();
-
-                    cellVersions.addToRow(newRow, entry.getKey(), cellPathEntry.getKey());
-                }
-            }
-        }
-
-        public TemporalRow.Set withNewViewPrimaryKey(java.util.Set<ColumnIdentifier> viewPrimaryKey)
-        {
-            TemporalRow.Set newSet = new Set(baseCfs, viewPrimaryKey, key);
-
-            for (TemporalRow row : this)
-                newSet.addRow(row);
-
-            return newSet;
-        }
-
-        public boolean hasTombstonedExisting()
-        {
-            return hasTombstonedExisting;
-        }
-
-        public void setTombstonedExisting()
-        {
-            hasTombstonedExisting = true;
-        }
-
-        public int size()
-        {
-            return clusteringToRow.size();
-        }
-    }
-}