iamaleksey commented on code in PR #1831:
URL: https://github.com/apache/cassandra/pull/1831#discussion_r963846345


##########
src/java/org/apache/cassandra/schema/SchemaKeyspace.java:
##########
@@ -955,6 +962,10 @@ private static TableMetadata fetchTable(String 
keyspaceName, String tableName, T
     static TableParams createTableParamsFromRow(UntypedResultSet.Row row)
     {
         return TableParams.builder()
+                          // allow_auto_snapshot column was introduced in 4.2
+                          .allowAutoSnapshot(row.has("allow_auto_snapshot") ?
+                                             
row.getBoolean("allow_auto_snapshot", DatabaseDescriptor.isAutoSnapshot()) :
+                                             
DatabaseDescriptor.isAutoSnapshot())

Review Comment:
   Again, when it comes to schema things, `DatabaseDescriptor.isAutoSnapshot()` 
shouldn't be relevant. Only set the value on the builder at all if the column 
is present in the row. Otherwise let it pick up the default builder value of 
`true` and later defer to `ColumnFamilyStore.isAutoSnapshot()` to see if both 
the per-table setting and the node-local one are true.



##########
src/java/org/apache/cassandra/schema/SchemaKeyspace.java:
##########
@@ -563,6 +565,11 @@ private static void addTableParamsToRowBuilder(TableParams 
params, Row.SimpleBui
         // in mixed operation with pre-4.1 versioned node during upgrades.
         if (params.memtable != MemtableParams.DEFAULT)
             builder.add("memtable", params.memtable.configurationKey());
+
+        // As above, only add the allow_auto_snapshot column if the value is 
not default (true) and
+        // auto-snapshotting is enabled, to avoid RTE in pre-4.2 versioned 
node during upgrades
+        if (DatabaseDescriptor.isAutoSnapshot() && !params.allowAutoSnapshot)

Review Comment:
   I would eliminate the `DatabaseDescriptor.isAutoSnapshot()` condition here - 
I'm unsure why it was added here? Only the value of the table param is 
important here.



##########
src/java/org/apache/cassandra/cql3/UntypedResultSet.java:
##########
@@ -359,6 +359,12 @@ public boolean getBoolean(String column)
             return BooleanType.instance.compose(data.get(column));
         }
 
+        public boolean getBoolean(String column, boolean ifNull)
+        {
+            ByteBuffer bytes = data.get(column);
+            return bytes == null ? ifNull : 
BooleanType.instance.compose(bytes);
+        }
+

Review Comment:
   The method will no longer be needed if the other comment in this review is 
followed.



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