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


##########
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 
getCurrentAndApplyMVMutations(SinglePartitionReadCommand query,

Review Comment:
   nit; should we name this to something like, 
resolveCurrentAndApplyMVMutations, because "get" might confuse the people in 
that they would assume that it is doing a read RPC to get the current state, 
which is not the case.



##########
src/java/org/apache/cassandra/cql3/statements/BatchStatement.java:
##########
@@ -233,6 +235,20 @@ public void validate() throws InvalidRequestException
 
         for (ModificationStatement statement : statements)
         {
+            if 
(DatabaseDescriptor.getMaterializedViewsBasetableMetricCollectionEnabled())

Review Comment:
   These four lines are redundant at multiple places. I think it would be a 
good idea to create a wrapper for it.



##########
src/java/org/apache/cassandra/service/paxos/PaxosRepair.java:
##########
@@ -294,8 +299,14 @@ else if (isAcceptedButNotCommitted || 
isPromisedButNotAccepted || latestWitnesse
                 // Since this operation is not urgent, and we can piggy-back 
on other paxos operations
                 if (logger.isTraceEnabled())
                     logger.trace("PaxosRepair of {} found incomplete promise 
or proposal; preparing stale ballot {}", partitionKey(), 
Ballot.toString(ballot));
+                SinglePartitionReadCommand readCommand = null;
+                if (isAcceptedButNotCommitted && table.strictMVEnabled() && 
!latestAccepted.update.isEmpty())
+                {
+                    assert latestAccepted.update.rowCount() == 1 : "latest 
accepted with multiple rows: " + latestAccepted.update.rowCount();

Review Comment:
   Is this assertion even feasible in everyday situations? Or possible in case 
of bugs only?



##########
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;

Review Comment:
   add this new configurations to config/cassandra_latest.yaml



-- 
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: [email protected]

For queries about this service, please contact Infrastructure at:
[email protected]


---------------------------------------------------------------------
To unsubscribe, e-mail: [email protected]
For additional commands, e-mail: [email protected]

Reply via email to