jaydeepkumar1984 commented on code in PR #4358:
URL: https://github.com/apache/cassandra/pull/4358#discussion_r2418235753


##########
conf/cassandra_latest.yaml:
##########
@@ -2054,6 +2054,12 @@ materialized_views_enabled: false
 # (losing > quorum # nodes in a replica set), you may have data in your 
SSTables that never makes it to the Materialized View.
 # materialized_views_on_repair_enabled: true
 
+# Controls whether metrics are collected for base tables when materialized 
views are present.
+# When enabled, additional metrics will be collected for the base tables that 
have materialized views.
+# The metrics are used for determine if current MV base table is qualified to 
move to strict MV consistency.

Review Comment:
   nit: Can you provide some example metrics that would be collected? Also, 
clarify that the metrics collection would not have any negative implications.



##########
src/java/org/apache/cassandra/service/paxos/Paxos.java:
##########
@@ -1168,6 +1179,29 @@ class WasRun implements Runnable { boolean v; public 
void run() { v = true; } }
         }
     }
 
+    public static void 
resolveCurrentAndApplyMVMutations(SinglePartitionReadCommand query,
+                                                         PartitionUpdate 
update,
+                                                         
PaxosPrepare.FoundIncompleteAccepted inProgress,
+                                                         ConsistencyLevel 
consistencyForConsensus,
+                                                         
Dispatcher.RequestTime requestTime)
+    {
+        // process to get current value of base table
+        Supplier<Participants> plan = () -> inProgress.participants;
+        DataResolver<?, ?> resolver = new 
DataResolver(ReadCoordinator.DEFAULT, query, plan, NoopReadRepair.instance, 
requestTime);
+        for (int i = 0 ; i < inProgress.responses.size() ; ++i)
+            resolver.preprocess(inProgress.responses.get(i));
+        PartitionIterator result = resolver.resolve();
+        FilteredPartition current;
+        try (RowIterator iter = PartitionIterators.getOnlyElement(result, 
query))
+        {
+            current = FilteredPartition.create(iter);
+        }
+        final ConsistencyLevel consistencyLevelForMVMutation = 
consistencyForConsensus == ConsistencyLevel.LOCAL_SERIAL

Review Comment:
   Add a reason why we are not using the user-supplied consistency because we 
want MV mutations to be applied on a QUORUM of replicas. The user might provide 
lower consistency, such as _ONE_.



##########
src/java/org/apache/cassandra/db/ColumnFamilyStore.java:
##########
@@ -2637,6 +2611,39 @@ public void unloadCf()
             FBUtilities.waitOnFuture(dumpMemtable());
     }
 
+    public void checkQualifiedForStrictMVConsistency()

Review Comment:
   Needs test cases for all the conditions inside this API 



##########
src/java/org/apache/cassandra/service/paxos/Paxos.java:
##########
@@ -1168,6 +1179,29 @@ class WasRun implements Runnable { boolean v; public 
void run() { v = true; } }
         }
     }
 
+    public static void 
resolveCurrentAndApplyMVMutations(SinglePartitionReadCommand query,
+                                                         PartitionUpdate 
update,
+                                                         
PaxosPrepare.FoundIncompleteAccepted inProgress,
+                                                         ConsistencyLevel 
consistencyForConsensus,
+                                                         
Dispatcher.RequestTime requestTime)
+    {
+        // process to get current value of base table
+        Supplier<Participants> plan = () -> inProgress.participants;
+        DataResolver<?, ?> resolver = new 
DataResolver(ReadCoordinator.DEFAULT, query, plan, NoopReadRepair.instance, 
requestTime);
+        for (int i = 0 ; i < inProgress.responses.size() ; ++i)

Review Comment:
   Add {}



##########
src/java/org/apache/cassandra/config/Config.java:
##########
@@ -635,6 +635,14 @@ public static class SSTableConfig
     // When false, it behaves the same as normal streaming.
     public volatile boolean materialized_views_on_repair_enabled = true;
 
+    public boolean materialized_view_base_table_metric_collection_enabled = 
false;
+
+    // node level setting, both materialized_view_strict_consistency_enabled 
and table level option
+    // strict_mv_consistency need to be enabled to enable the feature.
+    public boolean materialized_view_strict_consistency_enabled = false;
+
+    public boolean materialized_view_strict_consistency_enforced = false;

Review Comment:
   Can you clarify this parameter and the difference between the two?



##########
src/java/org/apache/cassandra/cql3/statements/ModificationStatement.java:
##########
@@ -678,6 +720,28 @@ private ResultMessage executeWithCondition(QueryState 
queryState, QueryOptions o
         }
     }
 
+    private ResultMessage executeWithStrictMVConsistency(QueryState 
queryState, QueryOptions options, Dispatcher.RequestTime requestTime)
+    {
+        // Strict MV consistency currently requires Paxos V2. Once additional 
transaction modes for strict MV are supported, we can relax this restriction.
+        assert metadata().params.transactionalMode == TransactionalMode.off && 
Paxos.useV2();
+        CQL3CasRequest request = makeCasRequest(queryState, options, 
requestTime);
+
+        try (RowIterator result = StorageProxy.cas(keyspace(),
+                table(),
+                request.key,
+                request,
+                options.getSerialConsistency(),
+                options.getConsistency(),
+                queryState.getClientState(),
+                options.getNowInSeconds(queryState),
+                requestTime))
+        {
+            if (hasConditions())

Review Comment:
   Add {



##########
src/java/org/apache/cassandra/cql3/statements/CQL3CasRequest.java:
##########
@@ -241,11 +244,18 @@ private RegularAndStaticColumns columnsToRead()
 
     public SinglePartitionReadCommand readCommand(long nowInSec)
     {
-        assert staticConditions != null || !conditions.isEmpty();
+        assert staticConditions != null || !conditions.isEmpty() || 
metadata.strictMVEnabled();
 
         // Fetch all columns, but query only the selected ones
         ColumnFilter columnFilter = ColumnFilter.selection(columnsToRead());
 
+        if (metadata.strictMVEnabled())
+        {
+            // making sure base table request is only updating one row
+            assert updates.size() == 1;

Review Comment:
   Instead of asserting, we should throw a proper exception, 
_InvalidRequestException_



##########
src/java/org/apache/cassandra/db/ColumnFamilyStore.java:
##########
@@ -2637,6 +2611,39 @@ public void unloadCf()
             FBUtilities.waitOnFuture(dumpMemtable());
     }
 
+    public void checkQualifiedForStrictMVConsistency()

Review Comment:
   I think this API should take a boolean argument `bool strictMVConsistency`
   If this is false, then please go ahead and return; else, continue the 
checks. Without the argument, the API provides an inaccurate view. For example, 
if `if (!DatabaseDescriptor.getMaterializedViewStrictConsistencyEnabled())`, we 
encounter an error.



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: pr-unsubscr...@cassandra.apache.org

For queries about this service, please contact Infrastructure at:
us...@infra.apache.org


---------------------------------------------------------------------
To unsubscribe, e-mail: pr-unsubscr...@cassandra.apache.org
For additional commands, e-mail: pr-h...@cassandra.apache.org

Reply via email to