You are viewing a plain text version of this content. The canonical link for it is here.
Posted to pr@cassandra.apache.org by GitBox <gi...@apache.org> on 2022/11/16 12:30:46 UTC

[GitHub] [cassandra] smiklosovic opened a new pull request, #2020: CASSANDRA-18044

smiklosovic opened a new pull request, #2020:
URL: https://github.com/apache/cassandra/pull/2020

   Thanks for sending a pull request! Here are some tips if you're new here:
    
    * Ensure you have added or run the [appropriate tests](https://cassandra.apache.org/_/development/testing.html) for your PR.
    * Be sure to keep the PR description updated to reflect all changes.
    * Write your PR title to summarize what this PR proposes.
    * If possible, provide a concise example to reproduce the issue for a faster review.
    * Read our [contributor guidelines](https://cassandra.apache.org/_/development/index.html)
    * If you're making a documentation change, see our [guide to documentation contribution](https://cassandra.apache.org/_/development/documentation.html)
    
   Commit messages should follow the following format:
   
   ```
   <One sentence description, usually Jira title or CHANGES.txt summary>
   
   <Optional lengthier description (context on patch)>
   
   patch by <Authors>; reviewed by <Reviewers> for CASSANDRA-#####
   
   Co-authored-by: Name1 <email1>
   Co-authored-by: Name2 <email2>
   
   ```
   
   The [Cassandra Jira](https://issues.apache.org/jira/projects/CASSANDRA/issues/)
   
   


-- 
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-unsubscribe@cassandra.apache.org

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


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


[GitHub] [cassandra] smiklosovic commented on a diff in pull request #2020: CASSANDRA-18044

Posted by GitBox <gi...@apache.org>.
smiklosovic commented on code in PR #2020:
URL: https://github.com/apache/cassandra/pull/2020#discussion_r1029169579


##########
src/java/org/apache/cassandra/db/commitlog/CommitLogReplayer.java:
##########
@@ -376,25 +377,45 @@ public static CommitLogPosition firstNotCovered(Collection<IntervalSet<CommitLog
         public static ReplayFilter create()
         {
             // If no replaylist is supplied an empty array of strings is used to replay everything.
-            if (System.getProperty("cassandra.replayList") == null)
+
+            String replayList = CassandraRelevantProperties.COMMIT_LOG_REPLAY_LIST.getString();
+
+            if (replayList == null)
                 return new AlwaysReplayFilter();
 
             Multimap<String, String> toReplay = HashMultimap.create();
-            for (String rawPair : System.getProperty("cassandra.replayList").split(","))
+            for (String rawPair : replayList.split(","))
             {
                 String[] pair = StringUtils.split(rawPair.trim(), '.');
-                if (pair.length != 2)
-                    throw new IllegalArgumentException("Each table to be replayed must be fully qualified with keyspace name, e.g., 'system.peers'");
 
-                Keyspace ks = Schema.instance.getKeyspaceInstance(pair[0]);
+                if (pair.length > 2)
+                    throw new IllegalStateException(String.format("cassandra.replayList property contain an item which is not " +
+                                                    "in format 'keyspace' or 'keyspace.table' but it is '%s'",
+                                                    String.join(".", pair)));
+
+                String keyspaceName = pair[0];
+
+                Keyspace ks = Schema.instance.getKeyspaceInstance(keyspaceName);
                 if (ks == null)
-                    throw new IllegalArgumentException("Unknown keyspace " + pair[0]);
-                ColumnFamilyStore cfs = ks.getColumnFamilyStore(pair[1]);
-                if (cfs == null)
-                    throw new IllegalArgumentException(String.format("Unknown table %s.%s", pair[0], pair[1]));
+                    throw new IllegalArgumentException("Unknown keyspace " + keyspaceName);
+
+                if (pair.length == 1)
+                {
+                    for (ColumnFamilyStore cfs : ks.getColumnFamilyStores())
+                        toReplay.put(keyspaceName, cfs.name);
+                }
+                else
+                {
+                    ColumnFamilyStore cfs = ks.getColumnFamilyStore(pair[1]);
+                    if (cfs == null)
+                        throw new IllegalArgumentException(String.format("Unknown table %s.%s", keyspaceName, pair[1]));
 
-                toReplay.put(pair[0], pair[1]);
+                    toReplay.put(keyspaceName, pair[1]);
+                }
             }
+
+            logger.info("Tables going to be replayed: {}", toReplay.asMap().toString());

Review Comment:
   @blambov I think we should also cover the case when `cassandra.replayList` is not set which means that `toReplay` would be empty map so this would look like `Tables to be replayed: <<empty map string>>` which is a little bit misleading.
   
   I suggest to do it like:
   
   ````
   if (toReplay.isEmpty())
       logger.info("All mutations in commit logs will be replayed.");
   else
       logger.info("Tables to be replayed: {}", toReplay.asMap().toString());
   ````
   
   EDIT:
   
   even better version:
   
   ````
   if (toReplay.isEmpty())
       logger.info("All tables will be included in commit log replay.");
   else
       logger.info("Tables to be replayed: {}", toReplay.asMap().toString());
   ````
   
   The problem with first iteration was that if we say "all mutations in commit logs will be replayed", it is again not entirely true as we might do point-in-time restoration and in that case not _all_ mutations are in fact replayed. But all tables are included in such replaying. It just does not necessarily replay all mutations for them.
   
   WDYT?



-- 
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-unsubscribe@cassandra.apache.org

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


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


[GitHub] [cassandra] blambov commented on a diff in pull request #2020: CASSANDRA-18044

Posted by GitBox <gi...@apache.org>.
blambov commented on code in PR #2020:
URL: https://github.com/apache/cassandra/pull/2020#discussion_r1029092025


##########
src/java/org/apache/cassandra/db/commitlog/CommitLogReplayer.java:
##########
@@ -376,25 +377,45 @@ public static CommitLogPosition firstNotCovered(Collection<IntervalSet<CommitLog
         public static ReplayFilter create()
         {
             // If no replaylist is supplied an empty array of strings is used to replay everything.
-            if (System.getProperty("cassandra.replayList") == null)
+
+            String replayList = CassandraRelevantProperties.COMMIT_LOG_REPLAY_LIST.getString();
+
+            if (replayList == null)
                 return new AlwaysReplayFilter();
 
             Multimap<String, String> toReplay = HashMultimap.create();
-            for (String rawPair : System.getProperty("cassandra.replayList").split(","))
+            for (String rawPair : replayList.split(","))
             {
                 String[] pair = StringUtils.split(rawPair.trim(), '.');
-                if (pair.length != 2)
-                    throw new IllegalArgumentException("Each table to be replayed must be fully qualified with keyspace name, e.g., 'system.peers'");
 
-                Keyspace ks = Schema.instance.getKeyspaceInstance(pair[0]);
+                if (pair.length > 2)
+                    throw new IllegalStateException(String.format("cassandra.replayList property contain an item which is not " +
+                                                    "in format 'keyspace' or 'keyspace.table' but it is '%s'",
+                                                    String.join(".", pair)));
+
+                String keyspaceName = pair[0];
+
+                Keyspace ks = Schema.instance.getKeyspaceInstance(keyspaceName);
                 if (ks == null)
-                    throw new IllegalArgumentException("Unknown keyspace " + pair[0]);
-                ColumnFamilyStore cfs = ks.getColumnFamilyStore(pair[1]);
-                if (cfs == null)
-                    throw new IllegalArgumentException(String.format("Unknown table %s.%s", pair[0], pair[1]));
+                    throw new IllegalArgumentException("Unknown keyspace " + keyspaceName);
+
+                if (pair.length == 1)
+                {
+                    for (ColumnFamilyStore cfs : ks.getColumnFamilyStores())
+                        toReplay.put(keyspaceName, cfs.name);
+                }
+                else
+                {
+                    ColumnFamilyStore cfs = ks.getColumnFamilyStore(pair[1]);
+                    if (cfs == null)
+                        throw new IllegalArgumentException(String.format("Unknown table %s.%s", keyspaceName, pair[1]));
 
-                toReplay.put(pair[0], pair[1]);
+                    toReplay.put(keyspaceName, pair[1]);
+                }
             }
+
+            logger.info("Tables going to be replayed: {}", toReplay.asMap().toString());

Review Comment:
   The message does not sound right, maybe `Tables to be replayed`?



-- 
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-unsubscribe@cassandra.apache.org

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


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


[GitHub] [cassandra] smiklosovic commented on a diff in pull request #2020: CASSANDRA-18044

Posted by GitBox <gi...@apache.org>.
smiklosovic commented on code in PR #2020:
URL: https://github.com/apache/cassandra/pull/2020#discussion_r1029169579


##########
src/java/org/apache/cassandra/db/commitlog/CommitLogReplayer.java:
##########
@@ -376,25 +377,45 @@ public static CommitLogPosition firstNotCovered(Collection<IntervalSet<CommitLog
         public static ReplayFilter create()
         {
             // If no replaylist is supplied an empty array of strings is used to replay everything.
-            if (System.getProperty("cassandra.replayList") == null)
+
+            String replayList = CassandraRelevantProperties.COMMIT_LOG_REPLAY_LIST.getString();
+
+            if (replayList == null)
                 return new AlwaysReplayFilter();
 
             Multimap<String, String> toReplay = HashMultimap.create();
-            for (String rawPair : System.getProperty("cassandra.replayList").split(","))
+            for (String rawPair : replayList.split(","))
             {
                 String[] pair = StringUtils.split(rawPair.trim(), '.');
-                if (pair.length != 2)
-                    throw new IllegalArgumentException("Each table to be replayed must be fully qualified with keyspace name, e.g., 'system.peers'");
 
-                Keyspace ks = Schema.instance.getKeyspaceInstance(pair[0]);
+                if (pair.length > 2)
+                    throw new IllegalStateException(String.format("cassandra.replayList property contain an item which is not " +
+                                                    "in format 'keyspace' or 'keyspace.table' but it is '%s'",
+                                                    String.join(".", pair)));
+
+                String keyspaceName = pair[0];
+
+                Keyspace ks = Schema.instance.getKeyspaceInstance(keyspaceName);
                 if (ks == null)
-                    throw new IllegalArgumentException("Unknown keyspace " + pair[0]);
-                ColumnFamilyStore cfs = ks.getColumnFamilyStore(pair[1]);
-                if (cfs == null)
-                    throw new IllegalArgumentException(String.format("Unknown table %s.%s", pair[0], pair[1]));
+                    throw new IllegalArgumentException("Unknown keyspace " + keyspaceName);
+
+                if (pair.length == 1)
+                {
+                    for (ColumnFamilyStore cfs : ks.getColumnFamilyStores())
+                        toReplay.put(keyspaceName, cfs.name);
+                }
+                else
+                {
+                    ColumnFamilyStore cfs = ks.getColumnFamilyStore(pair[1]);
+                    if (cfs == null)
+                        throw new IllegalArgumentException(String.format("Unknown table %s.%s", keyspaceName, pair[1]));
 
-                toReplay.put(pair[0], pair[1]);
+                    toReplay.put(keyspaceName, pair[1]);
+                }
             }
+
+            logger.info("Tables going to be replayed: {}", toReplay.asMap().toString());

Review Comment:
   @blambov I think we should also cover the case when `cassandra.replayList` is not set which means that `toReplay` would be empty map so this would look like `Tables to be replayed: <<empty map string>>` which is a little bit misleading.
   
   I suggest to do it like:
   
   ````
   if (toReplay.isEmpty())
       logger.info("All mutations in commit logs will be replayed.");
   else
       logger.info("Tables to be replayed: {}", toReplay.asMap().toString());
   ````



-- 
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-unsubscribe@cassandra.apache.org

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


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


[GitHub] [cassandra] smiklosovic commented on a diff in pull request #2020: CASSANDRA-18044

Posted by GitBox <gi...@apache.org>.
smiklosovic commented on code in PR #2020:
URL: https://github.com/apache/cassandra/pull/2020#discussion_r1024634307


##########
src/java/org/apache/cassandra/config/CassandraRelevantProperties.java:
##########
@@ -313,9 +313,24 @@
 
     /** Used when running in Client mode and the system and schema keyspaces need to be initialized outside of their normal initialization path **/
     FORCE_LOAD_LOCAL_KEYSPACES("cassandra.schema.force_load_local_keyspaces"),
-    ;
 
+    // commit log relevant properties
+    /**
+     * Entities to replay mutations for upon commit log replay, property is meant to contain
+     * comma-separated entities which are either names of keyspaces or keyspaces and tables or their mix.
+     * Examples:
+     * just keyspaces
+     * -Dcassandra.replayList=ks1,ks2,ks3
+     * specific tables
+     * -Dcassandra.replayList=ks1.tb1,ks2.tb2
+     * mix of tables and keyspaces
+     * -Dcassandra.replayList=ks1.tb1,ks2
+     *
+     * If only keyspaces are specified, mutations for all tables in such keyspace will be replayed
+     * */
+    COMMIT_LOG_REPLAY_LIST("cassandra.replayList", null)

Review Comment:
   I moved this one here from CommitLogReplayer when I was on it.



-- 
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-unsubscribe@cassandra.apache.org

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


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


[GitHub] [cassandra] smiklosovic commented on a diff in pull request #2020: CASSANDRA-18044

Posted by GitBox <gi...@apache.org>.
smiklosovic commented on code in PR #2020:
URL: https://github.com/apache/cassandra/pull/2020#discussion_r1024634470


##########
src/java/org/apache/cassandra/db/commitlog/CommitLogReplayer.java:
##########
@@ -376,25 +377,45 @@ public static CommitLogPosition firstNotCovered(Collection<IntervalSet<CommitLog
         public static ReplayFilter create()
         {
             // If no replaylist is supplied an empty array of strings is used to replay everything.
-            if (System.getProperty("cassandra.replayList") == null)
+
+            String replayList = CassandraRelevantProperties.COMMIT_LOG_REPLAY_LIST.getString();
+
+            if (replayList == null)
                 return new AlwaysReplayFilter();
 
             Multimap<String, String> toReplay = HashMultimap.create();
-            for (String rawPair : System.getProperty("cassandra.replayList").split(","))

Review Comment:
   moved to CassandraRelevantProperties as it should be 



-- 
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-unsubscribe@cassandra.apache.org

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


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


[GitHub] [cassandra] smiklosovic closed pull request #2020: CASSANDRA-18044

Posted by GitBox <gi...@apache.org>.
smiklosovic closed pull request #2020: CASSANDRA-18044
URL: https://github.com/apache/cassandra/pull/2020


-- 
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-unsubscribe@cassandra.apache.org

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


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


[GitHub] [cassandra] smiklosovic commented on a diff in pull request #2020: CASSANDRA-18044

Posted by GitBox <gi...@apache.org>.
smiklosovic commented on code in PR #2020:
URL: https://github.com/apache/cassandra/pull/2020#discussion_r1029169579


##########
src/java/org/apache/cassandra/db/commitlog/CommitLogReplayer.java:
##########
@@ -376,25 +377,45 @@ public static CommitLogPosition firstNotCovered(Collection<IntervalSet<CommitLog
         public static ReplayFilter create()
         {
             // If no replaylist is supplied an empty array of strings is used to replay everything.
-            if (System.getProperty("cassandra.replayList") == null)
+
+            String replayList = CassandraRelevantProperties.COMMIT_LOG_REPLAY_LIST.getString();
+
+            if (replayList == null)
                 return new AlwaysReplayFilter();
 
             Multimap<String, String> toReplay = HashMultimap.create();
-            for (String rawPair : System.getProperty("cassandra.replayList").split(","))
+            for (String rawPair : replayList.split(","))
             {
                 String[] pair = StringUtils.split(rawPair.trim(), '.');
-                if (pair.length != 2)
-                    throw new IllegalArgumentException("Each table to be replayed must be fully qualified with keyspace name, e.g., 'system.peers'");
 
-                Keyspace ks = Schema.instance.getKeyspaceInstance(pair[0]);
+                if (pair.length > 2)
+                    throw new IllegalStateException(String.format("cassandra.replayList property contain an item which is not " +
+                                                    "in format 'keyspace' or 'keyspace.table' but it is '%s'",
+                                                    String.join(".", pair)));
+
+                String keyspaceName = pair[0];
+
+                Keyspace ks = Schema.instance.getKeyspaceInstance(keyspaceName);
                 if (ks == null)
-                    throw new IllegalArgumentException("Unknown keyspace " + pair[0]);
-                ColumnFamilyStore cfs = ks.getColumnFamilyStore(pair[1]);
-                if (cfs == null)
-                    throw new IllegalArgumentException(String.format("Unknown table %s.%s", pair[0], pair[1]));
+                    throw new IllegalArgumentException("Unknown keyspace " + keyspaceName);
+
+                if (pair.length == 1)
+                {
+                    for (ColumnFamilyStore cfs : ks.getColumnFamilyStores())
+                        toReplay.put(keyspaceName, cfs.name);
+                }
+                else
+                {
+                    ColumnFamilyStore cfs = ks.getColumnFamilyStore(pair[1]);
+                    if (cfs == null)
+                        throw new IllegalArgumentException(String.format("Unknown table %s.%s", keyspaceName, pair[1]));
 
-                toReplay.put(pair[0], pair[1]);
+                    toReplay.put(keyspaceName, pair[1]);
+                }
             }
+
+            logger.info("Tables going to be replayed: {}", toReplay.asMap().toString());

Review Comment:
   @blambov I think we should also cover the case when `cassandra.replayList` is not set which means that `toReplay` would be empty map so this would look like `Tables to be replayed: <<empty map string>>` which is a little bit misleading.
   
   I suggest to do it like:
   
   ````
   if (toReplay.isEmpty())
       logger.info("All mutations in commit logs will be replayed.");
   else
       logger.info("Tables to be replayed: {}", toReplay.asMap().toString());
   ````
   
   EDIT:
   
   even better version:
   
   ````
   if (toReplay.isEmpty())
       logger.info("All tables will be included in commit log replay.");
   else
       logger.info("Tables to be replayed: {}", toReplay.asMap().toString());
   ````
   
   The problem with first iteration was that if we say "all mutations in commit logs will be replayed", it is again not entirely true as we might do point-in-time restoration and in that case not _all_ mutations are in fact replayed. But all tables are included in such replaying. It just does not replay all mutations for them.
   
   WDYT?



-- 
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-unsubscribe@cassandra.apache.org

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


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