You are viewing a plain text version of this content. The canonical link for it is here.
Posted to pr@cassandra.apache.org by "Maxwell-Guo (via GitHub)" <gi...@apache.org> on 2023/01/30 05:06:09 UTC

[GitHub] [cassandra] Maxwell-Guo opened a new pull request, #2117: Add a virtual table to list snapshots for CASSANDRA-18102

Maxwell-Guo opened a new pull request, #2117:
URL: https://github.com/apache/cassandra/pull/2117

   Add a virtual table to list snapshots for CASSANDRA-18102
   
   ```
   patch by maxwellguo; reviewed by <Reviewers> for CASSANDRA-18102
   ```
   
   The [CASSANDRA-18102](https://issues.apache.org/jira/browse/CASSANDRA-18102)
   
   


-- 
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 #2117: Add a virtual table to list snapshots for CASSANDRA-18102

Posted by "smiklosovic (via GitHub)" <gi...@apache.org>.
smiklosovic commented on code in PR #2117:
URL: https://github.com/apache/cassandra/pull/2117#discussion_r1124167982


##########
src/java/org/apache/cassandra/db/virtual/SnapshotsTable.java:
##########
@@ -0,0 +1,93 @@
+/*
+ * 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.virtual;
+
+import java.util.List;
+import java.util.Map;
+import java.util.Set;
+import javax.management.openmbean.TabularData;
+import javax.management.openmbean.TabularDataSupport;
+import com.google.common.collect.ImmutableMap;
+
+import org.apache.cassandra.db.SnapshotDetailsTabularData;
+import org.apache.cassandra.db.marshal.BooleanType;
+import org.apache.cassandra.db.marshal.LongType;
+import org.apache.cassandra.db.marshal.UTF8Type;
+import org.apache.cassandra.dht.LocalPartitioner;
+import org.apache.cassandra.schema.TableMetadata;
+import org.apache.cassandra.service.StorageService;
+
+public class SnapshotsTable extends AbstractVirtualTable
+{
+    private static final String SNAPSHOT_NAME = "snapshot_name";
+    private static final String KEYSPACE_NAME = "keyspace_name";
+    private static final String COLUMNFAMILY_NAME = "columnfamily_name";
+    private static final String TRUE_SIZE = "true_size";
+    private static final String SIZE_ON_DISK = "size_on_disk";
+    private static final String CREATE_TIME = "created_at";
+    private static final String EXPIRATION_TIME = "expires_at";
+    private static final String EPHEMERAL = "ephemeral";
+    
+    SnapshotsTable(String keyspace)
+    {
+        super(TableMetadata.builder(keyspace, "snapshots")
+                           .comment("tables snapshots")
+                           .kind(TableMetadata.Kind.VIRTUAL)
+                           .partitioner(new LocalPartitioner(UTF8Type.instance))
+                           .addPartitionKeyColumn(SNAPSHOT_NAME, UTF8Type.instance)

Review Comment:
   @pauloricardomg I think we should bring this to ML as we are changing the behavior suddenly and some snapshots will not be possible anymore (snapshots with same name).



-- 
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] pauloricardomg commented on a diff in pull request #2117: Add a virtual table to list snapshots for CASSANDRA-18102

Posted by "pauloricardomg (via GitHub)" <gi...@apache.org>.
pauloricardomg commented on code in PR #2117:
URL: https://github.com/apache/cassandra/pull/2117#discussion_r1124597812


##########
src/java/org/apache/cassandra/db/virtual/SnapshotsTable.java:
##########
@@ -0,0 +1,93 @@
+/*
+ * 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.virtual;
+
+import java.util.List;
+import java.util.Map;
+import java.util.Set;
+import javax.management.openmbean.TabularData;
+import javax.management.openmbean.TabularDataSupport;
+import com.google.common.collect.ImmutableMap;
+
+import org.apache.cassandra.db.SnapshotDetailsTabularData;
+import org.apache.cassandra.db.marshal.BooleanType;
+import org.apache.cassandra.db.marshal.LongType;
+import org.apache.cassandra.db.marshal.UTF8Type;
+import org.apache.cassandra.dht.LocalPartitioner;
+import org.apache.cassandra.schema.TableMetadata;
+import org.apache.cassandra.service.StorageService;
+
+public class SnapshotsTable extends AbstractVirtualTable
+{
+    private static final String SNAPSHOT_NAME = "snapshot_name";
+    private static final String KEYSPACE_NAME = "keyspace_name";
+    private static final String COLUMNFAMILY_NAME = "columnfamily_name";
+    private static final String TRUE_SIZE = "true_size";
+    private static final String SIZE_ON_DISK = "size_on_disk";
+    private static final String CREATE_TIME = "created_at";
+    private static final String EXPIRATION_TIME = "expires_at";
+    private static final String EPHEMERAL = "ephemeral";
+    
+    SnapshotsTable(String keyspace)
+    {
+        super(TableMetadata.builder(keyspace, "snapshots")
+                           .comment("tables snapshots")
+                           .kind(TableMetadata.Kind.VIRTUAL)
+                           .partitioner(new LocalPartitioner(UTF8Type.instance))
+                           .addPartitionKeyColumn(SNAPSHOT_NAME, UTF8Type.instance)

Review Comment:
   ok I will send a message shortly to the ML to see what others think. In any case we should make the behavior consistent whatever we agree to (either allow `takeSnapshot` to take snapshot with existing id on different table, or restrict `takeMultipleTableSnapshot` to not allow snapshot with existing id).



-- 
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 #2117: Add a virtual table to list snapshots for CASSANDRA-18102

Posted by "smiklosovic (via GitHub)" <gi...@apache.org>.
smiklosovic commented on code in PR #2117:
URL: https://github.com/apache/cassandra/pull/2117#discussion_r1123076202


##########
src/java/org/apache/cassandra/db/virtual/SnapshotsTable.java:
##########
@@ -0,0 +1,93 @@
+/*
+ * 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.virtual;
+
+import java.util.List;
+import java.util.Map;
+import java.util.Set;
+import javax.management.openmbean.TabularData;
+import javax.management.openmbean.TabularDataSupport;
+import com.google.common.collect.ImmutableMap;
+
+import org.apache.cassandra.db.SnapshotDetailsTabularData;
+import org.apache.cassandra.db.marshal.BooleanType;
+import org.apache.cassandra.db.marshal.LongType;
+import org.apache.cassandra.db.marshal.UTF8Type;
+import org.apache.cassandra.dht.LocalPartitioner;
+import org.apache.cassandra.schema.TableMetadata;
+import org.apache.cassandra.service.StorageService;
+
+public class SnapshotsTable extends AbstractVirtualTable
+{
+    private static final String SNAPSHOT_NAME = "snapshot_name";
+    private static final String KEYSPACE_NAME = "keyspace_name";
+    private static final String COLUMNFAMILY_NAME = "columnfamily_name";
+    private static final String TRUE_SIZE = "true_size";
+    private static final String SIZE_ON_DISK = "size_on_disk";
+    private static final String CREATE_TIME = "created_at";
+    private static final String EXPIRATION_TIME = "expires_at";
+    private static final String EPHEMERAL = "ephemeral";
+    
+    SnapshotsTable(String keyspace)
+    {
+        super(TableMetadata.builder(keyspace, "snapshots")
+                           .comment("tables snapshots")
+                           .kind(TableMetadata.Kind.VIRTUAL)
+                           .partitioner(new LocalPartitioner(UTF8Type.instance))
+                           .addPartitionKeyColumn(SNAPSHOT_NAME, UTF8Type.instance)

Review Comment:
   @pauloricardomg 
   
   Regardless of whether we can query without allow filtering after 18238 I think it is still a good practice to act as we would normally do when modeling the schema.
   
   As explained above and what Maxwell just explained, I think keyspace > table > snapshot id.
   
   We can have same snapshot names after all, no? So if keyspace and table are clustering columns, with two snapshots of the same name, we would have this partition:
   
       snapshotName1 | keyspace1 | table1
       snapshotName1 | keyspace2 | table2
   
   primary key would be `((snapshotName), keyspaceName, tableName)`
   
   Is not this counter-intuitive to have a partition which is logically coupling different keyspaces and names under the same snapshot name for logically different snapshots? That does not make sense to me. 
   
   However, on the other hand, it is possible to do this:
   
       ./bin/nodetool snapshot --kt-list ks.tb,system.local -t mysnapshot
   
   So `listsnapshots` will do this:
   
       mysnapshot  system  local  1.16 KiB  21.47 KiB    2023-03-02T13:19:13.757Z
       mysnapshot  ks      tb     1.02 KiB   6.08 KiB    2023-03-02T13:19:13.757Z
   
   But then I can do this as well, again:
   
       ./bin/nodetool snapshot --kt-list ks.tb2 -t mysnapshot
   
   Which would print it like:
   
       mysnapshot     ks       tb2        1.16 KiB  21.47 KiB    2023-03-02T13:19:42.140Z                
       mysnapshot     ks       tb         1.02 KiB  6.08 KiB     2023-03-02T13:19:13.757Z                
       mysnapshot     system   local      107 bytes 6.98 KiB     2023-03-02T13:19:13.757Z 
   
   So, with the primary key `((id), keyspace, table)`, the advantage is that we would be able to visually see what all tables were snapshotted in that one logical snapshot based on the same timestamp. Here, we see, from the timestamp, that ks.tb and system.local were snapshotted "together".
   
   So from this perspective it is better if snapshot id is partition key.
   
   I would go so far to include timestamp into primary key as well: `((id), keyspace, table, timestamp)`
   
   This way we would have them ordered too and it does not need to be specified when querying.
   
   If we made it like already suggested: (keyspace, table, snapshotid), we would lose the information, it might be like:
   
       ks1 tb1 snapshot1 2023-03-02T13:19:42.757Z        
       ks3 tb2 snapshot1 2023-03-02T13:19:13.757Z 
       ks3 tb3 snapshot1 2023-03-02T13:19:42.757Z 
       ks4 tb4 snapshot1 2023-03-02T13:19:13.757Z 
   
   But here it is not so simple to see that "ks4.tb4" and "ks3.tb2" are forming one logical snapshot and then "ks1.tb1" and "ks3.tb3" another.
   
   So, utimately, it should be like this:
   
   ((id), ts, ks, tb))
   
   Timestamp has to go first to have it ordered, same timestamp for some snapshot id would mean that all keyspaces and tables belong to that particular snapshot name which was taken at that exact time.
   
       cqlsh> select * from ks.test ;
        id | ts | ks | tb
       ----+----+----+----
         1 |  2 |  3 |  4      first snapshot
         1 |  2 |  3 |  5      first snapshot
         1 |  2 |  3 |  6      first snapshot
         1 |  3 |  1 |  2      second snapshot
         1 |  3 |  1 |  3      second snapshot
         2 |  5 |  2 |  6      third snapshot
   



-- 
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 #2117: Add a virtual table to list snapshots for CASSANDRA-18102

Posted by "smiklosovic (via GitHub)" <gi...@apache.org>.
smiklosovic commented on code in PR #2117:
URL: https://github.com/apache/cassandra/pull/2117#discussion_r1101474477


##########
src/java/org/apache/cassandra/db/virtual/SnapshotsTable.java:
##########
@@ -0,0 +1,93 @@
+/*
+ * 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.virtual;
+
+import java.util.List;
+import java.util.Map;
+import java.util.Set;
+import javax.management.openmbean.TabularData;
+import javax.management.openmbean.TabularDataSupport;
+import com.google.common.collect.ImmutableMap;
+
+import org.apache.cassandra.db.SnapshotDetailsTabularData;
+import org.apache.cassandra.db.marshal.BooleanType;
+import org.apache.cassandra.db.marshal.LongType;
+import org.apache.cassandra.db.marshal.UTF8Type;
+import org.apache.cassandra.dht.LocalPartitioner;
+import org.apache.cassandra.schema.TableMetadata;
+import org.apache.cassandra.service.StorageService;
+
+public class SnapshotsTable extends AbstractVirtualTable
+{
+    private static final String SNAPSHOT_NAME = "snapshot_name";
+    private static final String KEYSPACE_NAME = "keyspace_name";
+    private static final String COLUMNFAMILY_NAME = "columnfamily_name";
+    private static final String TRUE_SIZE = "true_size";
+    private static final String SIZE_ON_DISK = "size_on_disk";
+    private static final String CREATE_TIME = "created_at";
+    private static final String EXPIRATION_TIME = "expires_at";
+    private static final String EPHEMERAL = "ephemeral";
+    
+    SnapshotsTable(String keyspace)
+    {
+        super(TableMetadata.builder(keyspace, "snapshots")
+                           .comment("tables snapshots")
+                           .kind(TableMetadata.Kind.VIRTUAL)
+                           .partitioner(new LocalPartitioner(UTF8Type.instance))
+                           .addPartitionKeyColumn(SNAPSHOT_NAME, UTF8Type.instance)

Review Comment:
   @pauloricardomg what do you think about column names? Should we follow the naming in nodetool's output? I think we should not be tied by 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 #2117: Add a virtual table to list snapshots for CASSANDRA-18102

Posted by "smiklosovic (via GitHub)" <gi...@apache.org>.
smiklosovic commented on code in PR #2117:
URL: https://github.com/apache/cassandra/pull/2117#discussion_r1123076202


##########
src/java/org/apache/cassandra/db/virtual/SnapshotsTable.java:
##########
@@ -0,0 +1,93 @@
+/*
+ * 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.virtual;
+
+import java.util.List;
+import java.util.Map;
+import java.util.Set;
+import javax.management.openmbean.TabularData;
+import javax.management.openmbean.TabularDataSupport;
+import com.google.common.collect.ImmutableMap;
+
+import org.apache.cassandra.db.SnapshotDetailsTabularData;
+import org.apache.cassandra.db.marshal.BooleanType;
+import org.apache.cassandra.db.marshal.LongType;
+import org.apache.cassandra.db.marshal.UTF8Type;
+import org.apache.cassandra.dht.LocalPartitioner;
+import org.apache.cassandra.schema.TableMetadata;
+import org.apache.cassandra.service.StorageService;
+
+public class SnapshotsTable extends AbstractVirtualTable
+{
+    private static final String SNAPSHOT_NAME = "snapshot_name";
+    private static final String KEYSPACE_NAME = "keyspace_name";
+    private static final String COLUMNFAMILY_NAME = "columnfamily_name";
+    private static final String TRUE_SIZE = "true_size";
+    private static final String SIZE_ON_DISK = "size_on_disk";
+    private static final String CREATE_TIME = "created_at";
+    private static final String EXPIRATION_TIME = "expires_at";
+    private static final String EPHEMERAL = "ephemeral";
+    
+    SnapshotsTable(String keyspace)
+    {
+        super(TableMetadata.builder(keyspace, "snapshots")
+                           .comment("tables snapshots")
+                           .kind(TableMetadata.Kind.VIRTUAL)
+                           .partitioner(new LocalPartitioner(UTF8Type.instance))
+                           .addPartitionKeyColumn(SNAPSHOT_NAME, UTF8Type.instance)

Review Comment:
   @pauloricardomg 
   
   Regardless of whether we can query without allow filtering after 18238 I think it is still a good practice to act as we would normally do when modeling the schema.
   
   As explained above and what Maxwell just explained, I think keyspace > table > snapshot id.
   
   We can have same snapshot names after all, no? So if keyspace and table are clustering columns, with two snapshots of the same name, we would have this partition:
   
       snapshotName1 | keyspace1 | table1
       snapshotName1 | keyspace2 | table2
   
   primary key would be `((snapshotName), keyspaceName, tableName)`
   
   Is not this counter-intuitive to have a partition which is logically coupling different keyspaces and names under the same snapshot name? That does not make sense to me. 
   
   However, on the other hand, it is possible to do this:
   
       ./bin/nodetool snapshot --kt-list ks.tb,system.local -t mysnapshot
   
   So `listsnapshots` will do this:
   
       mysnapshot  system  local  1.16 KiB  21.47 KiB    2023-03-02T13:19:42.140Z
       mysnapshot  ks      tb     1.02 KiB   6.08 KiB    2023-03-02T13:19:13.757Z
   
   But then I can do this as well, again:
   
       ./bin/nodetool snapshot --kt-list ks.tb2 -t mysnapshot
   
   Which would print it like:
   
       mysnapshot     ks       tb2        1.16 KiB  21.47 KiB    2023-03-02T13:19:42.140Z                
       mysnapshot     ks       tb         1.02 KiB  6.08 KiB     2023-03-02T13:19:13.757Z                
       mysnapshot     system   local      107 bytes 6.98 KiB     2023-03-02T13:19:13.757Z 
   
   So, with the primary key `((id), keyspace, table)`, the advantage is that we would be able to visually see what all tables were snapshotted in that one logical snapshot based on the same timestamp. Here, we see, from the timestamp, that ks.tb and system.local were snapshotted "together".
   
   So from this perspective it is better if snapshot id is partition key.
   
   I would go so far to include timestamp into primary key as well: `((id), keyspace, table, timestamp)`
   
   This way we would have them ordered too and it does not need to be specified when querying.
   
   If we made it like already suggested: (keyspace, table, snapshotid), we would lose the information, it might be like:
   
       ks1 tb1 snapshot1 2023-03-02T13:19:42.757Z        
       ks3 tb2 snapshot1 2023-03-02T13:19:13.757Z 
       ks3 tb3 snapshot1 2023-03-02T13:19:42.757Z 
       ks4 tb4 snapshot1 2023-03-02T13:19:13.757Z 
   
   But here it is not so simple to see that "ks4.tb4" and "ks3.tb2" are forming one logical snapshot and then "ks1.tb1" and "ks3.tb3" another.
   
   So, utimately, it should be like this:
   
   ((id), ts, ks, tb))
   
   Timestamp has to go first to have it ordered, same timestamp for some snapshot id would mean that all keyspaces and tables belong to that particular snapshot name which was taken at that exact time.
   
   cqlsh> select * from ks.test ;
   
        id | ts | ks | tb
       ----+----+----+----
         1 |  2 |  3 |  4      first snapshot
         1 |  2 |  3 |  5      first snapshot
         1 |  2 |  3 |  6      first snapshot
         1 |  3 |  1 |  2      second snapshot
         1 |  3 |  1 |  3      second snapshot
         2 |  5 |  2 |  6      third snapshot
   



-- 
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] Maxwell-Guo commented on a diff in pull request #2117: Add a virtual table to list snapshots for CASSANDRA-18102

Posted by "Maxwell-Guo (via GitHub)" <gi...@apache.org>.
Maxwell-Guo commented on code in PR #2117:
URL: https://github.com/apache/cassandra/pull/2117#discussion_r1130314345


##########
test/unit/org/apache/cassandra/db/virtual/SnapshotsTableTest.java:
##########
@@ -0,0 +1,103 @@
+/*
+ * 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.virtual;
+
+import java.time.Instant;
+
+import com.google.common.collect.ImmutableList;
+import org.junit.After;
+import org.junit.Before;
+import org.junit.BeforeClass;
+import org.junit.Test;
+
+import org.apache.cassandra.config.DurationSpec;
+import org.apache.cassandra.cql3.CQLTester;
+import org.apache.cassandra.cql3.UntypedResultSet;
+
+public class SnapshotsTableTest extends CQLTester
+{
+    private static final String KS_NAME = "vts";
+    private final String SNAPSHOT_TTL = "snapshotTtl";
+    private final String SNAPSHOT_NO_TTL = "snapshotNoTtl";
+    private final String SNAPSHOT_EPHEMERAL = "snapshotEphemeral";
+    private final DurationSpec.IntSecondsBound ttl = new DurationSpec.IntSecondsBound ("4h");
+    
+    @BeforeClass
+    public static void setUpClass()
+    {
+        CQLTester.setUpClass();
+    }
+
+    @Before
+    public void before() throws Throwable
+    {
+        SnapshotsTable table = new SnapshotsTable(KS_NAME);
+        VirtualKeyspaceRegistry.instance.register(new VirtualKeyspace(KS_NAME, ImmutableList.of(table)));
+        
+        createTable("CREATE TABLE %s (pk int, ck int, PRIMARY KEY (pk, ck))");
+        for (int i = 0; i != 10; ++i)
+        {
+            execute("INSERT INTO %s (pk, ck) VALUES (?, ?)", i, i);
+        }
+        flush();
+    }
+    
+    @After
+    public void after()
+    {
+        clearSnapshot(SNAPSHOT_NO_TTL, currentKeyspace());
+        clearSnapshot(SNAPSHOT_TTL, currentKeyspace());
+        clearSnapshot(SNAPSHOT_EPHEMERAL, currentKeyspace());
+        dropTable("DROP TABLE %s");
+    }
+   
+    @Test
+    public void testSnapshots() throws Throwable
+    {
+        Instant createTime = Instant.now();
+        String createTimeStr = createTime.toString();
+        snapshot(SNAPSHOT_NO_TTL, createTime);
+        snapshot(SNAPSHOT_TTL, ttl, createTime);
+        snapshot(SNAPSHOT_EPHEMERAL, true, null, createTime);
+
+        // query all from snapshots virtual table
+        UntypedResultSet result = execute("SELECT id, keyspace_name, table_name, created_at, expires_at, ephemeral FROM vts.snapshots");
+        assertRows(result,
+                row(SNAPSHOT_EPHEMERAL, CQLTester.KEYSPACE, currentTable(), createTimeStr, null, true),
+                row(SNAPSHOT_NO_TTL, CQLTester.KEYSPACE, currentTable(), createTimeStr, null, false),
+                row(SNAPSHOT_TTL, CQLTester.KEYSPACE, currentTable(), createTimeStr, createTime.plusSeconds(ttl.toSeconds()).toString(), false));
+
+        // query with conditions
+        result = execute("SELECT id, keyspace_name, table_name, created_at, expires_at, ephemeral FROM vts.snapshots where ephemeral = true allow filtering");

Review Comment:
   It seems that I should pull to get the newest code



-- 
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] Maxwell-Guo commented on a diff in pull request #2117: Add a virtual table to list snapshots for CASSANDRA-18102

Posted by "Maxwell-Guo (via GitHub)" <gi...@apache.org>.
Maxwell-Guo commented on code in PR #2117:
URL: https://github.com/apache/cassandra/pull/2117#discussion_r1118291641


##########
src/java/org/apache/cassandra/db/virtual/SnapshotsTable.java:
##########
@@ -0,0 +1,93 @@
+/*
+ * 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.virtual;
+
+import java.util.List;
+import java.util.Map;
+import java.util.Set;
+import javax.management.openmbean.TabularData;
+import javax.management.openmbean.TabularDataSupport;
+import com.google.common.collect.ImmutableMap;
+
+import org.apache.cassandra.db.SnapshotDetailsTabularData;
+import org.apache.cassandra.db.marshal.BooleanType;
+import org.apache.cassandra.db.marshal.LongType;
+import org.apache.cassandra.db.marshal.UTF8Type;
+import org.apache.cassandra.dht.LocalPartitioner;
+import org.apache.cassandra.schema.TableMetadata;
+import org.apache.cassandra.service.StorageService;
+
+public class SnapshotsTable extends AbstractVirtualTable
+{
+    private static final String SNAPSHOT_NAME = "snapshot_name";
+    private static final String KEYSPACE_NAME = "keyspace_name";
+    private static final String COLUMNFAMILY_NAME = "columnfamily_name";
+    private static final String TRUE_SIZE = "true_size";
+    private static final String SIZE_ON_DISK = "size_on_disk";
+    private static final String CREATE_TIME = "created_at";
+    private static final String EXPIRATION_TIME = "expires_at";
+    private static final String EPHEMERAL = "ephemeral";
+    
+    SnapshotsTable(String keyspace)
+    {
+        super(TableMetadata.builder(keyspace, "snapshots")
+                           .comment("tables snapshots")
+                           .kind(TableMetadata.Kind.VIRTUAL)
+                           .partitioner(new LocalPartitioner(UTF8Type.instance))
+                           .addPartitionKeyColumn(SNAPSHOT_NAME, UTF8Type.instance)

Review Comment:
   Hi @pauloricardomg  and @smiklosovic 
   I found that the word keyspace is a reserved word for cql 
   https://github.com/apache/cassandra/blob/trunk/src/antlr/Parser.g#L1898
   So and error will occurs when doing select when specified keyspace = xxx or select keyspace  where xxx
   I think that's why other system table using keyspace_name instand of keyspace
   https://github.com/apache/cassandra/blob/trunk/src/java/org/apache/cassandra/db/SystemKeyspace.java#L238
   
   And I am ok if we change snapshot_name from name to tag, And i have already finish this.
   



-- 
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] pauloricardomg commented on a diff in pull request #2117: Add a virtual table to list snapshots for CASSANDRA-18102

Posted by "pauloricardomg (via GitHub)" <gi...@apache.org>.
pauloricardomg commented on code in PR #2117:
URL: https://github.com/apache/cassandra/pull/2117#discussion_r1118119607


##########
src/java/org/apache/cassandra/db/virtual/SnapshotsTable.java:
##########
@@ -0,0 +1,93 @@
+/*
+ * 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.virtual;
+
+import java.util.List;
+import java.util.Map;
+import java.util.Set;
+import javax.management.openmbean.TabularData;
+import javax.management.openmbean.TabularDataSupport;
+import com.google.common.collect.ImmutableMap;
+
+import org.apache.cassandra.db.SnapshotDetailsTabularData;
+import org.apache.cassandra.db.marshal.BooleanType;
+import org.apache.cassandra.db.marshal.LongType;
+import org.apache.cassandra.db.marshal.UTF8Type;
+import org.apache.cassandra.dht.LocalPartitioner;
+import org.apache.cassandra.schema.TableMetadata;
+import org.apache.cassandra.service.StorageService;
+
+public class SnapshotsTable extends AbstractVirtualTable
+{
+    private static final String SNAPSHOT_NAME = "snapshot_name";
+    private static final String KEYSPACE_NAME = "keyspace_name";
+    private static final String COLUMNFAMILY_NAME = "columnfamily_name";
+    private static final String TRUE_SIZE = "true_size";
+    private static final String SIZE_ON_DISK = "size_on_disk";
+    private static final String CREATE_TIME = "created_at";
+    private static final String EXPIRATION_TIME = "expires_at";
+    private static final String EPHEMERAL = "ephemeral";
+    
+    SnapshotsTable(String keyspace)
+    {
+        super(TableMetadata.builder(keyspace, "snapshots")
+                           .comment("tables snapshots")
+                           .kind(TableMetadata.Kind.VIRTUAL)
+                           .partitioner(new LocalPartitioner(UTF8Type.instance))
+                           .addPartitionKeyColumn(SNAPSHOT_NAME, UTF8Type.instance)

Review Comment:
   Sorry for the delay - have been busy in the past couple of weeks but will try to be faster on next iterations so we can wrap this up soon.
   
   I agree with with @smiklosovic comment about using more succint column names (ie. `keyspace` vs `keyspace_name` and `table` vs `columnfamily_name`), but I think we should use the column name "tag" and not "name" to refer to a snapshot identifier, and keep this as table partition key. We should enable allow filtering on this virtual table (with [CASSANDRA-18271](https://issues.apache.org/jira/browse/CASSANDRA-18271) to allow querying by non partition keys (ie. `keyspace` or `table`). This would allow the following queries:
   
   ```sql
   use system_views;
   select * from snapshots;
   select * from snapshots where tag = "mysnapshot" # query by partition key
   select * from snapshots where keyspace = 'ks1'; # query by non-partition keys
   select * from snapshots where keyspace = 'ks1' and table = 'tb1';
   select * from snapshots where keyspace = 'ks1' and table = 'tb1' and tag = "mysnapshot";
   ```
   
   What do you think?
   



-- 
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 #2117: Add a virtual table to list snapshots for CASSANDRA-18102

Posted by "smiklosovic (via GitHub)" <gi...@apache.org>.
smiklosovic commented on code in PR #2117:
URL: https://github.com/apache/cassandra/pull/2117#discussion_r1123076202


##########
src/java/org/apache/cassandra/db/virtual/SnapshotsTable.java:
##########
@@ -0,0 +1,93 @@
+/*
+ * 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.virtual;
+
+import java.util.List;
+import java.util.Map;
+import java.util.Set;
+import javax.management.openmbean.TabularData;
+import javax.management.openmbean.TabularDataSupport;
+import com.google.common.collect.ImmutableMap;
+
+import org.apache.cassandra.db.SnapshotDetailsTabularData;
+import org.apache.cassandra.db.marshal.BooleanType;
+import org.apache.cassandra.db.marshal.LongType;
+import org.apache.cassandra.db.marshal.UTF8Type;
+import org.apache.cassandra.dht.LocalPartitioner;
+import org.apache.cassandra.schema.TableMetadata;
+import org.apache.cassandra.service.StorageService;
+
+public class SnapshotsTable extends AbstractVirtualTable
+{
+    private static final String SNAPSHOT_NAME = "snapshot_name";
+    private static final String KEYSPACE_NAME = "keyspace_name";
+    private static final String COLUMNFAMILY_NAME = "columnfamily_name";
+    private static final String TRUE_SIZE = "true_size";
+    private static final String SIZE_ON_DISK = "size_on_disk";
+    private static final String CREATE_TIME = "created_at";
+    private static final String EXPIRATION_TIME = "expires_at";
+    private static final String EPHEMERAL = "ephemeral";
+    
+    SnapshotsTable(String keyspace)
+    {
+        super(TableMetadata.builder(keyspace, "snapshots")
+                           .comment("tables snapshots")
+                           .kind(TableMetadata.Kind.VIRTUAL)
+                           .partitioner(new LocalPartitioner(UTF8Type.instance))
+                           .addPartitionKeyColumn(SNAPSHOT_NAME, UTF8Type.instance)

Review Comment:
   @pauloricardomg 
   
   Regardless of whether we can query without allow filtering after 18238 I think it is still a good practice to act as we would normally do when modeling the schema.
   
   As explained above and what Maxwell just explained, I think keyspace > table > snapshot id.
   
   We can have same snapshot names after all, no? So if keyspace and table are clustering columns, with two snapshots of the same name, we would have this partition:
   
       snapshotName1 | keyspace1 | table1
       snapshotName1 | keyspace2 | table2
   
   primary key would be `(snapshotName, (keyspaceName, tableName))`
   
   Is not this counter-intuitive to have a partition which is logically coupling different keyspaces and names under the same snapshot name? That does not make sense to me. 
   
   However, on the other hand, it is possible to do this:
   
       ./bin/nodetool snapshot --kt-list ks.tb,system.local -t mysnapshot
   
   So `listsnapshots` will do this:
   
       mysnapshot  system  local  1.16 KiB  21.47 KiB    2023-03-02T13:19:42.140Z
       mysnapshot  ks      tb     1.02 KiB   6.08 KiB    2023-03-02T13:19:13.757Z
   
   But then I can do this as well, again:
   
       ./bin/nodetool snapshot --kt-list ks.tb2 -t mysnapshot
   
   Which would print it like:
   
       mysnapshot     ks       tb2        1.16 KiB  21.47 KiB    2023-03-02T13:19:42.140Z                
       mysnapshot     ks       tb         1.02 KiB  6.08 KiB     2023-03-02T13:19:13.757Z                
       mysnapshot     system   local      107 bytes 6.98 KiB     2023-03-02T13:19:13.757Z 
   
   So, with the primary key `(id, (keyspace, table))`, the advantage is that we would be able to visually see what all tables were snapshotted in that one logical snapshot based on the very similar timestamp. Here, we see, from the timestamp, that ks.tb and system.local were snapshotted "together".
   
   So from this perspective it is better if snapshot id is partition key.
   
   I would go so far to include timestamp into primary key as well: `(id, (keyspace, table, timestamp))`
   
   This way we would have them ordered too and it does not need to be specified when querying.
   
   If we made it like I suggested: (keyspace, table, snapshotid), we would lose the information, it migth be like:
   
       ks1 tb1 snapshot1 2023-03-02T13:19:42.140Z        
       ks4 tb4 snapshot1 2023-03-02T13:19:13.757Z 
       ks3 tb2 snapshot1 2023-03-02T13:19:13.757Z 
       ks3 tb3 snapshot1 2023-03-02T13:19:42.757Z 
   
   But here it is not so simple to see that "ks4.tb4" and "ks3.tb2" are forming one logical snapshot and then "ks1.tb1" and "ks3.tb3" another.



-- 
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] pauloricardomg commented on a diff in pull request #2117: Add a virtual table to list snapshots for CASSANDRA-18102

Posted by "pauloricardomg (via GitHub)" <gi...@apache.org>.
pauloricardomg commented on code in PR #2117:
URL: https://github.com/apache/cassandra/pull/2117#discussion_r1122368486


##########
src/java/org/apache/cassandra/db/virtual/SnapshotsTable.java:
##########
@@ -0,0 +1,93 @@
+/*
+ * 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.virtual;
+
+import java.util.List;
+import java.util.Map;
+import java.util.Set;
+import javax.management.openmbean.TabularData;
+import javax.management.openmbean.TabularDataSupport;
+import com.google.common.collect.ImmutableMap;
+
+import org.apache.cassandra.db.SnapshotDetailsTabularData;
+import org.apache.cassandra.db.marshal.BooleanType;
+import org.apache.cassandra.db.marshal.LongType;
+import org.apache.cassandra.db.marshal.UTF8Type;
+import org.apache.cassandra.dht.LocalPartitioner;
+import org.apache.cassandra.schema.TableMetadata;
+import org.apache.cassandra.service.StorageService;
+
+public class SnapshotsTable extends AbstractVirtualTable
+{
+    private static final String SNAPSHOT_NAME = "snapshot_name";
+    private static final String KEYSPACE_NAME = "keyspace_name";
+    private static final String COLUMNFAMILY_NAME = "columnfamily_name";
+    private static final String TRUE_SIZE = "true_size";
+    private static final String SIZE_ON_DISK = "size_on_disk";
+    private static final String CREATE_TIME = "created_at";
+    private static final String EXPIRATION_TIME = "expires_at";
+    private static final String EPHEMERAL = "ephemeral";
+    
+    SnapshotsTable(String keyspace)
+    {
+        super(TableMetadata.builder(keyspace, "snapshots")
+                           .comment("tables snapshots")
+                           .kind(TableMetadata.Kind.VIRTUAL)
+                           .partitioner(new LocalPartitioner(UTF8Type.instance))
+                           .addPartitionKeyColumn(SNAPSHOT_NAME, UTF8Type.instance)

Review Comment:
   > Partition key would be keyspace_name and clustering columns would be table_name and id of the snapshot.
   
   Why do you think they should be primary keys?  I think primary key of snapshot should be `id`, since this is the primary snapshot identifier (ie. different keyspaces cannot have the same snapshot id). It's still possible to query by `keyspace_name` and `table_name` if needed after CASSANDRA-18238.



-- 
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] Maxwell-Guo commented on a diff in pull request #2117: Add a virtual table to list snapshots for CASSANDRA-18102

Posted by "Maxwell-Guo (via GitHub)" <gi...@apache.org>.
Maxwell-Guo commented on code in PR #2117:
URL: https://github.com/apache/cassandra/pull/2117#discussion_r1128857477


##########
src/java/org/apache/cassandra/db/virtual/SnapshotsTable.java:
##########
@@ -0,0 +1,93 @@
+/*
+ * 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.virtual;
+
+import java.util.List;
+import java.util.Map;
+import java.util.Set;
+import javax.management.openmbean.TabularData;
+import javax.management.openmbean.TabularDataSupport;
+import com.google.common.collect.ImmutableMap;
+
+import org.apache.cassandra.db.SnapshotDetailsTabularData;
+import org.apache.cassandra.db.marshal.BooleanType;
+import org.apache.cassandra.db.marshal.LongType;
+import org.apache.cassandra.db.marshal.UTF8Type;
+import org.apache.cassandra.dht.LocalPartitioner;
+import org.apache.cassandra.schema.TableMetadata;
+import org.apache.cassandra.service.StorageService;
+
+public class SnapshotsTable extends AbstractVirtualTable
+{
+    private static final String SNAPSHOT_NAME = "snapshot_name";
+    private static final String KEYSPACE_NAME = "keyspace_name";
+    private static final String COLUMNFAMILY_NAME = "columnfamily_name";
+    private static final String TRUE_SIZE = "true_size";
+    private static final String SIZE_ON_DISK = "size_on_disk";
+    private static final String CREATE_TIME = "created_at";
+    private static final String EXPIRATION_TIME = "expires_at";
+    private static final String EPHEMERAL = "ephemeral";
+    
+    SnapshotsTable(String keyspace)
+    {
+        super(TableMetadata.builder(keyspace, "snapshots")
+                           .comment("tables snapshots")
+                           .kind(TableMetadata.Kind.VIRTUAL)
+                           .partitioner(new LocalPartitioner(UTF8Type.instance))
+                           .addPartitionKeyColumn(SNAPSHOT_NAME, UTF8Type.instance)

Review Comment:
   thanks @smiklosovic @pauloricardomg , I will fix them later today. 



-- 
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] Maxwell-Guo commented on a diff in pull request #2117: Add a virtual table to list snapshots for CASSANDRA-18102

Posted by "Maxwell-Guo (via GitHub)" <gi...@apache.org>.
Maxwell-Guo commented on code in PR #2117:
URL: https://github.com/apache/cassandra/pull/2117#discussion_r1129017646


##########
test/unit/org/apache/cassandra/db/virtual/SnapshotsTableTest.java:
##########
@@ -0,0 +1,103 @@
+/*
+ * 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.virtual;
+
+import java.time.Instant;
+
+import com.google.common.collect.ImmutableList;
+import org.junit.After;
+import org.junit.Before;
+import org.junit.BeforeClass;
+import org.junit.Test;
+
+import org.apache.cassandra.config.DurationSpec;
+import org.apache.cassandra.cql3.CQLTester;
+import org.apache.cassandra.cql3.UntypedResultSet;
+
+public class SnapshotsTableTest extends CQLTester
+{
+    private static final String KS_NAME = "vts";
+    private final String SNAPSHOT_TTL = "snapshotTtl";
+    private final String SNAPSHOT_NO_TTL = "snapshotNoTtl";
+    private final String SNAPSHOT_EPHEMERAL = "snapshotEphemeral";
+    private final DurationSpec.IntSecondsBound ttl = new DurationSpec.IntSecondsBound ("4h");
+    
+    @BeforeClass
+    public static void setUpClass()
+    {
+        CQLTester.setUpClass();
+    }
+
+    @Before
+    public void before() throws Throwable
+    {
+        SnapshotsTable table = new SnapshotsTable(KS_NAME);
+        VirtualKeyspaceRegistry.instance.register(new VirtualKeyspace(KS_NAME, ImmutableList.of(table)));
+        
+        createTable("CREATE TABLE %s (pk int, ck int, PRIMARY KEY (pk, ck))");
+        for (int i = 0; i != 10; ++i)
+        {
+            execute("INSERT INTO %s (pk, ck) VALUES (?, ?)", i, i);
+        }
+        flush();
+    }
+    
+    @After
+    public void after()
+    {
+        clearSnapshot(SNAPSHOT_NO_TTL, currentKeyspace());
+        clearSnapshot(SNAPSHOT_TTL, currentKeyspace());
+        clearSnapshot(SNAPSHOT_EPHEMERAL, currentKeyspace());
+        dropTable("DROP TABLE %s");
+    }
+   
+    @Test
+    public void testSnapshots() throws Throwable
+    {
+        Instant createTime = Instant.now();
+        String createTimeStr = createTime.toString();
+        snapshot(SNAPSHOT_NO_TTL, createTime);
+        snapshot(SNAPSHOT_TTL, ttl, createTime);
+        snapshot(SNAPSHOT_EPHEMERAL, true, null, createTime);
+
+        // query all from snapshots virtual table
+        UntypedResultSet result = execute("SELECT id, keyspace_name, table_name, created_at, expires_at, ephemeral FROM vts.snapshots");
+        assertRows(result,
+                row(SNAPSHOT_EPHEMERAL, CQLTester.KEYSPACE, currentTable(), createTimeStr, null, true),
+                row(SNAPSHOT_NO_TTL, CQLTester.KEYSPACE, currentTable(), createTimeStr, null, false),
+                row(SNAPSHOT_TTL, CQLTester.KEYSPACE, currentTable(), createTimeStr, createTime.plusSeconds(ttl.toSeconds()).toString(), false));
+
+        // query with conditions
+        result = execute("SELECT id, keyspace_name, table_name, created_at, expires_at, ephemeral FROM vts.snapshots where ephemeral = true allow filtering");

Review Comment:
   It's needed  here when  whereclause is ephemeral = true , ephemeral is not partition key or clustering key



-- 
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 #2117: Add a virtual table to list snapshots for CASSANDRA-18102

Posted by "smiklosovic (via GitHub)" <gi...@apache.org>.
smiklosovic commented on code in PR #2117:
URL: https://github.com/apache/cassandra/pull/2117#discussion_r1127997518


##########
src/java/org/apache/cassandra/db/virtual/SnapshotsTable.java:
##########
@@ -0,0 +1,93 @@
+/*
+ * 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.virtual;
+
+import java.util.List;
+import java.util.Map;
+import java.util.Set;
+import javax.management.openmbean.TabularData;
+import javax.management.openmbean.TabularDataSupport;
+import com.google.common.collect.ImmutableMap;
+
+import org.apache.cassandra.db.SnapshotDetailsTabularData;
+import org.apache.cassandra.db.marshal.BooleanType;
+import org.apache.cassandra.db.marshal.LongType;
+import org.apache.cassandra.db.marshal.UTF8Type;
+import org.apache.cassandra.dht.LocalPartitioner;
+import org.apache.cassandra.schema.TableMetadata;
+import org.apache.cassandra.service.StorageService;
+
+public class SnapshotsTable extends AbstractVirtualTable
+{
+    private static final String SNAPSHOT_NAME = "snapshot_name";
+    private static final String KEYSPACE_NAME = "keyspace_name";
+    private static final String COLUMNFAMILY_NAME = "columnfamily_name";
+    private static final String TRUE_SIZE = "true_size";
+    private static final String SIZE_ON_DISK = "size_on_disk";
+    private static final String CREATE_TIME = "created_at";
+    private static final String EXPIRATION_TIME = "expires_at";
+    private static final String EPHEMERAL = "ephemeral";
+    
+    SnapshotsTable(String keyspace)
+    {
+        super(TableMetadata.builder(keyspace, "snapshots")
+                           .comment("tables snapshots")
+                           .kind(TableMetadata.Kind.VIRTUAL)
+                           .partitioner(new LocalPartitioner(UTF8Type.instance))
+                           .addPartitionKeyColumn(SNAPSHOT_NAME, UTF8Type.instance)

Review Comment:
   @pauloricardomg  based on the thread in ML it seems like we are ok with what is here. I put some more comments to PR, @Maxwell-Guo , could you please address them? So we might approach the CI.



-- 
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 #2117: Add a virtual table to list snapshots for CASSANDRA-18102

Posted by "smiklosovic (via GitHub)" <gi...@apache.org>.
smiklosovic commented on code in PR #2117:
URL: https://github.com/apache/cassandra/pull/2117#discussion_r1097526285


##########
src/java/org/apache/cassandra/db/virtual/SnapshotsTable.java:
##########
@@ -0,0 +1,93 @@
+/*
+ * 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.virtual;
+
+import java.util.List;
+import java.util.Map;
+import java.util.Set;
+import javax.management.openmbean.TabularData;
+import javax.management.openmbean.TabularDataSupport;
+import com.google.common.collect.ImmutableMap;
+
+import org.apache.cassandra.db.SnapshotDetailsTabularData;
+import org.apache.cassandra.db.marshal.BooleanType;
+import org.apache.cassandra.db.marshal.LongType;
+import org.apache.cassandra.db.marshal.UTF8Type;
+import org.apache.cassandra.dht.LocalPartitioner;
+import org.apache.cassandra.schema.TableMetadata;
+import org.apache.cassandra.service.StorageService;
+
+public class SnapshotsTable extends AbstractVirtualTable
+{
+    private static final String SNAPSHOT_NAME = "snapshot_name";
+    private static final String KEYSPACE_NAME = "keyspace_name";
+    private static final String COLUMNFAMILY_NAME = "columnfamily_name";
+    private static final String TRUE_SIZE = "true_size";
+    private static final String SIZE_ON_DISK = "size_on_disk";
+    private static final String CREATE_TIME = "created_at";
+    private static final String EXPIRATION_TIME = "expires_at";
+    private static final String EPHEMERAL = "ephemeral";
+    
+    SnapshotsTable(String keyspace)
+    {
+        super(TableMetadata.builder(keyspace, "snapshots")
+                           .comment("tables snapshots")
+                           .kind(TableMetadata.Kind.VIRTUAL)
+                           .partitioner(new LocalPartitioner(UTF8Type.instance))
+                           .addPartitionKeyColumn(SNAPSHOT_NAME, UTF8Type.instance)

Review Comment:
   hi @Maxwell-Guo , any chance to rework this as discussed above?



-- 
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 #2117: Add a virtual table to list snapshots for CASSANDRA-18102

Posted by "smiklosovic (via GitHub)" <gi...@apache.org>.
smiklosovic commented on code in PR #2117:
URL: https://github.com/apache/cassandra/pull/2117#discussion_r1123076202


##########
src/java/org/apache/cassandra/db/virtual/SnapshotsTable.java:
##########
@@ -0,0 +1,93 @@
+/*
+ * 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.virtual;
+
+import java.util.List;
+import java.util.Map;
+import java.util.Set;
+import javax.management.openmbean.TabularData;
+import javax.management.openmbean.TabularDataSupport;
+import com.google.common.collect.ImmutableMap;
+
+import org.apache.cassandra.db.SnapshotDetailsTabularData;
+import org.apache.cassandra.db.marshal.BooleanType;
+import org.apache.cassandra.db.marshal.LongType;
+import org.apache.cassandra.db.marshal.UTF8Type;
+import org.apache.cassandra.dht.LocalPartitioner;
+import org.apache.cassandra.schema.TableMetadata;
+import org.apache.cassandra.service.StorageService;
+
+public class SnapshotsTable extends AbstractVirtualTable
+{
+    private static final String SNAPSHOT_NAME = "snapshot_name";
+    private static final String KEYSPACE_NAME = "keyspace_name";
+    private static final String COLUMNFAMILY_NAME = "columnfamily_name";
+    private static final String TRUE_SIZE = "true_size";
+    private static final String SIZE_ON_DISK = "size_on_disk";
+    private static final String CREATE_TIME = "created_at";
+    private static final String EXPIRATION_TIME = "expires_at";
+    private static final String EPHEMERAL = "ephemeral";
+    
+    SnapshotsTable(String keyspace)
+    {
+        super(TableMetadata.builder(keyspace, "snapshots")
+                           .comment("tables snapshots")
+                           .kind(TableMetadata.Kind.VIRTUAL)
+                           .partitioner(new LocalPartitioner(UTF8Type.instance))
+                           .addPartitionKeyColumn(SNAPSHOT_NAME, UTF8Type.instance)

Review Comment:
   @pauloricardomg 
   
   Regardless of whether we can query without allow filtering after 18238 I think it is still a good practice to act as we would normally do when modeling the schema.
   
   As explained above and what Maxwell just explained, I think keyspace > table > snapshot id.
   
   We can have same snapshot names after all, no? So if keyspace and table are clustering columns, with two snapshots of the same name, we would have this partition:
   
       snapshotName1 | keyspace1 | table1
       snapshotName1 | keyspace2 | table2
   
   primary key would be `((snapshotName), keyspaceName, tableName)`
   
   Is not this counter-intuitive to have a partition which is logically coupling different keyspaces and names under the same snapshot name? That does not make sense to me. 
   
   However, on the other hand, it is possible to do this:
   
       ./bin/nodetool snapshot --kt-list ks.tb,system.local -t mysnapshot
   
   So `listsnapshots` will do this:
   
       mysnapshot  system  local  1.16 KiB  21.47 KiB    2023-03-02T13:19:42.140Z
       mysnapshot  ks      tb     1.02 KiB   6.08 KiB    2023-03-02T13:19:13.757Z
   
   But then I can do this as well, again:
   
       ./bin/nodetool snapshot --kt-list ks.tb2 -t mysnapshot
   
   Which would print it like:
   
       mysnapshot     ks       tb2        1.16 KiB  21.47 KiB    2023-03-02T13:19:42.140Z                
       mysnapshot     ks       tb         1.02 KiB  6.08 KiB     2023-03-02T13:19:13.757Z                
       mysnapshot     system   local      107 bytes 6.98 KiB     2023-03-02T13:19:13.757Z 
   
   So, with the primary key `((id), keyspace, table)`, the advantage is that we would be able to visually see what all tables were snapshotted in that one logical snapshot based on the same timestamp. Here, we see, from the timestamp, that ks.tb and system.local were snapshotted "together".
   
   So from this perspective it is better if snapshot id is partition key.
   
   I would go so far to include timestamp into primary key as well: `((id), keyspace, table, timestamp)`
   
   This way we would have them ordered too and it does not need to be specified when querying.
   
   If we made it like already suggested: (keyspace, table, snapshotid), we would lose the information, it might be like:
   
       ks1 tb1 snapshot1 2023-03-02T13:19:42.757Z        
       ks3 tb2 snapshot1 2023-03-02T13:19:13.757Z 
       ks3 tb3 snapshot1 2023-03-02T13:19:42.757Z 
       ks4 tb4 snapshot1 2023-03-02T13:19:13.757Z 
   
   But here it is not so simple to see that "ks4.tb4" and "ks3.tb2" are forming one logical snapshot and then "ks1.tb1" and "ks3.tb3" another.
   
   So, utimately, it should be like this:
   
   ((id), ts, ks, tb))
   
   Timestamp has to go first to have it ordered, same timestamp for some snapshot id would mean that all keyspaces and tables belong to that particular snapshot name which was taken at that exact time.
   
       cqlsh> select * from ks.test ;
        id | ts | ks | tb
       ----+----+----+----
         1 |  2 |  3 |  4      first snapshot
         1 |  2 |  3 |  5      first snapshot
         1 |  2 |  3 |  6      first snapshot
         1 |  3 |  1 |  2      second snapshot
         1 |  3 |  1 |  3      second snapshot
         2 |  5 |  2 |  6      third snapshot
   



-- 
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 #2117: Add a virtual table to list snapshots for CASSANDRA-18102

Posted by "smiklosovic (via GitHub)" <gi...@apache.org>.
smiklosovic commented on code in PR #2117:
URL: https://github.com/apache/cassandra/pull/2117#discussion_r1123076202


##########
src/java/org/apache/cassandra/db/virtual/SnapshotsTable.java:
##########
@@ -0,0 +1,93 @@
+/*
+ * 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.virtual;
+
+import java.util.List;
+import java.util.Map;
+import java.util.Set;
+import javax.management.openmbean.TabularData;
+import javax.management.openmbean.TabularDataSupport;
+import com.google.common.collect.ImmutableMap;
+
+import org.apache.cassandra.db.SnapshotDetailsTabularData;
+import org.apache.cassandra.db.marshal.BooleanType;
+import org.apache.cassandra.db.marshal.LongType;
+import org.apache.cassandra.db.marshal.UTF8Type;
+import org.apache.cassandra.dht.LocalPartitioner;
+import org.apache.cassandra.schema.TableMetadata;
+import org.apache.cassandra.service.StorageService;
+
+public class SnapshotsTable extends AbstractVirtualTable
+{
+    private static final String SNAPSHOT_NAME = "snapshot_name";
+    private static final String KEYSPACE_NAME = "keyspace_name";
+    private static final String COLUMNFAMILY_NAME = "columnfamily_name";
+    private static final String TRUE_SIZE = "true_size";
+    private static final String SIZE_ON_DISK = "size_on_disk";
+    private static final String CREATE_TIME = "created_at";
+    private static final String EXPIRATION_TIME = "expires_at";
+    private static final String EPHEMERAL = "ephemeral";
+    
+    SnapshotsTable(String keyspace)
+    {
+        super(TableMetadata.builder(keyspace, "snapshots")
+                           .comment("tables snapshots")
+                           .kind(TableMetadata.Kind.VIRTUAL)
+                           .partitioner(new LocalPartitioner(UTF8Type.instance))
+                           .addPartitionKeyColumn(SNAPSHOT_NAME, UTF8Type.instance)

Review Comment:
   @pauloricardomg 
   
   Regardless of whether we can query without allow filtering after 18238 I think it is still a good practice to act as we would normally do when modeling the schema.
   
   As explained above and what Maxwell just explained, I think keyspace > table > snapshot id.
   
   We can have same snapshot names after all, no? So if keyspace and table are clustering columns, with two snapshots of the same name, we would have this partition:
   
       snapshotName1 | keyspace1 | table1
       snapshotName1 | keyspace2 | table2
   
   primary key would be `(snapshotName, (keyspaceName, tableName))`
   
   Is not this counter-intuitive to have a partition which is logically coupling different keyspaces and names under the same snapshot name? That does not make sense to me. 
   
   However, on the other hand, it is possible to do this:
   
       ./bin/nodetool snapshot --kt-list ks.tb,system.local -t mysnapshot
   
   So `listsnapshots` will do this:
   
       mysnapshot  system  local  1.16 KiB  21.47 KiB    2023-03-02T13:19:42.140Z
       mysnapshot  ks      tb     1.02 KiB   6.08 KiB    2023-03-02T13:19:13.757Z
   
   But then I can do this as well, again:
   
       ./bin/nodetool snapshot --kt-list ks.tb2 -t mysnapshot
   
   Which would print it like:
   
       mysnapshot     ks       tb2        1.16 KiB  21.47 KiB    2023-03-02T13:19:42.140Z                
       mysnapshot     ks       tb         1.02 KiB  6.08 KiB     2023-03-02T13:19:13.757Z                
       mysnapshot     system   local      107 bytes 6.98 KiB     2023-03-02T13:19:13.757Z 
   
   So, with the primary key `(id, (keyspace, table))`, the advantage is that we would be able to visually see what all tables were snapshotted in that one logical snapshot based on the same timestamp. Here, we see, from the timestamp, that ks.tb and system.local were snapshotted "together".
   
   So from this perspective it is better if snapshot id is partition key.
   
   I would go so far to include timestamp into primary key as well: `(id, (keyspace, table, timestamp))`
   
   This way we would have them ordered too and it does not need to be specified when querying.
   
   If we made it like I suggested: (keyspace, table, snapshotid), we would lose the information, it migth be like:
   
       ks1 tb1 snapshot1 2023-03-02T13:19:42.757Z        
       ks3 tb2 snapshot1 2023-03-02T13:19:13.757Z 
       ks3 tb3 snapshot1 2023-03-02T13:19:42.757Z 
       ks4 tb4 snapshot1 2023-03-02T13:19:13.757Z 
   
   But here it is not so simple to see that "ks4.tb4" and "ks3.tb2" are forming one logical snapshot and then "ks1.tb1" and "ks3.tb3" another.
   
   So, utimately, it should be like this:
   
   (id, (timestamp, ks, tb))
   
   Timestamp has to go first to have it ordered, same timestamp for some snapshot id would mean that all keyspaces and tables belong to that particular snapshot name which was taken at that exact time.



-- 
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 #2117: Add a virtual table to list snapshots for CASSANDRA-18102

Posted by "smiklosovic (via GitHub)" <gi...@apache.org>.
smiklosovic commented on code in PR #2117:
URL: https://github.com/apache/cassandra/pull/2117#discussion_r1090321653


##########
src/java/org/apache/cassandra/db/virtual/SnapshotsTable.java:
##########
@@ -0,0 +1,93 @@
+/*
+ * 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.virtual;
+
+import java.util.List;
+import java.util.Map;
+import java.util.Set;
+import javax.management.openmbean.TabularData;
+import javax.management.openmbean.TabularDataSupport;
+import com.google.common.collect.ImmutableMap;
+
+import org.apache.cassandra.db.SnapshotDetailsTabularData;
+import org.apache.cassandra.db.marshal.BooleanType;
+import org.apache.cassandra.db.marshal.LongType;
+import org.apache.cassandra.db.marshal.UTF8Type;
+import org.apache.cassandra.dht.LocalPartitioner;
+import org.apache.cassandra.schema.TableMetadata;
+import org.apache.cassandra.service.StorageService;
+
+public class SnapshotsTable extends AbstractVirtualTable
+{
+    private static final String SNAPSHOT_NAME = "snapshot_name";
+    private static final String KEYSPACE_NAME = "keyspace_name";
+    private static final String COLUMNFAMILY_NAME = "columnfamily_name";
+    private static final String TRUE_SIZE = "true_size";
+    private static final String SIZE_ON_DISK = "size_on_disk";
+    private static final String CREATE_TIME = "created_at";
+    private static final String EXPIRATION_TIME = "expires_at";
+    private static final String EPHEMERAL = "ephemeral";
+    
+    SnapshotsTable(String keyspace)
+    {
+        super(TableMetadata.builder(keyspace, "snapshots")
+                           .comment("tables snapshots")
+                           .kind(TableMetadata.Kind.VIRTUAL)
+                           .partitioner(new LocalPartitioner(UTF8Type.instance))
+                           .addPartitionKeyColumn(SNAPSHOT_NAME, UTF8Type.instance)

Review Comment:
   I would change the key. Partition key would be keyspace and clustering column would be table and name of the snapshot. That way, you can ask:
   
   ````
   use system_views;
   select * from snapshots;
   select * from snapshots where keyspace = 'ks1';
   select * from snapshots where keyspace = 'ks1' and table = 'tb1';
   select * from snapshots where keyspace = 'ks1' and table = 'tb1' and name = "mysnapshot";
   select * from snapshots where name = "mysnapshot" allow filtering;
   ````
   
   In practice, I think it is more common to ask questions like "what snapshots this keyspace / table" has? Instead of querying that table by name of the particular snapshot. What if I do not know the name of the snapshot in advance?
   
   @pauloricardomg what do you think? I think this is the most important aspect of this PR.
   



-- 
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] pauloricardomg commented on a diff in pull request #2117: Add a virtual table to list snapshots for CASSANDRA-18102

Posted by "pauloricardomg (via GitHub)" <gi...@apache.org>.
pauloricardomg commented on code in PR #2117:
URL: https://github.com/apache/cassandra/pull/2117#discussion_r1118127209


##########
src/java/org/apache/cassandra/db/virtual/SnapshotsTable.java:
##########
@@ -0,0 +1,93 @@
+/*
+ * 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.virtual;
+
+import java.util.List;
+import java.util.Map;
+import java.util.Set;
+import javax.management.openmbean.TabularData;
+import javax.management.openmbean.TabularDataSupport;
+import com.google.common.collect.ImmutableMap;
+
+import org.apache.cassandra.db.SnapshotDetailsTabularData;
+import org.apache.cassandra.db.marshal.BooleanType;
+import org.apache.cassandra.db.marshal.LongType;
+import org.apache.cassandra.db.marshal.UTF8Type;
+import org.apache.cassandra.dht.LocalPartitioner;
+import org.apache.cassandra.schema.TableMetadata;
+import org.apache.cassandra.service.StorageService;
+
+public class SnapshotsTable extends AbstractVirtualTable
+{
+    private static final String SNAPSHOT_NAME = "snapshot_name";
+    private static final String KEYSPACE_NAME = "keyspace_name";
+    private static final String COLUMNFAMILY_NAME = "columnfamily_name";
+    private static final String TRUE_SIZE = "true_size";
+    private static final String SIZE_ON_DISK = "size_on_disk";
+    private static final String CREATE_TIME = "created_at";
+    private static final String EXPIRATION_TIME = "expires_at";
+    private static final String EPHEMERAL = "ephemeral";
+    
+    SnapshotsTable(String keyspace)
+    {
+        super(TableMetadata.builder(keyspace, "snapshots")
+                           .comment("tables snapshots")
+                           .kind(TableMetadata.Kind.VIRTUAL)
+                           .partitioner(new LocalPartitioner(UTF8Type.instance))
+                           .addPartitionKeyColumn(SNAPSHOT_NAME, UTF8Type.instance)

Review Comment:
   > I think we should use the column name "tag" and not "name" to refer to a snapshot identifier
   
   Even though we have been using `tag` to refer to snapshot ids, this name is a bit misleading since a resource could in theory have multiple tags (but just one ID). @smiklosovic I wonder if it would make sense to just call it snapshot `id` and update all internal usages to refer to snapshots by `id` and not `name/tag` ?
   
   Updated queries with `id` colum name for snapshot identifier:
   ```sql
   use system_views;
   select * from snapshots;
   select * from snapshots where id = "mysnapshot" # query by partition key
   select * from snapshots where keyspace = 'ks1'; # query by non-partition keys
   select * from snapshots where keyspace = 'ks1' and table = 'tb1';
   select * from snapshots where keyspace = 'ks1' and table = 'tb1' and id = "mysnapshot";
   ```



-- 
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] Maxwell-Guo commented on a diff in pull request #2117: Add a virtual table to list snapshots for CASSANDRA-18102

Posted by "Maxwell-Guo (via GitHub)" <gi...@apache.org>.
Maxwell-Guo commented on code in PR #2117:
URL: https://github.com/apache/cassandra/pull/2117#discussion_r1129013668


##########
test/unit/org/apache/cassandra/cql3/CQLTester.java:
##########
@@ -680,6 +684,60 @@ public void cleanupCache()
             store.cleanupCache();
     }
 
+    public void snapshot(String snapshotName)
+    {
+        snapshot(snapshotName, null, now());
+    }
+
+    public void snapshot(String snapshotName, Instant creationTime)
+    {
+        snapshot(snapshotName, null, creationTime);
+    }
+
+    public void snapshot(String snapshotName, DurationSpec.IntSecondsBound ttl)
+    {
+        snapshot(snapshotName, false, ttl, null, now());
+    }
+    
+    public void snapshot(String snapshotName, DurationSpec.IntSecondsBound ttl, Instant creationTime)
+    {
+        snapshot(snapshotName, false, ttl, null, creationTime);
+    }
+
+    public void snapshot(String snapshotName, boolean ephemeral, DurationSpec.IntSecondsBound ttl, Instant creationTime)
+    {
+        snapshot(snapshotName, ephemeral,false, ttl, null, creationTime);
+    }
+
+    public void snapshot(String snapshotName, boolean skipMemtable, DurationSpec.IntSecondsBound ttl, RateLimiter rateLimiter, Instant creationTime)
+    {
+        snapshot(snapshotName, false, skipMemtable, ttl, rateLimiter, creationTime);
+    }
+
+    public void snapshot(String snapshotName, boolean ephemeral, boolean skipMemtable, DurationSpec.IntSecondsBound ttl, RateLimiter rateLimiter, Instant creationTime)
+    {
+        ColumnFamilyStore store = getCurrentColumnFamilyStore();
+        if(store != null)
+        {
+            store.snapshot(snapshotName, null, ephemeral, skipMemtable, ttl, rateLimiter, creationTime);
+        }
+    }
+    
+    public void clearSnapshot(String keyspace)

Review Comment:
   This method ("clearSnapshot(String keyspace)")is not used, what I used is "void clearSnapshot(String snapshotName, String keyspace)" ,but I think some one may need it in the future so I left it here.
   Besides I think we can also implement some other method like  clearSnapshot(String id) in the future too.



-- 
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] Maxwell-Guo commented on a diff in pull request #2117: Add a virtual table to list snapshots for CASSANDRA-18102

Posted by "Maxwell-Guo (via GitHub)" <gi...@apache.org>.
Maxwell-Guo commented on code in PR #2117:
URL: https://github.com/apache/cassandra/pull/2117#discussion_r1124099527


##########
src/java/org/apache/cassandra/db/virtual/SnapshotsTable.java:
##########
@@ -0,0 +1,93 @@
+/*
+ * 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.virtual;
+
+import java.util.List;
+import java.util.Map;
+import java.util.Set;
+import javax.management.openmbean.TabularData;
+import javax.management.openmbean.TabularDataSupport;
+import com.google.common.collect.ImmutableMap;
+
+import org.apache.cassandra.db.SnapshotDetailsTabularData;
+import org.apache.cassandra.db.marshal.BooleanType;
+import org.apache.cassandra.db.marshal.LongType;
+import org.apache.cassandra.db.marshal.UTF8Type;
+import org.apache.cassandra.dht.LocalPartitioner;
+import org.apache.cassandra.schema.TableMetadata;
+import org.apache.cassandra.service.StorageService;
+
+public class SnapshotsTable extends AbstractVirtualTable
+{
+    private static final String SNAPSHOT_NAME = "snapshot_name";
+    private static final String KEYSPACE_NAME = "keyspace_name";
+    private static final String COLUMNFAMILY_NAME = "columnfamily_name";
+    private static final String TRUE_SIZE = "true_size";
+    private static final String SIZE_ON_DISK = "size_on_disk";
+    private static final String CREATE_TIME = "created_at";
+    private static final String EXPIRATION_TIME = "expires_at";
+    private static final String EPHEMERAL = "ephemeral";
+    
+    SnapshotsTable(String keyspace)
+    {
+        super(TableMetadata.builder(keyspace, "snapshots")
+                           .comment("tables snapshots")
+                           .kind(TableMetadata.Kind.VIRTUAL)
+                           .partitioner(new LocalPartitioner(UTF8Type.instance))
+                           .addPartitionKeyColumn(SNAPSHOT_NAME, UTF8Type.instance)

Review Comment:
   If so,It seems that there is no need for me to change the code logic ?



-- 
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 #2117: Add a virtual table to list snapshots for CASSANDRA-18102

Posted by "smiklosovic (via GitHub)" <gi...@apache.org>.
smiklosovic commented on code in PR #2117:
URL: https://github.com/apache/cassandra/pull/2117#discussion_r1123076202


##########
src/java/org/apache/cassandra/db/virtual/SnapshotsTable.java:
##########
@@ -0,0 +1,93 @@
+/*
+ * 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.virtual;
+
+import java.util.List;
+import java.util.Map;
+import java.util.Set;
+import javax.management.openmbean.TabularData;
+import javax.management.openmbean.TabularDataSupport;
+import com.google.common.collect.ImmutableMap;
+
+import org.apache.cassandra.db.SnapshotDetailsTabularData;
+import org.apache.cassandra.db.marshal.BooleanType;
+import org.apache.cassandra.db.marshal.LongType;
+import org.apache.cassandra.db.marshal.UTF8Type;
+import org.apache.cassandra.dht.LocalPartitioner;
+import org.apache.cassandra.schema.TableMetadata;
+import org.apache.cassandra.service.StorageService;
+
+public class SnapshotsTable extends AbstractVirtualTable
+{
+    private static final String SNAPSHOT_NAME = "snapshot_name";
+    private static final String KEYSPACE_NAME = "keyspace_name";
+    private static final String COLUMNFAMILY_NAME = "columnfamily_name";
+    private static final String TRUE_SIZE = "true_size";
+    private static final String SIZE_ON_DISK = "size_on_disk";
+    private static final String CREATE_TIME = "created_at";
+    private static final String EXPIRATION_TIME = "expires_at";
+    private static final String EPHEMERAL = "ephemeral";
+    
+    SnapshotsTable(String keyspace)
+    {
+        super(TableMetadata.builder(keyspace, "snapshots")
+                           .comment("tables snapshots")
+                           .kind(TableMetadata.Kind.VIRTUAL)
+                           .partitioner(new LocalPartitioner(UTF8Type.instance))
+                           .addPartitionKeyColumn(SNAPSHOT_NAME, UTF8Type.instance)

Review Comment:
   @pauloricardomg 
   
   Regardless of whether we can query without allow filtering after 18238 I think it is still a good practice to act as we would normally do when modeling the schema.
   
   As explained above and what Maxwell just explained, I think keyspace > table > snapshot id.
   
   We can have same snapshot names after all, no? So if keyspace and table are clustering columns, with two snapshots of the same name, we would have this partition:
   
       snapshotName1 | keyspace1 | table1
       snapshotName1 | keyspace2 | table2
   
   primary key would be `(snapshotName, (keyspaceName, tableName))`
   
   Is not this counter-intuitive to have a partition which is logically coupling different keyspaces and names under the same snapshot name? That does not make sense to me. 



-- 
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] pauloricardomg commented on a diff in pull request #2117: Add a virtual table to list snapshots for CASSANDRA-18102

Posted by "pauloricardomg (via GitHub)" <gi...@apache.org>.
pauloricardomg commented on code in PR #2117:
URL: https://github.com/apache/cassandra/pull/2117#discussion_r1123966843


##########
src/java/org/apache/cassandra/db/virtual/SnapshotsTable.java:
##########
@@ -0,0 +1,93 @@
+/*
+ * 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.virtual;
+
+import java.util.List;
+import java.util.Map;
+import java.util.Set;
+import javax.management.openmbean.TabularData;
+import javax.management.openmbean.TabularDataSupport;
+import com.google.common.collect.ImmutableMap;
+
+import org.apache.cassandra.db.SnapshotDetailsTabularData;
+import org.apache.cassandra.db.marshal.BooleanType;
+import org.apache.cassandra.db.marshal.LongType;
+import org.apache.cassandra.db.marshal.UTF8Type;
+import org.apache.cassandra.dht.LocalPartitioner;
+import org.apache.cassandra.schema.TableMetadata;
+import org.apache.cassandra.service.StorageService;
+
+public class SnapshotsTable extends AbstractVirtualTable
+{
+    private static final String SNAPSHOT_NAME = "snapshot_name";
+    private static final String KEYSPACE_NAME = "keyspace_name";
+    private static final String COLUMNFAMILY_NAME = "columnfamily_name";
+    private static final String TRUE_SIZE = "true_size";
+    private static final String SIZE_ON_DISK = "size_on_disk";
+    private static final String CREATE_TIME = "created_at";
+    private static final String EXPIRATION_TIME = "expires_at";
+    private static final String EPHEMERAL = "ephemeral";
+    
+    SnapshotsTable(String keyspace)
+    {
+        super(TableMetadata.builder(keyspace, "snapshots")
+                           .comment("tables snapshots")
+                           .kind(TableMetadata.Kind.VIRTUAL)
+                           .partitioner(new LocalPartitioner(UTF8Type.instance))
+                           .addPartitionKeyColumn(SNAPSHOT_NAME, UTF8Type.instance)

Review Comment:
   > Is not this counter-intuitive to have a partition which is logically coupling different keyspaces and names under the same snapshot name for logically different snapshots? That does not make sense to me.
   
   Agreed, thanks for the clarification.
   
   > But then I can do this as well, again: `./bin/nodetool snapshot --kt-list ks.tb2 -t mysnapshot`
   
   Even though this is allowed I think this is an inconsistency between [takeSnapshot](https://github.com/apache/cassandra/blob/trunk/src/java/org/apache/cassandra/service/StorageService.java#L4162) and [takeMultipleTableSnapshot](https://github.com/apache/cassandra/blob/trunk/src/java/org/apache/cassandra/service/StorageService.java#L4208). In the first method [it's not possible](https://github.com/apache/cassandra/blob/trunk/src/java/org/apache/cassandra/service/StorageService.java#L4183) to create a snapshot if a snapshot with the same id/tag already exists in any keyspace, while in the second method it's only checked if a snapshot with the same name does not exist [in the same table](https://github.com/apache/cassandra/blob/trunk/src/java/org/apache/cassandra/service/StorageService.java#L4235).
   
   I believe the behavior of `takeSnapshot` is the correct one, because it was implemented first and avoids confusion if there are multiple snapshots with the same name taken at different times for different tables. I think we should unify these methods and make this consistent on [CASSANDRA-18271](https://issues.apache.org/jira/browse/CASSANDRA-18271), so it's only possible to have a single logical snapshot with the same id.
   
   If we do that, we should make the partition key `((snapshotId), keyspaceName, tableName)` and not include the timestamp in the primary key, since logical snapshots taken at the same time will be grouped together by id - in this case only "legacy" snapshots (taken before  [CASSANDRA-18271](https://issues.apache.org/jira/browse/CASSANDRA-18271)) will be unordered which is not a big deal. 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] pauloricardomg commented on a diff in pull request #2117: Add a virtual table to list snapshots for CASSANDRA-18102

Posted by "pauloricardomg (via GitHub)" <gi...@apache.org>.
pauloricardomg commented on code in PR #2117:
URL: https://github.com/apache/cassandra/pull/2117#discussion_r1118127209


##########
src/java/org/apache/cassandra/db/virtual/SnapshotsTable.java:
##########
@@ -0,0 +1,93 @@
+/*
+ * 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.virtual;
+
+import java.util.List;
+import java.util.Map;
+import java.util.Set;
+import javax.management.openmbean.TabularData;
+import javax.management.openmbean.TabularDataSupport;
+import com.google.common.collect.ImmutableMap;
+
+import org.apache.cassandra.db.SnapshotDetailsTabularData;
+import org.apache.cassandra.db.marshal.BooleanType;
+import org.apache.cassandra.db.marshal.LongType;
+import org.apache.cassandra.db.marshal.UTF8Type;
+import org.apache.cassandra.dht.LocalPartitioner;
+import org.apache.cassandra.schema.TableMetadata;
+import org.apache.cassandra.service.StorageService;
+
+public class SnapshotsTable extends AbstractVirtualTable
+{
+    private static final String SNAPSHOT_NAME = "snapshot_name";
+    private static final String KEYSPACE_NAME = "keyspace_name";
+    private static final String COLUMNFAMILY_NAME = "columnfamily_name";
+    private static final String TRUE_SIZE = "true_size";
+    private static final String SIZE_ON_DISK = "size_on_disk";
+    private static final String CREATE_TIME = "created_at";
+    private static final String EXPIRATION_TIME = "expires_at";
+    private static final String EPHEMERAL = "ephemeral";
+    
+    SnapshotsTable(String keyspace)
+    {
+        super(TableMetadata.builder(keyspace, "snapshots")
+                           .comment("tables snapshots")
+                           .kind(TableMetadata.Kind.VIRTUAL)
+                           .partitioner(new LocalPartitioner(UTF8Type.instance))
+                           .addPartitionKeyColumn(SNAPSHOT_NAME, UTF8Type.instance)

Review Comment:
   > I think we should use the column name "tag" and not "name" to refer to a snapshot identifier
   
   Even though we have been using `tag` to refer to snapshot ids, this name is a bit misleading since a resource could in theory have multiple tags (but just one ID). @smiklosovic I wonder if it would make sense to just call it snapshot `id` and update internal usages to refer to snapshots by `id` and not `name/tag` ?
   
   Updated queries with `id` colum name for snapshot identifier:
   ```sql
   use system_views;
   select * from snapshots;
   select * from snapshots where id = "mysnapshot" # query by partition key
   select * from snapshots where keyspace = 'ks1'; # query by non-partition keys
   select * from snapshots where keyspace = 'ks1' and table = 'tb1';
   select * from snapshots where keyspace = 'ks1' and table = 'tb1' and id = "mysnapshot";
   ```



-- 
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] Maxwell-Guo commented on a diff in pull request #2117: Add a virtual table to list snapshots for CASSANDRA-18102

Posted by "Maxwell-Guo (via GitHub)" <gi...@apache.org>.
Maxwell-Guo commented on code in PR #2117:
URL: https://github.com/apache/cassandra/pull/2117#discussion_r1090634449


##########
src/java/org/apache/cassandra/db/virtual/SnapshotsTable.java:
##########
@@ -0,0 +1,93 @@
+/*
+ * 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.virtual;
+
+import java.util.List;
+import java.util.Map;
+import java.util.Set;
+import javax.management.openmbean.TabularData;
+import javax.management.openmbean.TabularDataSupport;
+import com.google.common.collect.ImmutableMap;
+
+import org.apache.cassandra.db.SnapshotDetailsTabularData;
+import org.apache.cassandra.db.marshal.BooleanType;
+import org.apache.cassandra.db.marshal.LongType;
+import org.apache.cassandra.db.marshal.UTF8Type;
+import org.apache.cassandra.dht.LocalPartitioner;
+import org.apache.cassandra.schema.TableMetadata;
+import org.apache.cassandra.service.StorageService;
+
+public class SnapshotsTable extends AbstractVirtualTable
+{
+    private static final String SNAPSHOT_NAME = "snapshot_name";

Review Comment:
   I hope that the  selectstatement' output result of snapshot virtual table is similar to that of listsnapshot, otherwise the user may have some confusion.
   The description of snapshot name is "snapshot_name", "keyspace_name", "columnfamilt_name" and so on.
   Though I think table or table name is better than "columnfamily_name" as what you said below.



-- 
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 #2117: Add a virtual table to list snapshots for CASSANDRA-18102

Posted by "smiklosovic (via GitHub)" <gi...@apache.org>.
smiklosovic commented on code in PR #2117:
URL: https://github.com/apache/cassandra/pull/2117#discussion_r1090314635


##########
src/java/org/apache/cassandra/db/compaction/CompactionManager.java:
##########
@@ -1848,7 +1848,7 @@ public void run()
                 }
             }
         };
-
+        

Review Comment:
   this is not necessary, please return it back as it was.



##########
src/java/org/apache/cassandra/db/virtual/SnapshotsTable.java:
##########
@@ -0,0 +1,93 @@
+/*
+ * 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.virtual;
+
+import java.util.List;
+import java.util.Map;
+import java.util.Set;
+import javax.management.openmbean.TabularData;
+import javax.management.openmbean.TabularDataSupport;
+import com.google.common.collect.ImmutableMap;
+
+import org.apache.cassandra.db.SnapshotDetailsTabularData;
+import org.apache.cassandra.db.marshal.BooleanType;
+import org.apache.cassandra.db.marshal.LongType;
+import org.apache.cassandra.db.marshal.UTF8Type;
+import org.apache.cassandra.dht.LocalPartitioner;
+import org.apache.cassandra.schema.TableMetadata;
+import org.apache.cassandra.service.StorageService;
+
+public class SnapshotsTable extends AbstractVirtualTable
+{
+    private static final String SNAPSHOT_NAME = "snapshot_name";
+    private static final String KEYSPACE_NAME = "keyspace_name";
+    private static final String COLUMNFAMILY_NAME = "columnfamily_name";
+    private static final String TRUE_SIZE = "true_size";
+    private static final String SIZE_ON_DISK = "size_on_disk";
+    private static final String CREATE_TIME = "created_at";
+    private static final String EXPIRATION_TIME = "expires_at";
+    private static final String EPHEMERAL = "ephemeral";
+    
+    SnapshotsTable(String keyspace)
+    {
+        super(TableMetadata.builder(keyspace, "snapshots")
+                           .comment("tables snapshots")

Review Comment:
   I would change this to "available snapshots" or "current snapshots" or "present snapshots"



##########
src/java/org/apache/cassandra/db/virtual/SnapshotsTable.java:
##########
@@ -0,0 +1,93 @@
+/*
+ * 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.virtual;
+
+import java.util.List;
+import java.util.Map;
+import java.util.Set;
+import javax.management.openmbean.TabularData;
+import javax.management.openmbean.TabularDataSupport;
+import com.google.common.collect.ImmutableMap;
+
+import org.apache.cassandra.db.SnapshotDetailsTabularData;
+import org.apache.cassandra.db.marshal.BooleanType;
+import org.apache.cassandra.db.marshal.LongType;
+import org.apache.cassandra.db.marshal.UTF8Type;
+import org.apache.cassandra.dht.LocalPartitioner;
+import org.apache.cassandra.schema.TableMetadata;
+import org.apache.cassandra.service.StorageService;
+
+public class SnapshotsTable extends AbstractVirtualTable
+{
+    private static final String SNAPSHOT_NAME = "snapshot_name";
+    private static final String KEYSPACE_NAME = "keyspace_name";
+    private static final String COLUMNFAMILY_NAME = "columnfamily_name";
+    private static final String TRUE_SIZE = "true_size";
+    private static final String SIZE_ON_DISK = "size_on_disk";
+    private static final String CREATE_TIME = "created_at";
+    private static final String EXPIRATION_TIME = "expires_at";
+    private static final String EPHEMERAL = "ephemeral";
+    
+    SnapshotsTable(String keyspace)
+    {
+        super(TableMetadata.builder(keyspace, "snapshots")
+                           .comment("tables snapshots")
+                           .kind(TableMetadata.Kind.VIRTUAL)
+                           .partitioner(new LocalPartitioner(UTF8Type.instance))
+                           .addPartitionKeyColumn(SNAPSHOT_NAME, UTF8Type.instance)

Review Comment:
   I would change the key. Partition key would be keyspace and clustering column would be table and name of the snapshot. That way, you can ask:
   
   ````
   use system_views;
   select * from snapshots;
   select * from snapshots where keyspace = 'ks1';
   select * from snapshots where keyspace = 'ks1' and table = 'tb1';
   select * from snapshots where keyspace = 'ks1' and table = 'tb1' and name = "mysnapshot";
   ````
   
   In practice, I think it is more common to ask questions like "what snapshots this keyspace / table" has? Instead of querying that table by name of the particular snapshot. What if I do not know the name of the snapshot in advance?
   



##########
src/java/org/apache/cassandra/db/virtual/SnapshotsTable.java:
##########
@@ -0,0 +1,93 @@
+/*
+ * 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.virtual;
+
+import java.util.List;
+import java.util.Map;
+import java.util.Set;
+import javax.management.openmbean.TabularData;
+import javax.management.openmbean.TabularDataSupport;
+import com.google.common.collect.ImmutableMap;
+
+import org.apache.cassandra.db.SnapshotDetailsTabularData;
+import org.apache.cassandra.db.marshal.BooleanType;
+import org.apache.cassandra.db.marshal.LongType;
+import org.apache.cassandra.db.marshal.UTF8Type;
+import org.apache.cassandra.dht.LocalPartitioner;
+import org.apache.cassandra.schema.TableMetadata;
+import org.apache.cassandra.service.StorageService;
+
+public class SnapshotsTable extends AbstractVirtualTable
+{
+    private static final String SNAPSHOT_NAME = "snapshot_name";
+    private static final String KEYSPACE_NAME = "keyspace_name";

Review Comment:
   isnt "keyspace" enough?



##########
src/java/org/apache/cassandra/db/virtual/SnapshotsTable.java:
##########
@@ -0,0 +1,93 @@
+/*
+ * 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.virtual;
+
+import java.util.List;
+import java.util.Map;
+import java.util.Set;
+import javax.management.openmbean.TabularData;
+import javax.management.openmbean.TabularDataSupport;
+import com.google.common.collect.ImmutableMap;
+
+import org.apache.cassandra.db.SnapshotDetailsTabularData;
+import org.apache.cassandra.db.marshal.BooleanType;
+import org.apache.cassandra.db.marshal.LongType;
+import org.apache.cassandra.db.marshal.UTF8Type;
+import org.apache.cassandra.dht.LocalPartitioner;
+import org.apache.cassandra.schema.TableMetadata;
+import org.apache.cassandra.service.StorageService;
+
+public class SnapshotsTable extends AbstractVirtualTable
+{
+    private static final String SNAPSHOT_NAME = "snapshot_name";
+    private static final String KEYSPACE_NAME = "keyspace_name";
+    private static final String COLUMNFAMILY_NAME = "columnfamily_name";
+    private static final String TRUE_SIZE = "true_size";
+    private static final String SIZE_ON_DISK = "size_on_disk";
+    private static final String CREATE_TIME = "created_at";
+    private static final String EXPIRATION_TIME = "expires_at";
+    private static final String EPHEMERAL = "ephemeral";
+    
+    SnapshotsTable(String keyspace)
+    {
+        super(TableMetadata.builder(keyspace, "snapshots")
+                           .comment("tables snapshots")
+                           .kind(TableMetadata.Kind.VIRTUAL)
+                           .partitioner(new LocalPartitioner(UTF8Type.instance))
+                           .addPartitionKeyColumn(SNAPSHOT_NAME, UTF8Type.instance)
+                           .addClusteringColumn(KEYSPACE_NAME, UTF8Type.instance)
+                           .addClusteringColumn(COLUMNFAMILY_NAME, UTF8Type.instance)
+                           .addRegularColumn(TRUE_SIZE, LongType.instance)
+                           .addRegularColumn(SIZE_ON_DISK, LongType.instance)
+                           .addRegularColumn(CREATE_TIME, UTF8Type.instance)
+                           .addRegularColumn(EXPIRATION_TIME, UTF8Type.instance)
+                           .addRegularColumn(EPHEMERAL, BooleanType.instance)
+                           .build());
+    }
+  
+    @Override
+    public DataSet data() 
+    {
+        SimpleDataSet result = new SimpleDataSet(metadata());
+        
+        // include snapshots with ttl and ephemeral
+        Map<String, TabularData> snapshotDetails = StorageService.instance.getSnapshotDetails(ImmutableMap.of("no_ttl", "false", "include_ephemeral", "true"));
+        for(Map.Entry<String, TabularData> entry : snapshotDetails.entrySet())

Review Comment:
   `for (`



##########
src/java/org/apache/cassandra/db/virtual/SnapshotsTable.java:
##########
@@ -0,0 +1,93 @@
+/*
+ * 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.virtual;
+
+import java.util.List;
+import java.util.Map;
+import java.util.Set;
+import javax.management.openmbean.TabularData;
+import javax.management.openmbean.TabularDataSupport;
+import com.google.common.collect.ImmutableMap;
+
+import org.apache.cassandra.db.SnapshotDetailsTabularData;
+import org.apache.cassandra.db.marshal.BooleanType;
+import org.apache.cassandra.db.marshal.LongType;
+import org.apache.cassandra.db.marshal.UTF8Type;
+import org.apache.cassandra.dht.LocalPartitioner;
+import org.apache.cassandra.schema.TableMetadata;
+import org.apache.cassandra.service.StorageService;
+
+public class SnapshotsTable extends AbstractVirtualTable
+{
+    private static final String SNAPSHOT_NAME = "snapshot_name";
+    private static final String KEYSPACE_NAME = "keyspace_name";
+    private static final String COLUMNFAMILY_NAME = "columnfamily_name";

Review Comment:
   I would rename "columnfamily" to "table" so "columnfamily_name" to "table" only?



##########
src/java/org/apache/cassandra/db/virtual/SnapshotsTable.java:
##########
@@ -0,0 +1,93 @@
+/*
+ * 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.virtual;
+
+import java.util.List;
+import java.util.Map;
+import java.util.Set;
+import javax.management.openmbean.TabularData;
+import javax.management.openmbean.TabularDataSupport;
+import com.google.common.collect.ImmutableMap;
+
+import org.apache.cassandra.db.SnapshotDetailsTabularData;
+import org.apache.cassandra.db.marshal.BooleanType;
+import org.apache.cassandra.db.marshal.LongType;
+import org.apache.cassandra.db.marshal.UTF8Type;
+import org.apache.cassandra.dht.LocalPartitioner;
+import org.apache.cassandra.schema.TableMetadata;
+import org.apache.cassandra.service.StorageService;
+
+public class SnapshotsTable extends AbstractVirtualTable
+{
+    private static final String SNAPSHOT_NAME = "snapshot_name";

Review Comment:
   isnt "name" enough?



-- 
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] Maxwell-Guo commented on a diff in pull request #2117: Add a virtual table to list snapshots for CASSANDRA-18102

Posted by "Maxwell-Guo (via GitHub)" <gi...@apache.org>.
Maxwell-Guo commented on code in PR #2117:
URL: https://github.com/apache/cassandra/pull/2117#discussion_r1100201808


##########
src/java/org/apache/cassandra/db/virtual/SnapshotsTable.java:
##########
@@ -0,0 +1,93 @@
+/*
+ * 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.virtual;
+
+import java.util.List;
+import java.util.Map;
+import java.util.Set;
+import javax.management.openmbean.TabularData;
+import javax.management.openmbean.TabularDataSupport;
+import com.google.common.collect.ImmutableMap;
+
+import org.apache.cassandra.db.SnapshotDetailsTabularData;
+import org.apache.cassandra.db.marshal.BooleanType;
+import org.apache.cassandra.db.marshal.LongType;
+import org.apache.cassandra.db.marshal.UTF8Type;
+import org.apache.cassandra.dht.LocalPartitioner;
+import org.apache.cassandra.schema.TableMetadata;
+import org.apache.cassandra.service.StorageService;
+
+public class SnapshotsTable extends AbstractVirtualTable
+{
+    private static final String SNAPSHOT_NAME = "snapshot_name";
+    private static final String KEYSPACE_NAME = "keyspace_name";
+    private static final String COLUMNFAMILY_NAME = "columnfamily_name";
+    private static final String TRUE_SIZE = "true_size";
+    private static final String SIZE_ON_DISK = "size_on_disk";
+    private static final String CREATE_TIME = "created_at";
+    private static final String EXPIRATION_TIME = "expires_at";
+    private static final String EPHEMERAL = "ephemeral";
+    
+    SnapshotsTable(String keyspace)
+    {
+        super(TableMetadata.builder(keyspace, "snapshots")
+                           .comment("tables snapshots")
+                           .kind(TableMetadata.Kind.VIRTUAL)
+                           .partitioner(new LocalPartitioner(UTF8Type.instance))
+                           .addPartitionKeyColumn(SNAPSHOT_NAME, UTF8Type.instance)

Review Comment:
   hi @smiklosovic ,thanks so much. Busy these days so I feel sorry for the delay in replying to the message.
   I have already finished the above work.
   Besides I hope you can take a look at the reply of naming below.Thanks



-- 
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 #2117: Add a virtual table to list snapshots for CASSANDRA-18102

Posted by "smiklosovic (via GitHub)" <gi...@apache.org>.
smiklosovic commented on code in PR #2117:
URL: https://github.com/apache/cassandra/pull/2117#discussion_r1118409113


##########
src/java/org/apache/cassandra/db/virtual/SnapshotsTable.java:
##########
@@ -0,0 +1,93 @@
+/*
+ * 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.virtual;
+
+import java.util.List;
+import java.util.Map;
+import java.util.Set;
+import javax.management.openmbean.TabularData;
+import javax.management.openmbean.TabularDataSupport;
+import com.google.common.collect.ImmutableMap;
+
+import org.apache.cassandra.db.SnapshotDetailsTabularData;
+import org.apache.cassandra.db.marshal.BooleanType;
+import org.apache.cassandra.db.marshal.LongType;
+import org.apache.cassandra.db.marshal.UTF8Type;
+import org.apache.cassandra.dht.LocalPartitioner;
+import org.apache.cassandra.schema.TableMetadata;
+import org.apache.cassandra.service.StorageService;
+
+public class SnapshotsTable extends AbstractVirtualTable
+{
+    private static final String SNAPSHOT_NAME = "snapshot_name";
+    private static final String KEYSPACE_NAME = "keyspace_name";
+    private static final String COLUMNFAMILY_NAME = "columnfamily_name";
+    private static final String TRUE_SIZE = "true_size";
+    private static final String SIZE_ON_DISK = "size_on_disk";
+    private static final String CREATE_TIME = "created_at";
+    private static final String EXPIRATION_TIME = "expires_at";
+    private static final String EPHEMERAL = "ephemeral";
+    
+    SnapshotsTable(String keyspace)
+    {
+        super(TableMetadata.builder(keyspace, "snapshots")
+                           .comment("tables snapshots")
+                           .kind(TableMetadata.Kind.VIRTUAL)
+                           .partitioner(new LocalPartitioner(UTF8Type.instance))
+                           .addPartitionKeyColumn(SNAPSHOT_NAME, UTF8Type.instance)

Review Comment:
   yes it is very unfortunate that keyspace is a reserved word. It may be workarounded by 'keyspace' (putting that to quotes) but that is quite offputting and not intuitive for users to deal with this ... So next best thing is "keyspace_name".
   
   Or we fix cql grammar to not fail on that.
   
   I dont object using 'id' but doing that internally might be dedicated to a separate ticket for the sake of simplicity in this ticket.
   
   Filtering will be enabled by default on vtables when not explicitly disallowed so nothing to cover there.



-- 
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 #2117: Add a virtual table to list snapshots for CASSANDRA-18102

Posted by "smiklosovic (via GitHub)" <gi...@apache.org>.
smiklosovic commented on code in PR #2117:
URL: https://github.com/apache/cassandra/pull/2117#discussion_r1123076202


##########
src/java/org/apache/cassandra/db/virtual/SnapshotsTable.java:
##########
@@ -0,0 +1,93 @@
+/*
+ * 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.virtual;
+
+import java.util.List;
+import java.util.Map;
+import java.util.Set;
+import javax.management.openmbean.TabularData;
+import javax.management.openmbean.TabularDataSupport;
+import com.google.common.collect.ImmutableMap;
+
+import org.apache.cassandra.db.SnapshotDetailsTabularData;
+import org.apache.cassandra.db.marshal.BooleanType;
+import org.apache.cassandra.db.marshal.LongType;
+import org.apache.cassandra.db.marshal.UTF8Type;
+import org.apache.cassandra.dht.LocalPartitioner;
+import org.apache.cassandra.schema.TableMetadata;
+import org.apache.cassandra.service.StorageService;
+
+public class SnapshotsTable extends AbstractVirtualTable
+{
+    private static final String SNAPSHOT_NAME = "snapshot_name";
+    private static final String KEYSPACE_NAME = "keyspace_name";
+    private static final String COLUMNFAMILY_NAME = "columnfamily_name";
+    private static final String TRUE_SIZE = "true_size";
+    private static final String SIZE_ON_DISK = "size_on_disk";
+    private static final String CREATE_TIME = "created_at";
+    private static final String EXPIRATION_TIME = "expires_at";
+    private static final String EPHEMERAL = "ephemeral";
+    
+    SnapshotsTable(String keyspace)
+    {
+        super(TableMetadata.builder(keyspace, "snapshots")
+                           .comment("tables snapshots")
+                           .kind(TableMetadata.Kind.VIRTUAL)
+                           .partitioner(new LocalPartitioner(UTF8Type.instance))
+                           .addPartitionKeyColumn(SNAPSHOT_NAME, UTF8Type.instance)

Review Comment:
   @pauloricardomg 
   
   Regardless of whether we can query without allow filtering after 18238 I think it is still a good practice to act as we would normally do when modeling the schema.
   
   As explained above and what Maxwell just explained, I think keyspace > table > snapshot id.
   
   We can have same snapshot names after all, no? So if keyspace and table are clustering columns, with two snapshots of the same name, we would have this partition:
   
       snapshotName1 | keyspace1 | table1
       snapshotName1 | keyspace2 | table2
   
   primary key would be `((snapshotName), keyspaceName, tableName)`
   
   Is not this counter-intuitive to have a partition which is logically coupling different keyspaces and names under the same snapshot name? That does not make sense to me. 
   
   However, on the other hand, it is possible to do this:
   
       ./bin/nodetool snapshot --kt-list ks.tb,system.local -t mysnapshot
   
   So `listsnapshots` will do this:
   
       mysnapshot  system  local  1.16 KiB  21.47 KiB    2023-03-02T13:19:42.140Z
       mysnapshot  ks      tb     1.02 KiB   6.08 KiB    2023-03-02T13:19:13.757Z
   
   But then I can do this as well, again:
   
       ./bin/nodetool snapshot --kt-list ks.tb2 -t mysnapshot
   
   Which would print it like:
   
       mysnapshot     ks       tb2        1.16 KiB  21.47 KiB    2023-03-02T13:19:42.140Z                
       mysnapshot     ks       tb         1.02 KiB  6.08 KiB     2023-03-02T13:19:13.757Z                
       mysnapshot     system   local      107 bytes 6.98 KiB     2023-03-02T13:19:13.757Z 
   
   So, with the primary key `((id), keyspace, table)`, the advantage is that we would be able to visually see what all tables were snapshotted in that one logical snapshot based on the same timestamp. Here, we see, from the timestamp, that ks.tb and system.local were snapshotted "together".
   
   So from this perspective it is better if snapshot id is partition key.
   
   I would go so far to include timestamp into primary key as well: `((id), keyspace, table, timestamp)`
   
   This way we would have them ordered too and it does not need to be specified when querying.
   
   If we made it like already suggested: (keyspace, table, snapshotid), we would lose the information, it might be like:
   
       ks1 tb1 snapshot1 2023-03-02T13:19:42.757Z        
       ks3 tb2 snapshot1 2023-03-02T13:19:13.757Z 
       ks3 tb3 snapshot1 2023-03-02T13:19:42.757Z 
       ks4 tb4 snapshot1 2023-03-02T13:19:13.757Z 
   
   But here it is not so simple to see that "ks4.tb4" and "ks3.tb2" are forming one logical snapshot and then "ks1.tb1" and "ks3.tb3" another.
   
   So, utimately, it should be like this:
   
   ((id), ts, ks, tb))
   
   Timestamp has to go first to have it ordered, same timestamp for some snapshot id would mean that all keyspaces and tables belong to that particular snapshot name which was taken at that exact time.



-- 
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 #2117: Add a virtual table to list snapshots for CASSANDRA-18102

Posted by "smiklosovic (via GitHub)" <gi...@apache.org>.
smiklosovic commented on code in PR #2117:
URL: https://github.com/apache/cassandra/pull/2117#discussion_r1127983106


##########
test/unit/org/apache/cassandra/cql3/CQLTester.java:
##########
@@ -680,6 +684,60 @@ public void cleanupCache()
             store.cleanupCache();
     }
 
+    public void snapshot(String snapshotName)
+    {
+        snapshot(snapshotName, null, now());
+    }
+
+    public void snapshot(String snapshotName, Instant creationTime)
+    {
+        snapshot(snapshotName, null, creationTime);
+    }
+
+    public void snapshot(String snapshotName, DurationSpec.IntSecondsBound ttl)
+    {
+        snapshot(snapshotName, false, ttl, null, now());
+    }
+    
+    public void snapshot(String snapshotName, DurationSpec.IntSecondsBound ttl, Instant creationTime)
+    {
+        snapshot(snapshotName, false, ttl, null, creationTime);
+    }
+
+    public void snapshot(String snapshotName, boolean ephemeral, DurationSpec.IntSecondsBound ttl, Instant creationTime)
+    {
+        snapshot(snapshotName, ephemeral,false, ttl, null, creationTime);
+    }
+
+    public void snapshot(String snapshotName, boolean skipMemtable, DurationSpec.IntSecondsBound ttl, RateLimiter rateLimiter, Instant creationTime)
+    {
+        snapshot(snapshotName, false, skipMemtable, ttl, rateLimiter, creationTime);
+    }
+
+    public void snapshot(String snapshotName, boolean ephemeral, boolean skipMemtable, DurationSpec.IntSecondsBound ttl, RateLimiter rateLimiter, Instant creationTime)
+    {
+        ColumnFamilyStore store = getCurrentColumnFamilyStore();
+        if(store != null)
+        {
+            store.snapshot(snapshotName, null, ephemeral, skipMemtable, ttl, rateLimiter, creationTime);
+        }
+    }
+    
+    public void clearSnapshot(String keyspace)

Review Comment:
   `clearSnapshotForKeyspace` is better, I would think that paramter for this method is name of the snapshot, not name of keyspace.



-- 
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] Maxwell-Guo commented on a diff in pull request #2117: Add a virtual table to list snapshots for CASSANDRA-18102

Posted by "Maxwell-Guo (via GitHub)" <gi...@apache.org>.
Maxwell-Guo commented on code in PR #2117:
URL: https://github.com/apache/cassandra/pull/2117#discussion_r1121017836


##########
src/java/org/apache/cassandra/db/virtual/SnapshotsTable.java:
##########
@@ -0,0 +1,93 @@
+/*
+ * 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.virtual;
+
+import java.util.List;
+import java.util.Map;
+import java.util.Set;
+import javax.management.openmbean.TabularData;
+import javax.management.openmbean.TabularDataSupport;
+import com.google.common.collect.ImmutableMap;
+
+import org.apache.cassandra.db.SnapshotDetailsTabularData;
+import org.apache.cassandra.db.marshal.BooleanType;
+import org.apache.cassandra.db.marshal.LongType;
+import org.apache.cassandra.db.marshal.UTF8Type;
+import org.apache.cassandra.dht.LocalPartitioner;
+import org.apache.cassandra.schema.TableMetadata;
+import org.apache.cassandra.service.StorageService;
+
+public class SnapshotsTable extends AbstractVirtualTable
+{
+    private static final String SNAPSHOT_NAME = "snapshot_name";
+    private static final String KEYSPACE_NAME = "keyspace_name";
+    private static final String COLUMNFAMILY_NAME = "columnfamily_name";
+    private static final String TRUE_SIZE = "true_size";
+    private static final String SIZE_ON_DISK = "size_on_disk";
+    private static final String CREATE_TIME = "created_at";
+    private static final String EXPIRATION_TIME = "expires_at";
+    private static final String EPHEMERAL = "ephemeral";
+    
+    SnapshotsTable(String keyspace)
+    {
+        super(TableMetadata.builder(keyspace, "snapshots")
+                           .comment("tables snapshots")
+                           .kind(TableMetadata.Kind.VIRTUAL)
+                           .partitioner(new LocalPartitioner(UTF8Type.instance))
+                           .addPartitionKeyColumn(SNAPSHOT_NAME, UTF8Type.instance)

Review Comment:
   Thanks @smiklosovic , So what about using keyspace_name and table_name ? Because I found that some of the system tables just using keyspace_name and table_name , although virtual table is not a real system table, it is similar in a sense. 
   
   I think open a new ticket for change the tag/name to id internally is ok, if we all agree on this I can open a ticket for this and finish the task.
   @pauloricardomg  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] smiklosovic commented on a diff in pull request #2117: Add a virtual table to list snapshots for CASSANDRA-18102

Posted by "smiklosovic (via GitHub)" <gi...@apache.org>.
smiklosovic commented on code in PR #2117:
URL: https://github.com/apache/cassandra/pull/2117#discussion_r1123076202


##########
src/java/org/apache/cassandra/db/virtual/SnapshotsTable.java:
##########
@@ -0,0 +1,93 @@
+/*
+ * 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.virtual;
+
+import java.util.List;
+import java.util.Map;
+import java.util.Set;
+import javax.management.openmbean.TabularData;
+import javax.management.openmbean.TabularDataSupport;
+import com.google.common.collect.ImmutableMap;
+
+import org.apache.cassandra.db.SnapshotDetailsTabularData;
+import org.apache.cassandra.db.marshal.BooleanType;
+import org.apache.cassandra.db.marshal.LongType;
+import org.apache.cassandra.db.marshal.UTF8Type;
+import org.apache.cassandra.dht.LocalPartitioner;
+import org.apache.cassandra.schema.TableMetadata;
+import org.apache.cassandra.service.StorageService;
+
+public class SnapshotsTable extends AbstractVirtualTable
+{
+    private static final String SNAPSHOT_NAME = "snapshot_name";
+    private static final String KEYSPACE_NAME = "keyspace_name";
+    private static final String COLUMNFAMILY_NAME = "columnfamily_name";
+    private static final String TRUE_SIZE = "true_size";
+    private static final String SIZE_ON_DISK = "size_on_disk";
+    private static final String CREATE_TIME = "created_at";
+    private static final String EXPIRATION_TIME = "expires_at";
+    private static final String EPHEMERAL = "ephemeral";
+    
+    SnapshotsTable(String keyspace)
+    {
+        super(TableMetadata.builder(keyspace, "snapshots")
+                           .comment("tables snapshots")
+                           .kind(TableMetadata.Kind.VIRTUAL)
+                           .partitioner(new LocalPartitioner(UTF8Type.instance))
+                           .addPartitionKeyColumn(SNAPSHOT_NAME, UTF8Type.instance)

Review Comment:
   @pauloricardomg 
   
   Regardless of whether we can query without allow filtering after 18238 I think it is still a good practice to act as we would normally do when modeling the schema.
   
   As explained above and what Maxwell just explained, I think keyspace > table > snapshot id.
   
   We can have same snapshot names after all, no? So if keyspace and table are clustering columns, with two snapshots of the same name, we would have this partition:
   
       snapshotName1 | keyspace1 | table1
       snapshotName1 | keyspace2 | table2
   
   primary key would be `(snapshotName, (keyspaceName, tableName))`
   
   Is not this counter-intuitive to have a partition which is logically coupling different keyspaces and names under the same snapshot name? That does not make sense to me. 
   
   However, on the other hand, it is possible to do this:
   
       ./bin/nodetool snapshot --kt-list ks.tb,system.local -t mysnapshot
   
   So `listsnapshots` will do this:
   
       mysnapshot  system  local  1.16 KiB  21.47 KiB    2023-03-02T13:19:42.140Z
       mysnapshot  ks      tb     1.02 KiB   6.08 KiB    2023-03-02T13:19:13.757Z
   
   But then I can do this as well, again:
   
       ./bin/nodetool snapshot --kt-list ks.tb2 -t mysnapshot
   
   Which would print it like:
   
       mysnapshot     ks       tb2        1.16 KiB  21.47 KiB    2023-03-02T13:19:42.140Z                
       mysnapshot     ks       tb         1.02 KiB  6.08 KiB     2023-03-02T13:19:13.757Z                
       mysnapshot     system   local      107 bytes 6.98 KiB     2023-03-02T13:19:13.757Z 
   
   So, with the primary key `(id, (keyspace, table))`, the advantage is that we would be able to visually see what all tables were snapshotted in that one logical snapshot based on the same timestamp. Here, we see, from the timestamp, that ks.tb and system.local were snapshotted "together".
   
   So from this perspective it is better if snapshot id is partition key.
   
   I would go so far to include timestamp into primary key as well: `(id, (keyspace, table, timestamp))`
   
   This way we would have them ordered too and it does not need to be specified when querying.
   
   If we made it like I suggested: (keyspace, table, snapshotid), we would lose the information, it migth be like:
   
       ks1 tb1 snapshot1 2023-03-02T13:19:42.140Z        
       ks3 tb2 snapshot1 2023-03-02T13:19:13.757Z 
       ks3 tb3 snapshot1 2023-03-02T13:19:42.757Z 
       ks4 tb4 snapshot1 2023-03-02T13:19:13.757Z 
   
   But here it is not so simple to see that "ks4.tb4" and "ks3.tb2" are forming one logical snapshot and then "ks1.tb1" and "ks3.tb3" another.



-- 
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 #2117: Add a virtual table to list snapshots for CASSANDRA-18102

Posted by "smiklosovic (via GitHub)" <gi...@apache.org>.
smiklosovic commented on code in PR #2117:
URL: https://github.com/apache/cassandra/pull/2117#discussion_r1124167982


##########
src/java/org/apache/cassandra/db/virtual/SnapshotsTable.java:
##########
@@ -0,0 +1,93 @@
+/*
+ * 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.virtual;
+
+import java.util.List;
+import java.util.Map;
+import java.util.Set;
+import javax.management.openmbean.TabularData;
+import javax.management.openmbean.TabularDataSupport;
+import com.google.common.collect.ImmutableMap;
+
+import org.apache.cassandra.db.SnapshotDetailsTabularData;
+import org.apache.cassandra.db.marshal.BooleanType;
+import org.apache.cassandra.db.marshal.LongType;
+import org.apache.cassandra.db.marshal.UTF8Type;
+import org.apache.cassandra.dht.LocalPartitioner;
+import org.apache.cassandra.schema.TableMetadata;
+import org.apache.cassandra.service.StorageService;
+
+public class SnapshotsTable extends AbstractVirtualTable
+{
+    private static final String SNAPSHOT_NAME = "snapshot_name";
+    private static final String KEYSPACE_NAME = "keyspace_name";
+    private static final String COLUMNFAMILY_NAME = "columnfamily_name";
+    private static final String TRUE_SIZE = "true_size";
+    private static final String SIZE_ON_DISK = "size_on_disk";
+    private static final String CREATE_TIME = "created_at";
+    private static final String EXPIRATION_TIME = "expires_at";
+    private static final String EPHEMERAL = "ephemeral";
+    
+    SnapshotsTable(String keyspace)
+    {
+        super(TableMetadata.builder(keyspace, "snapshots")
+                           .comment("tables snapshots")
+                           .kind(TableMetadata.Kind.VIRTUAL)
+                           .partitioner(new LocalPartitioner(UTF8Type.instance))
+                           .addPartitionKeyColumn(SNAPSHOT_NAME, UTF8Type.instance)

Review Comment:
   @pauloricardomg I think we should bring this to ML as we are changing the behavior suddenly and some snapshots will not be possible to do anymore (snapshots with same name).



-- 
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] Maxwell-Guo commented on a diff in pull request #2117: Add a virtual table to list snapshots for CASSANDRA-18102

Posted by "Maxwell-Guo (via GitHub)" <gi...@apache.org>.
Maxwell-Guo commented on code in PR #2117:
URL: https://github.com/apache/cassandra/pull/2117#discussion_r1090635080


##########
src/java/org/apache/cassandra/db/virtual/SnapshotsTable.java:
##########
@@ -0,0 +1,93 @@
+/*
+ * 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.virtual;
+
+import java.util.List;
+import java.util.Map;
+import java.util.Set;
+import javax.management.openmbean.TabularData;
+import javax.management.openmbean.TabularDataSupport;
+import com.google.common.collect.ImmutableMap;
+
+import org.apache.cassandra.db.SnapshotDetailsTabularData;
+import org.apache.cassandra.db.marshal.BooleanType;
+import org.apache.cassandra.db.marshal.LongType;
+import org.apache.cassandra.db.marshal.UTF8Type;
+import org.apache.cassandra.dht.LocalPartitioner;
+import org.apache.cassandra.schema.TableMetadata;
+import org.apache.cassandra.service.StorageService;
+
+public class SnapshotsTable extends AbstractVirtualTable
+{
+    private static final String SNAPSHOT_NAME = "snapshot_name";
+    private static final String KEYSPACE_NAME = "keyspace_name";

Review Comment:
   same with the upper comment



##########
src/java/org/apache/cassandra/db/virtual/SnapshotsTable.java:
##########
@@ -0,0 +1,93 @@
+/*
+ * 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.virtual;
+
+import java.util.List;
+import java.util.Map;
+import java.util.Set;
+import javax.management.openmbean.TabularData;
+import javax.management.openmbean.TabularDataSupport;
+import com.google.common.collect.ImmutableMap;
+
+import org.apache.cassandra.db.SnapshotDetailsTabularData;
+import org.apache.cassandra.db.marshal.BooleanType;
+import org.apache.cassandra.db.marshal.LongType;
+import org.apache.cassandra.db.marshal.UTF8Type;
+import org.apache.cassandra.dht.LocalPartitioner;
+import org.apache.cassandra.schema.TableMetadata;
+import org.apache.cassandra.service.StorageService;
+
+public class SnapshotsTable extends AbstractVirtualTable
+{
+    private static final String SNAPSHOT_NAME = "snapshot_name";
+    private static final String KEYSPACE_NAME = "keyspace_name";
+    private static final String COLUMNFAMILY_NAME = "columnfamily_name";

Review Comment:
   same with the upper comment



-- 
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 #2117: Add a virtual table to list snapshots for CASSANDRA-18102

Posted by "smiklosovic (via GitHub)" <gi...@apache.org>.
smiklosovic commented on code in PR #2117:
URL: https://github.com/apache/cassandra/pull/2117#discussion_r1129685069


##########
test/unit/org/apache/cassandra/db/virtual/SnapshotsTableTest.java:
##########
@@ -0,0 +1,103 @@
+/*
+ * 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.virtual;
+
+import java.time.Instant;
+
+import com.google.common.collect.ImmutableList;
+import org.junit.After;
+import org.junit.Before;
+import org.junit.BeforeClass;
+import org.junit.Test;
+
+import org.apache.cassandra.config.DurationSpec;
+import org.apache.cassandra.cql3.CQLTester;
+import org.apache.cassandra.cql3.UntypedResultSet;
+
+public class SnapshotsTableTest extends CQLTester
+{
+    private static final String KS_NAME = "vts";
+    private final String SNAPSHOT_TTL = "snapshotTtl";
+    private final String SNAPSHOT_NO_TTL = "snapshotNoTtl";
+    private final String SNAPSHOT_EPHEMERAL = "snapshotEphemeral";
+    private final DurationSpec.IntSecondsBound ttl = new DurationSpec.IntSecondsBound ("4h");
+    
+    @BeforeClass
+    public static void setUpClass()
+    {
+        CQLTester.setUpClass();
+    }
+
+    @Before
+    public void before() throws Throwable
+    {
+        SnapshotsTable table = new SnapshotsTable(KS_NAME);
+        VirtualKeyspaceRegistry.instance.register(new VirtualKeyspace(KS_NAME, ImmutableList.of(table)));
+        
+        createTable("CREATE TABLE %s (pk int, ck int, PRIMARY KEY (pk, ck))");
+        for (int i = 0; i != 10; ++i)
+        {
+            execute("INSERT INTO %s (pk, ck) VALUES (?, ?)", i, i);
+        }
+        flush();
+    }
+    
+    @After
+    public void after()
+    {
+        clearSnapshot(SNAPSHOT_NO_TTL, currentKeyspace());
+        clearSnapshot(SNAPSHOT_TTL, currentKeyspace());
+        clearSnapshot(SNAPSHOT_EPHEMERAL, currentKeyspace());
+        dropTable("DROP TABLE %s");
+    }
+   
+    @Test
+    public void testSnapshots() throws Throwable
+    {
+        Instant createTime = Instant.now();
+        String createTimeStr = createTime.toString();
+        snapshot(SNAPSHOT_NO_TTL, createTime);
+        snapshot(SNAPSHOT_TTL, ttl, createTime);
+        snapshot(SNAPSHOT_EPHEMERAL, true, null, createTime);
+
+        // query all from snapshots virtual table
+        UntypedResultSet result = execute("SELECT id, keyspace_name, table_name, created_at, expires_at, ephemeral FROM vts.snapshots");
+        assertRows(result,
+                row(SNAPSHOT_EPHEMERAL, CQLTester.KEYSPACE, currentTable(), createTimeStr, null, true),
+                row(SNAPSHOT_NO_TTL, CQLTester.KEYSPACE, currentTable(), createTimeStr, null, false),
+                row(SNAPSHOT_TTL, CQLTester.KEYSPACE, currentTable(), createTimeStr, createTime.plusSeconds(ttl.toSeconds()).toString(), false));
+
+        // query with conditions
+        result = execute("SELECT id, keyspace_name, table_name, created_at, expires_at, ephemeral FROM vts.snapshots where ephemeral = true allow filtering");

Review Comment:
   @Maxwell-Guo you really do not need that. Please check CASSANDRA-18238.



-- 
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 #2117: Add a virtual table to list snapshots for CASSANDRA-18102

Posted by "smiklosovic (via GitHub)" <gi...@apache.org>.
smiklosovic commented on code in PR #2117:
URL: https://github.com/apache/cassandra/pull/2117#discussion_r1090321653


##########
src/java/org/apache/cassandra/db/virtual/SnapshotsTable.java:
##########
@@ -0,0 +1,93 @@
+/*
+ * 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.virtual;
+
+import java.util.List;
+import java.util.Map;
+import java.util.Set;
+import javax.management.openmbean.TabularData;
+import javax.management.openmbean.TabularDataSupport;
+import com.google.common.collect.ImmutableMap;
+
+import org.apache.cassandra.db.SnapshotDetailsTabularData;
+import org.apache.cassandra.db.marshal.BooleanType;
+import org.apache.cassandra.db.marshal.LongType;
+import org.apache.cassandra.db.marshal.UTF8Type;
+import org.apache.cassandra.dht.LocalPartitioner;
+import org.apache.cassandra.schema.TableMetadata;
+import org.apache.cassandra.service.StorageService;
+
+public class SnapshotsTable extends AbstractVirtualTable
+{
+    private static final String SNAPSHOT_NAME = "snapshot_name";
+    private static final String KEYSPACE_NAME = "keyspace_name";
+    private static final String COLUMNFAMILY_NAME = "columnfamily_name";
+    private static final String TRUE_SIZE = "true_size";
+    private static final String SIZE_ON_DISK = "size_on_disk";
+    private static final String CREATE_TIME = "created_at";
+    private static final String EXPIRATION_TIME = "expires_at";
+    private static final String EPHEMERAL = "ephemeral";
+    
+    SnapshotsTable(String keyspace)
+    {
+        super(TableMetadata.builder(keyspace, "snapshots")
+                           .comment("tables snapshots")
+                           .kind(TableMetadata.Kind.VIRTUAL)
+                           .partitioner(new LocalPartitioner(UTF8Type.instance))
+                           .addPartitionKeyColumn(SNAPSHOT_NAME, UTF8Type.instance)

Review Comment:
   I would change the key. Partition key would be keyspace and clustering column would be table and name of the snapshot. That way, you can ask:
   
   ````
   use system_views;
   select * from snapshots;
   select * from snapshots where keyspace = 'ks1';
   select * from snapshots where keyspace = 'ks1' and table = 'tb1';
   select * from snapshots where keyspace = 'ks1' and table = 'tb1' and name = "mysnapshot";
   select * from snapshots where name = "mysnapshot" allow filtering;
   ````
   
   In practice, I think it is more common to ask questions like "what snapshots this keyspace / table" has? Instead of querying that table by name of the particular snapshot. What if I do not know the name of the snapshot in advance?
   



-- 
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] Maxwell-Guo commented on a diff in pull request #2117: Add a virtual table to list snapshots for CASSANDRA-18102

Posted by "Maxwell-Guo (via GitHub)" <gi...@apache.org>.
Maxwell-Guo commented on code in PR #2117:
URL: https://github.com/apache/cassandra/pull/2117#discussion_r1090628890


##########
src/java/org/apache/cassandra/db/virtual/SnapshotsTable.java:
##########
@@ -0,0 +1,93 @@
+/*
+ * 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.virtual;
+
+import java.util.List;
+import java.util.Map;
+import java.util.Set;
+import javax.management.openmbean.TabularData;
+import javax.management.openmbean.TabularDataSupport;
+import com.google.common.collect.ImmutableMap;
+
+import org.apache.cassandra.db.SnapshotDetailsTabularData;
+import org.apache.cassandra.db.marshal.BooleanType;
+import org.apache.cassandra.db.marshal.LongType;
+import org.apache.cassandra.db.marshal.UTF8Type;
+import org.apache.cassandra.dht.LocalPartitioner;
+import org.apache.cassandra.schema.TableMetadata;
+import org.apache.cassandra.service.StorageService;
+
+public class SnapshotsTable extends AbstractVirtualTable
+{
+    private static final String SNAPSHOT_NAME = "snapshot_name";
+    private static final String KEYSPACE_NAME = "keyspace_name";
+    private static final String COLUMNFAMILY_NAME = "columnfamily_name";
+    private static final String TRUE_SIZE = "true_size";
+    private static final String SIZE_ON_DISK = "size_on_disk";
+    private static final String CREATE_TIME = "created_at";
+    private static final String EXPIRATION_TIME = "expires_at";
+    private static final String EPHEMERAL = "ephemeral";
+    
+    SnapshotsTable(String keyspace)
+    {
+        super(TableMetadata.builder(keyspace, "snapshots")
+                           .comment("tables snapshots")
+                           .kind(TableMetadata.Kind.VIRTUAL)
+                           .partitioner(new LocalPartitioner(UTF8Type.instance))
+                           .addPartitionKeyColumn(SNAPSHOT_NAME, UTF8Type.instance)

Review Comment:
   +1  on what you said 



-- 
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 #2117: Add a virtual table to list snapshots for CASSANDRA-18102

Posted by "smiklosovic (via GitHub)" <gi...@apache.org>.
smiklosovic commented on code in PR #2117:
URL: https://github.com/apache/cassandra/pull/2117#discussion_r1127978069


##########
test/unit/org/apache/cassandra/db/virtual/SnapshotsTableTest.java:
##########
@@ -0,0 +1,103 @@
+/*
+ * 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.virtual;
+
+import java.time.Instant;
+
+import com.google.common.collect.ImmutableList;
+import org.junit.After;
+import org.junit.Before;
+import org.junit.BeforeClass;
+import org.junit.Test;
+
+import org.apache.cassandra.config.DurationSpec;
+import org.apache.cassandra.cql3.CQLTester;
+import org.apache.cassandra.cql3.UntypedResultSet;
+
+public class SnapshotsTableTest extends CQLTester
+{
+    private static final String KS_NAME = "vts";
+    private final String SNAPSHOT_TTL = "snapshotTtl";
+    private final String SNAPSHOT_NO_TTL = "snapshotNoTtl";
+    private final String SNAPSHOT_EPHEMERAL = "snapshotEphemeral";
+    private final DurationSpec.IntSecondsBound ttl = new DurationSpec.IntSecondsBound ("4h");
+    
+    @BeforeClass
+    public static void setUpClass()
+    {
+        CQLTester.setUpClass();
+    }
+
+    @Before
+    public void before() throws Throwable
+    {
+        SnapshotsTable table = new SnapshotsTable(KS_NAME);
+        VirtualKeyspaceRegistry.instance.register(new VirtualKeyspace(KS_NAME, ImmutableList.of(table)));
+        
+        createTable("CREATE TABLE %s (pk int, ck int, PRIMARY KEY (pk, ck))");
+        for (int i = 0; i != 10; ++i)
+        {
+            execute("INSERT INTO %s (pk, ck) VALUES (?, ?)", i, i);
+        }
+        flush();
+    }
+    
+    @After
+    public void after()
+    {
+        clearSnapshot(SNAPSHOT_NO_TTL, currentKeyspace());
+        clearSnapshot(SNAPSHOT_TTL, currentKeyspace());
+        clearSnapshot(SNAPSHOT_EPHEMERAL, currentKeyspace());
+        dropTable("DROP TABLE %s");
+    }
+   
+    @Test
+    public void testSnapshots() throws Throwable
+    {
+        Instant createTime = Instant.now();
+        String createTimeStr = createTime.toString();
+        snapshot(SNAPSHOT_NO_TTL, createTime);
+        snapshot(SNAPSHOT_TTL, ttl, createTime);
+        snapshot(SNAPSHOT_EPHEMERAL, true, null, createTime);
+
+        // query all from snapshots virtual table
+        UntypedResultSet result = execute("SELECT id, keyspace_name, table_name, created_at, expires_at, ephemeral FROM vts.snapshots");
+        assertRows(result,
+                row(SNAPSHOT_EPHEMERAL, CQLTester.KEYSPACE, currentTable(), createTimeStr, null, true),
+                row(SNAPSHOT_NO_TTL, CQLTester.KEYSPACE, currentTable(), createTimeStr, null, false),
+                row(SNAPSHOT_TTL, CQLTester.KEYSPACE, currentTable(), createTimeStr, createTime.plusSeconds(ttl.toSeconds()).toString(), false));
+
+        // query with conditions
+        result = execute("SELECT id, keyspace_name, table_name, created_at, expires_at, ephemeral FROM vts.snapshots where ephemeral = true allow filtering");
+        assertRows(result,
+                row(SNAPSHOT_EPHEMERAL, CQLTester.KEYSPACE, currentTable(), createTimeStr, null, true));
+
+        result = execute("SELECT id, keyspace_name, table_name, created_at, expires_at, ephemeral FROM vts.snapshots where id = ? allow filtering", SNAPSHOT_TTL);

Review Comment:
   `allow filtering` is not necessary



-- 
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 #2117: Add a virtual table to list snapshots for CASSANDRA-18102

Posted by "smiklosovic (via GitHub)" <gi...@apache.org>.
smiklosovic commented on code in PR #2117:
URL: https://github.com/apache/cassandra/pull/2117#discussion_r1127977665


##########
test/unit/org/apache/cassandra/db/virtual/SnapshotsTableTest.java:
##########
@@ -0,0 +1,103 @@
+/*
+ * 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.virtual;
+
+import java.time.Instant;
+
+import com.google.common.collect.ImmutableList;
+import org.junit.After;
+import org.junit.Before;
+import org.junit.BeforeClass;
+import org.junit.Test;
+
+import org.apache.cassandra.config.DurationSpec;
+import org.apache.cassandra.cql3.CQLTester;
+import org.apache.cassandra.cql3.UntypedResultSet;
+
+public class SnapshotsTableTest extends CQLTester
+{
+    private static final String KS_NAME = "vts";
+    private final String SNAPSHOT_TTL = "snapshotTtl";
+    private final String SNAPSHOT_NO_TTL = "snapshotNoTtl";
+    private final String SNAPSHOT_EPHEMERAL = "snapshotEphemeral";
+    private final DurationSpec.IntSecondsBound ttl = new DurationSpec.IntSecondsBound ("4h");
+    
+    @BeforeClass
+    public static void setUpClass()
+    {
+        CQLTester.setUpClass();
+    }
+
+    @Before
+    public void before() throws Throwable
+    {
+        SnapshotsTable table = new SnapshotsTable(KS_NAME);
+        VirtualKeyspaceRegistry.instance.register(new VirtualKeyspace(KS_NAME, ImmutableList.of(table)));
+        
+        createTable("CREATE TABLE %s (pk int, ck int, PRIMARY KEY (pk, ck))");
+        for (int i = 0; i != 10; ++i)
+        {
+            execute("INSERT INTO %s (pk, ck) VALUES (?, ?)", i, i);
+        }
+        flush();
+    }
+    
+    @After
+    public void after()
+    {
+        clearSnapshot(SNAPSHOT_NO_TTL, currentKeyspace());
+        clearSnapshot(SNAPSHOT_TTL, currentKeyspace());
+        clearSnapshot(SNAPSHOT_EPHEMERAL, currentKeyspace());
+        dropTable("DROP TABLE %s");
+    }
+   
+    @Test
+    public void testSnapshots() throws Throwable
+    {
+        Instant createTime = Instant.now();
+        String createTimeStr = createTime.toString();
+        snapshot(SNAPSHOT_NO_TTL, createTime);
+        snapshot(SNAPSHOT_TTL, ttl, createTime);
+        snapshot(SNAPSHOT_EPHEMERAL, true, null, createTime);
+
+        // query all from snapshots virtual table
+        UntypedResultSet result = execute("SELECT id, keyspace_name, table_name, created_at, expires_at, ephemeral FROM vts.snapshots");
+        assertRows(result,
+                row(SNAPSHOT_EPHEMERAL, CQLTester.KEYSPACE, currentTable(), createTimeStr, null, true),
+                row(SNAPSHOT_NO_TTL, CQLTester.KEYSPACE, currentTable(), createTimeStr, null, false),
+                row(SNAPSHOT_TTL, CQLTester.KEYSPACE, currentTable(), createTimeStr, createTime.plusSeconds(ttl.toSeconds()).toString(), false));
+
+        // query with conditions
+        result = execute("SELECT id, keyspace_name, table_name, created_at, expires_at, ephemeral FROM vts.snapshots where ephemeral = true allow filtering");

Review Comment:
   `allow filtering` is not necessary



-- 
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 #2117: Add a virtual table to list snapshots for CASSANDRA-18102

Posted by "smiklosovic (via GitHub)" <gi...@apache.org>.
smiklosovic commented on code in PR #2117:
URL: https://github.com/apache/cassandra/pull/2117#discussion_r1127979693


##########
test/unit/org/apache/cassandra/db/virtual/SnapshotsTableTest.java:
##########
@@ -0,0 +1,103 @@
+/*
+ * 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.virtual;
+
+import java.time.Instant;
+
+import com.google.common.collect.ImmutableList;
+import org.junit.After;
+import org.junit.Before;
+import org.junit.BeforeClass;
+import org.junit.Test;
+
+import org.apache.cassandra.config.DurationSpec;
+import org.apache.cassandra.cql3.CQLTester;
+import org.apache.cassandra.cql3.UntypedResultSet;
+
+public class SnapshotsTableTest extends CQLTester
+{
+    private static final String KS_NAME = "vts";
+    private final String SNAPSHOT_TTL = "snapshotTtl";
+    private final String SNAPSHOT_NO_TTL = "snapshotNoTtl";
+    private final String SNAPSHOT_EPHEMERAL = "snapshotEphemeral";
+    private final DurationSpec.IntSecondsBound ttl = new DurationSpec.IntSecondsBound ("4h");
+    
+    @BeforeClass
+    public static void setUpClass()

Review Comment:
   is this method really necessary? Is not it executed anyway?



-- 
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 #2117: Add a virtual table to list snapshots for CASSANDRA-18102

Posted by "smiklosovic (via GitHub)" <gi...@apache.org>.
smiklosovic commented on code in PR #2117:
URL: https://github.com/apache/cassandra/pull/2117#discussion_r1122363137


##########
src/java/org/apache/cassandra/db/virtual/SnapshotsTable.java:
##########
@@ -0,0 +1,93 @@
+/*
+ * 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.virtual;
+
+import java.util.List;
+import java.util.Map;
+import java.util.Set;
+import javax.management.openmbean.TabularData;
+import javax.management.openmbean.TabularDataSupport;
+import com.google.common.collect.ImmutableMap;
+
+import org.apache.cassandra.db.SnapshotDetailsTabularData;
+import org.apache.cassandra.db.marshal.BooleanType;
+import org.apache.cassandra.db.marshal.LongType;
+import org.apache.cassandra.db.marshal.UTF8Type;
+import org.apache.cassandra.dht.LocalPartitioner;
+import org.apache.cassandra.schema.TableMetadata;
+import org.apache.cassandra.service.StorageService;
+
+public class SnapshotsTable extends AbstractVirtualTable
+{
+    private static final String SNAPSHOT_NAME = "snapshot_name";
+    private static final String KEYSPACE_NAME = "keyspace_name";
+    private static final String COLUMNFAMILY_NAME = "columnfamily_name";
+    private static final String TRUE_SIZE = "true_size";
+    private static final String SIZE_ON_DISK = "size_on_disk";
+    private static final String CREATE_TIME = "created_at";
+    private static final String EXPIRATION_TIME = "expires_at";
+    private static final String EPHEMERAL = "ephemeral";
+    
+    SnapshotsTable(String keyspace)
+    {
+        super(TableMetadata.builder(keyspace, "snapshots")
+                           .comment("tables snapshots")
+                           .kind(TableMetadata.Kind.VIRTUAL)
+                           .partitioner(new LocalPartitioner(UTF8Type.instance))
+                           .addPartitionKeyColumn(SNAPSHOT_NAME, UTF8Type.instance)

Review Comment:
   Please change the code to include my suggestion I posted above:
   
   _Partition key would be `keyspace_name` and clustering columns would be `table_name` and `id` of the snapshot._



-- 
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 #2117: Add a virtual table to list snapshots for CASSANDRA-18102

Posted by "smiklosovic (via GitHub)" <gi...@apache.org>.
smiklosovic commented on code in PR #2117:
URL: https://github.com/apache/cassandra/pull/2117#discussion_r1122360512


##########
src/java/org/apache/cassandra/db/virtual/SnapshotsTable.java:
##########
@@ -0,0 +1,93 @@
+/*
+ * 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.virtual;
+
+import java.util.List;
+import java.util.Map;
+import java.util.Set;
+import javax.management.openmbean.TabularData;
+import javax.management.openmbean.TabularDataSupport;
+import com.google.common.collect.ImmutableMap;
+
+import org.apache.cassandra.db.SnapshotDetailsTabularData;
+import org.apache.cassandra.db.marshal.BooleanType;
+import org.apache.cassandra.db.marshal.LongType;
+import org.apache.cassandra.db.marshal.UTF8Type;
+import org.apache.cassandra.dht.LocalPartitioner;
+import org.apache.cassandra.schema.TableMetadata;
+import org.apache.cassandra.service.StorageService;
+
+public class SnapshotsTable extends AbstractVirtualTable
+{
+    private static final String SNAPSHOT_NAME = "snapshot_name";
+    private static final String KEYSPACE_NAME = "keyspace_name";
+    private static final String COLUMNFAMILY_NAME = "columnfamily_name";
+    private static final String TRUE_SIZE = "true_size";
+    private static final String SIZE_ON_DISK = "size_on_disk";
+    private static final String CREATE_TIME = "created_at";
+    private static final String EXPIRATION_TIME = "expires_at";
+    private static final String EPHEMERAL = "ephemeral";
+    
+    SnapshotsTable(String keyspace)
+    {
+        super(TableMetadata.builder(keyspace, "snapshots")
+                           .comment("tables snapshots")
+                           .kind(TableMetadata.Kind.VIRTUAL)
+                           .partitioner(new LocalPartitioner(UTF8Type.instance))
+                           .addPartitionKeyColumn(SNAPSHOT_NAME, UTF8Type.instance)

Review Comment:
   I think it is fine to have keyspace_name and table_name.



-- 
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 #2117: Add a virtual table to list snapshots for CASSANDRA-18102

Posted by "smiklosovic (via GitHub)" <gi...@apache.org>.
smiklosovic commented on code in PR #2117:
URL: https://github.com/apache/cassandra/pull/2117#discussion_r1123076202


##########
src/java/org/apache/cassandra/db/virtual/SnapshotsTable.java:
##########
@@ -0,0 +1,93 @@
+/*
+ * 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.virtual;
+
+import java.util.List;
+import java.util.Map;
+import java.util.Set;
+import javax.management.openmbean.TabularData;
+import javax.management.openmbean.TabularDataSupport;
+import com.google.common.collect.ImmutableMap;
+
+import org.apache.cassandra.db.SnapshotDetailsTabularData;
+import org.apache.cassandra.db.marshal.BooleanType;
+import org.apache.cassandra.db.marshal.LongType;
+import org.apache.cassandra.db.marshal.UTF8Type;
+import org.apache.cassandra.dht.LocalPartitioner;
+import org.apache.cassandra.schema.TableMetadata;
+import org.apache.cassandra.service.StorageService;
+
+public class SnapshotsTable extends AbstractVirtualTable
+{
+    private static final String SNAPSHOT_NAME = "snapshot_name";
+    private static final String KEYSPACE_NAME = "keyspace_name";
+    private static final String COLUMNFAMILY_NAME = "columnfamily_name";
+    private static final String TRUE_SIZE = "true_size";
+    private static final String SIZE_ON_DISK = "size_on_disk";
+    private static final String CREATE_TIME = "created_at";
+    private static final String EXPIRATION_TIME = "expires_at";
+    private static final String EPHEMERAL = "ephemeral";
+    
+    SnapshotsTable(String keyspace)
+    {
+        super(TableMetadata.builder(keyspace, "snapshots")
+                           .comment("tables snapshots")
+                           .kind(TableMetadata.Kind.VIRTUAL)
+                           .partitioner(new LocalPartitioner(UTF8Type.instance))
+                           .addPartitionKeyColumn(SNAPSHOT_NAME, UTF8Type.instance)

Review Comment:
   @pauloricardomg 
   
   Regardless of whether we can query without allow filtering after 18238 I think it is still a good practice to act as we would normally do when modeling the schema.
   
   As explained above and what Maxwell just explained, I think keyspace > table > snapshot id.
   
   We can have same snapshot names after all, no? So if keyspace and table are clustering columns, with two snapshots of the same name, we would have this partition:
   
       snapshotName1 | keyspace1 | table1
       snapshotName1 | keyspace2 | table2
   
   primary key would be `(snapshotName, (keyspaceName, tableName))`
   
   Is not this counter-intuitive to have a partition which is logically coupling different keyspaces and names under the same snapshot name? That does not make sense to me. 
   
   However, on the other hand, it is possible to do this:
   
       ./bin/nodetool snapshot --kt-list ks.tb,system.local -t mysnapshot
   
   So `listsnapshots` will do this:
   
       mysnapshot  system  local  1.16 KiB  21.47 KiB    2023-03-02T13:19:42.140Z
       mysnapshot  ks      tb     1.02 KiB   6.08 KiB    2023-03-02T13:19:13.757Z
   
   But then I can do this as well, again:
   
       ./bin/nodetool snapshot --kt-list ks.tb2 -t mysnapshot
   
   Which would print it like:
   
       mysnapshot       ks       tb2        1.16 KiB  21.47 KiB    2023-03-02T13:19:42.140Z                
       mysnapshot       ks       tb         1.02 KiB  6.08 KiB     2023-03-02T13:19:13.757Z                
       mysnapshot       system   local      107 bytes 6.98 KiB     2023-03-02T13:19:13.757Z 
   
   So, with the primary key `(id, (keyspace, table))`, the advantage is that we would be able to visually see what all tables were snapshotted in that one logical snapshot based on the very similar timestamp. Here, we see, from the timestamp, that ks.tb and system.local were snapshotted "together".
   
   So from this perspective it is better if snapshot id is partition key.
   
   I would go so far to include timestamp into primary key as well: `(id, (keyspace, table, timestamp))`
   
   This way we would have them ordered too and it does not need to be specified when querying.
   
   
   If we made it like I suggested: (keyspace, table, snapshotid), we would lose the information, it migth be like:
   
       ks1 tb1 snapshot1 2023-03-02T13:19:42.140Z        
       ks4 tb4 snapshot1 2023-03-02T13:19:13.757Z 
       ks3 tb2 snapshot1 2023-03-02T13:19:13.757Z 
       ks3 tb3 snapshot1 2023-03-02T13:19:42.757Z 
   
   But here it is not so simple to see that "ks4.tb4" and "ks3.tb2" are forming one logical snapshot and then "ks1.tb1" and "ks3.tb3" another.



-- 
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 #2117: Add a virtual table to list snapshots for CASSANDRA-18102

Posted by "smiklosovic (via GitHub)" <gi...@apache.org>.
smiklosovic commented on code in PR #2117:
URL: https://github.com/apache/cassandra/pull/2117#discussion_r1123076202


##########
src/java/org/apache/cassandra/db/virtual/SnapshotsTable.java:
##########
@@ -0,0 +1,93 @@
+/*
+ * 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.virtual;
+
+import java.util.List;
+import java.util.Map;
+import java.util.Set;
+import javax.management.openmbean.TabularData;
+import javax.management.openmbean.TabularDataSupport;
+import com.google.common.collect.ImmutableMap;
+
+import org.apache.cassandra.db.SnapshotDetailsTabularData;
+import org.apache.cassandra.db.marshal.BooleanType;
+import org.apache.cassandra.db.marshal.LongType;
+import org.apache.cassandra.db.marshal.UTF8Type;
+import org.apache.cassandra.dht.LocalPartitioner;
+import org.apache.cassandra.schema.TableMetadata;
+import org.apache.cassandra.service.StorageService;
+
+public class SnapshotsTable extends AbstractVirtualTable
+{
+    private static final String SNAPSHOT_NAME = "snapshot_name";
+    private static final String KEYSPACE_NAME = "keyspace_name";
+    private static final String COLUMNFAMILY_NAME = "columnfamily_name";
+    private static final String TRUE_SIZE = "true_size";
+    private static final String SIZE_ON_DISK = "size_on_disk";
+    private static final String CREATE_TIME = "created_at";
+    private static final String EXPIRATION_TIME = "expires_at";
+    private static final String EPHEMERAL = "ephemeral";
+    
+    SnapshotsTable(String keyspace)
+    {
+        super(TableMetadata.builder(keyspace, "snapshots")
+                           .comment("tables snapshots")
+                           .kind(TableMetadata.Kind.VIRTUAL)
+                           .partitioner(new LocalPartitioner(UTF8Type.instance))
+                           .addPartitionKeyColumn(SNAPSHOT_NAME, UTF8Type.instance)

Review Comment:
   @pauloricardomg 
   
   Regardless of whether we can query without allow filtering after 18238 I think it is still a good practice to act as we would normally do when modeling the schema.
   
   As explained above and what Maxwell just explained, I think keyspace > table > snapshot id.
   
   We can have same snapshot names after all, no? So if keyspace and table are clustering columns, with two snapshots of the same name, we would have this partition:
   
       snapshotName1 | keyspace1 | table1
       snapshotName1 | keyspace2 | table2
   
   primary key would be `(snapshotName, (keyspaceName, tableName))`
   
   Is not this counter-intuitive to have a partition which is logically coupling different keyspaces and names under the same snapshot name? That does not make sense to me. 
   
   However, on the other hand, it is possible to do this:
   
       ./bin/nodetool snapshot --kt-list ks.tb,system.local -t mysnapshot
   
   So `listsnapshots` will do this:
   
       mysnapshot  system  local  1.16 KiB  21.47 KiB    2023-03-02T13:19:42.140Z
       mysnapshot  ks      tb     1.02 KiB   6.08 KiB    2023-03-02T13:19:13.757Z
   
   But then I can do this as well, again:
   
       ./bin/nodetool snapshot --kt-list ks.tb2 -t mysnapshot
   
   Which would print it like:
   
       mysnapshot     ks       tb2        1.16 KiB  21.47 KiB    2023-03-02T13:19:42.140Z                
       mysnapshot     ks       tb         1.02 KiB  6.08 KiB     2023-03-02T13:19:13.757Z                
       mysnapshot     system   local      107 bytes 6.98 KiB     2023-03-02T13:19:13.757Z 
   
   So, with the primary key `(id, (keyspace, table))`, the advantage is that we would be able to visually see what all tables were snapshotted in that one logical snapshot based on the same timestamp. Here, we see, from the timestamp, that ks.tb and system.local were snapshotted "together".
   
   So from this perspective it is better if snapshot id is partition key.
   
   I would go so far to include timestamp into primary key as well: `(id, (keyspace, table, timestamp))`
   
   This way we would have them ordered too and it does not need to be specified when querying.
   
   If we made it like I suggested: (keyspace, table, snapshotid), we would lose the information, it migth be like:
   
       ks1 tb1 snapshot1 2023-03-02T13:19:42.757Z        
       ks3 tb2 snapshot1 2023-03-02T13:19:13.757Z 
       ks3 tb3 snapshot1 2023-03-02T13:19:42.757Z 
       ks4 tb4 snapshot1 2023-03-02T13:19:13.757Z 
   
   But here it is not so simple to see that "ks4.tb4" and "ks3.tb2" are forming one logical snapshot and then "ks1.tb1" and "ks3.tb3" another.



-- 
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 #2117: Add a virtual table to list snapshots for CASSANDRA-18102

Posted by "smiklosovic (via GitHub)" <gi...@apache.org>.
smiklosovic commented on code in PR #2117:
URL: https://github.com/apache/cassandra/pull/2117#discussion_r1123076202


##########
src/java/org/apache/cassandra/db/virtual/SnapshotsTable.java:
##########
@@ -0,0 +1,93 @@
+/*
+ * 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.virtual;
+
+import java.util.List;
+import java.util.Map;
+import java.util.Set;
+import javax.management.openmbean.TabularData;
+import javax.management.openmbean.TabularDataSupport;
+import com.google.common.collect.ImmutableMap;
+
+import org.apache.cassandra.db.SnapshotDetailsTabularData;
+import org.apache.cassandra.db.marshal.BooleanType;
+import org.apache.cassandra.db.marshal.LongType;
+import org.apache.cassandra.db.marshal.UTF8Type;
+import org.apache.cassandra.dht.LocalPartitioner;
+import org.apache.cassandra.schema.TableMetadata;
+import org.apache.cassandra.service.StorageService;
+
+public class SnapshotsTable extends AbstractVirtualTable
+{
+    private static final String SNAPSHOT_NAME = "snapshot_name";
+    private static final String KEYSPACE_NAME = "keyspace_name";
+    private static final String COLUMNFAMILY_NAME = "columnfamily_name";
+    private static final String TRUE_SIZE = "true_size";
+    private static final String SIZE_ON_DISK = "size_on_disk";
+    private static final String CREATE_TIME = "created_at";
+    private static final String EXPIRATION_TIME = "expires_at";
+    private static final String EPHEMERAL = "ephemeral";
+    
+    SnapshotsTable(String keyspace)
+    {
+        super(TableMetadata.builder(keyspace, "snapshots")
+                           .comment("tables snapshots")
+                           .kind(TableMetadata.Kind.VIRTUAL)
+                           .partitioner(new LocalPartitioner(UTF8Type.instance))
+                           .addPartitionKeyColumn(SNAPSHOT_NAME, UTF8Type.instance)

Review Comment:
   @pauloricardomg 
   
   Regardless of whether we can query without allow filtering after 18238 I think it is still a good practice to act as we would normally do when modeling the schema.
   
   As explained above and what Maxwell just explained, I think keyspace > table > snapshot id.
   
   We can have same snapshot names after all, no? So if keyspace and table are clustering columns, with two snapshots of the same name, we would have this partition:
   
       snapshotName1 | keyspace1 | table1
       snapshotName1 | keyspace2 | table2
   
   primary key would be `(snapshotName, (keyspaceName, tableName))`
   
   Is not this counter-intuitive to have a partition which is logically coupling different keyspaces and names under the same snapshot name? That does not make sense to me. 
   
   However, on the other hand, it is possible to do this:
   
       ./bin/nodetool snapshot --kt-list ks.tb,system.local -t mysnapshot
   
   So `listsnapshots` will do this:
   
       mysnapshot  system  local  1.16 KiB  21.47 KiB    2023-03-02T13:19:42.140Z
       mysnapshot  ks      tb     1.02 KiB   6.08 KiB    2023-03-02T13:19:13.757Z
   
   But then I can do this as well, again:
   
       ./bin/nodetool snapshot --kt-list ks.tb2 -t mysnapshot
   
   Which would print it like:
   
       mysnapshot     ks       tb2        1.16 KiB  21.47 KiB    2023-03-02T13:19:42.140Z                
       mysnapshot     ks       tb         1.02 KiB  6.08 KiB     2023-03-02T13:19:13.757Z                
       mysnapshot     system   local      107 bytes 6.98 KiB     2023-03-02T13:19:13.757Z 
   
   So, with the primary key `(id, (keyspace, table))`, the advantage is that we would be able to visually see what all tables were snapshotted in that one logical snapshot based on the same timestamp. Here, we see, from the timestamp, that ks.tb and system.local were snapshotted "together".
   
   So from this perspective it is better if snapshot id is partition key.
   
   I would go so far to include timestamp into primary key as well: `(id, (keyspace, table, timestamp))`
   
   This way we would have them ordered too and it does not need to be specified when querying.
   
   If we made it like I suggested: (keyspace, table, snapshotid), we would lose the information, it migth be like:
   
       ks1 tb1 snapshot1 2023-03-02T13:19:42.140Z        
       ks4 tb4 snapshot1 2023-03-02T13:19:13.757Z 
       ks3 tb2 snapshot1 2023-03-02T13:19:13.757Z 
       ks3 tb3 snapshot1 2023-03-02T13:19:42.757Z 
   
   But here it is not so simple to see that "ks4.tb4" and "ks3.tb2" are forming one logical snapshot and then "ks1.tb1" and "ks3.tb3" another.



-- 
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] Maxwell-Guo commented on a diff in pull request #2117: Add a virtual table to list snapshots for CASSANDRA-18102

Posted by "Maxwell-Guo (via GitHub)" <gi...@apache.org>.
Maxwell-Guo commented on code in PR #2117:
URL: https://github.com/apache/cassandra/pull/2117#discussion_r1122521713


##########
src/java/org/apache/cassandra/db/virtual/SnapshotsTable.java:
##########
@@ -0,0 +1,93 @@
+/*
+ * 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.virtual;
+
+import java.util.List;
+import java.util.Map;
+import java.util.Set;
+import javax.management.openmbean.TabularData;
+import javax.management.openmbean.TabularDataSupport;
+import com.google.common.collect.ImmutableMap;
+
+import org.apache.cassandra.db.SnapshotDetailsTabularData;
+import org.apache.cassandra.db.marshal.BooleanType;
+import org.apache.cassandra.db.marshal.LongType;
+import org.apache.cassandra.db.marshal.UTF8Type;
+import org.apache.cassandra.dht.LocalPartitioner;
+import org.apache.cassandra.schema.TableMetadata;
+import org.apache.cassandra.service.StorageService;
+
+public class SnapshotsTable extends AbstractVirtualTable
+{
+    private static final String SNAPSHOT_NAME = "snapshot_name";
+    private static final String KEYSPACE_NAME = "keyspace_name";
+    private static final String COLUMNFAMILY_NAME = "columnfamily_name";
+    private static final String TRUE_SIZE = "true_size";
+    private static final String SIZE_ON_DISK = "size_on_disk";
+    private static final String CREATE_TIME = "created_at";
+    private static final String EXPIRATION_TIME = "expires_at";
+    private static final String EPHEMERAL = "ephemeral";
+    
+    SnapshotsTable(String keyspace)
+    {
+        super(TableMetadata.builder(keyspace, "snapshots")
+                           .comment("tables snapshots")
+                           .kind(TableMetadata.Kind.VIRTUAL)
+                           .partitioner(new LocalPartitioner(UTF8Type.instance))
+                           .addPartitionKeyColumn(SNAPSHOT_NAME, UTF8Type.instance)

Review Comment:
   In my mind ,I think the scope of snapshot name is bigger than keyspace as different keyspace may have same snapshot name, so first time I treate id as partition key and the output format is same with snapshot detail information.But latter I am +1 on @smiklosovic , for the first perspective of a user may be the keyspace, and they want to know a certain snapshot under the keyspace when writing a cql.I want to make sure it's what you think。 @smiklosovic 



-- 
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 #2117: Add a virtual table to list snapshots for CASSANDRA-18102

Posted by "smiklosovic (via GitHub)" <gi...@apache.org>.
smiklosovic commented on code in PR #2117:
URL: https://github.com/apache/cassandra/pull/2117#discussion_r1123076202


##########
src/java/org/apache/cassandra/db/virtual/SnapshotsTable.java:
##########
@@ -0,0 +1,93 @@
+/*
+ * 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.virtual;
+
+import java.util.List;
+import java.util.Map;
+import java.util.Set;
+import javax.management.openmbean.TabularData;
+import javax.management.openmbean.TabularDataSupport;
+import com.google.common.collect.ImmutableMap;
+
+import org.apache.cassandra.db.SnapshotDetailsTabularData;
+import org.apache.cassandra.db.marshal.BooleanType;
+import org.apache.cassandra.db.marshal.LongType;
+import org.apache.cassandra.db.marshal.UTF8Type;
+import org.apache.cassandra.dht.LocalPartitioner;
+import org.apache.cassandra.schema.TableMetadata;
+import org.apache.cassandra.service.StorageService;
+
+public class SnapshotsTable extends AbstractVirtualTable
+{
+    private static final String SNAPSHOT_NAME = "snapshot_name";
+    private static final String KEYSPACE_NAME = "keyspace_name";
+    private static final String COLUMNFAMILY_NAME = "columnfamily_name";
+    private static final String TRUE_SIZE = "true_size";
+    private static final String SIZE_ON_DISK = "size_on_disk";
+    private static final String CREATE_TIME = "created_at";
+    private static final String EXPIRATION_TIME = "expires_at";
+    private static final String EPHEMERAL = "ephemeral";
+    
+    SnapshotsTable(String keyspace)
+    {
+        super(TableMetadata.builder(keyspace, "snapshots")
+                           .comment("tables snapshots")
+                           .kind(TableMetadata.Kind.VIRTUAL)
+                           .partitioner(new LocalPartitioner(UTF8Type.instance))
+                           .addPartitionKeyColumn(SNAPSHOT_NAME, UTF8Type.instance)

Review Comment:
   @pauloricardomg 
   
   Regardless of whether we can query without allow filtering after 18238 I think it is still a good practice to act as we would normally do when modeling the schema.
   
   As explained above and what Maxwell just explained, I think keyspace > table > snapshot id.
   
   We can have same snapshot names after all, no? So if keyspace and table are clustering columns, with two snapshots of the same name, we would have this partition:
   
       snapshotName1 | keyspace1 | table1
       snapshotName1 | keyspace2 | table2
   
   primary key would be `((snapshotName), keyspaceName, tableName)`
   
   Is not this counter-intuitive to have a partition which is logically coupling different keyspaces and names under the same snapshot name? That does not make sense to me. 
   
   However, on the other hand, it is possible to do this:
   
       ./bin/nodetool snapshot --kt-list ks.tb,system.local -t mysnapshot
   
   So `listsnapshots` will do this:
   
       mysnapshot  system  local  1.16 KiB  21.47 KiB    2023-03-02T13:19:13.757Z
       mysnapshot  ks      tb     1.02 KiB   6.08 KiB    2023-03-02T13:19:13.757Z
   
   But then I can do this as well, again:
   
       ./bin/nodetool snapshot --kt-list ks.tb2 -t mysnapshot
   
   Which would print it like:
   
       mysnapshot     ks       tb2        1.16 KiB  21.47 KiB    2023-03-02T13:19:42.140Z                
       mysnapshot     ks       tb         1.02 KiB  6.08 KiB     2023-03-02T13:19:13.757Z                
       mysnapshot     system   local      107 bytes 6.98 KiB     2023-03-02T13:19:13.757Z 
   
   So, with the primary key `((id), keyspace, table)`, the advantage is that we would be able to visually see what all tables were snapshotted in that one logical snapshot based on the same timestamp. Here, we see, from the timestamp, that ks.tb and system.local were snapshotted "together".
   
   So from this perspective it is better if snapshot id is partition key.
   
   I would go so far to include timestamp into primary key as well: `((id), keyspace, table, timestamp)`
   
   This way we would have them ordered too and it does not need to be specified when querying.
   
   If we made it like already suggested: (keyspace, table, snapshotid), we would lose the information, it might be like:
   
       ks1 tb1 snapshot1 2023-03-02T13:19:42.757Z        
       ks3 tb2 snapshot1 2023-03-02T13:19:13.757Z 
       ks3 tb3 snapshot1 2023-03-02T13:19:42.757Z 
       ks4 tb4 snapshot1 2023-03-02T13:19:13.757Z 
   
   But here it is not so simple to see that "ks4.tb4" and "ks3.tb2" are forming one logical snapshot and then "ks1.tb1" and "ks3.tb3" another.
   
   So, utimately, it should be like this:
   
   ((id), ts, ks, tb))
   
   Timestamp has to go first to have it ordered, same timestamp for some snapshot id would mean that all keyspaces and tables belong to that particular snapshot name which was taken at that exact time.
   
       cqlsh> select * from ks.test ;
        id | ts | ks | tb
       ----+----+----+----
         1 |  2 |  3 |  4      first snapshot
         1 |  2 |  3 |  5      first snapshot
         1 |  2 |  3 |  6      first snapshot
         1 |  3 |  1 |  2      second snapshot
         1 |  3 |  1 |  3      second snapshot
         2 |  5 |  2 |  6      third snapshot
   



-- 
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] pauloricardomg commented on a diff in pull request #2117: Add a virtual table to list snapshots for CASSANDRA-18102

Posted by "pauloricardomg (via GitHub)" <gi...@apache.org>.
pauloricardomg commented on code in PR #2117:
URL: https://github.com/apache/cassandra/pull/2117#discussion_r1118119607


##########
src/java/org/apache/cassandra/db/virtual/SnapshotsTable.java:
##########
@@ -0,0 +1,93 @@
+/*
+ * 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.virtual;
+
+import java.util.List;
+import java.util.Map;
+import java.util.Set;
+import javax.management.openmbean.TabularData;
+import javax.management.openmbean.TabularDataSupport;
+import com.google.common.collect.ImmutableMap;
+
+import org.apache.cassandra.db.SnapshotDetailsTabularData;
+import org.apache.cassandra.db.marshal.BooleanType;
+import org.apache.cassandra.db.marshal.LongType;
+import org.apache.cassandra.db.marshal.UTF8Type;
+import org.apache.cassandra.dht.LocalPartitioner;
+import org.apache.cassandra.schema.TableMetadata;
+import org.apache.cassandra.service.StorageService;
+
+public class SnapshotsTable extends AbstractVirtualTable
+{
+    private static final String SNAPSHOT_NAME = "snapshot_name";
+    private static final String KEYSPACE_NAME = "keyspace_name";
+    private static final String COLUMNFAMILY_NAME = "columnfamily_name";
+    private static final String TRUE_SIZE = "true_size";
+    private static final String SIZE_ON_DISK = "size_on_disk";
+    private static final String CREATE_TIME = "created_at";
+    private static final String EXPIRATION_TIME = "expires_at";
+    private static final String EPHEMERAL = "ephemeral";
+    
+    SnapshotsTable(String keyspace)
+    {
+        super(TableMetadata.builder(keyspace, "snapshots")
+                           .comment("tables snapshots")
+                           .kind(TableMetadata.Kind.VIRTUAL)
+                           .partitioner(new LocalPartitioner(UTF8Type.instance))
+                           .addPartitionKeyColumn(SNAPSHOT_NAME, UTF8Type.instance)

Review Comment:
   Sorry for the delay - have been busy in the past couple of weeks but will try to be faster on next iterations so we can wrap this up soon.
   
   I agree with with @smiklosovic comment about using more succint column names (ie. `keyspace` vs `keyspace_name` and `table` vs `columnfamily_name`), but I think we should use the column name "tag" and not "name" to refer to a snapshot identifier, and keep this as table partition key. We should enable allow filtering on this virtual table (with [CASSANDRA-18271](https://issues.apache.org/jira/browse/CASSANDRA-18271)) to allow querying by non partition keys (ie. `keyspace` or `table`). This would allow the following queries:
   
   ```sql
   use system_views;
   select * from snapshots;
   select * from snapshots where tag = "mysnapshot" # query by partition key
   select * from snapshots where keyspace = 'ks1'; # query by non-partition keys
   select * from snapshots where keyspace = 'ks1' and table = 'tb1';
   select * from snapshots where keyspace = 'ks1' and table = 'tb1' and tag = "mysnapshot";
   ```
   
   What do you think?
   



-- 
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] Maxwell-Guo commented on a diff in pull request #2117: Add a virtual table to list snapshots for CASSANDRA-18102

Posted by "Maxwell-Guo (via GitHub)" <gi...@apache.org>.
Maxwell-Guo commented on code in PR #2117:
URL: https://github.com/apache/cassandra/pull/2117#discussion_r1122521713


##########
src/java/org/apache/cassandra/db/virtual/SnapshotsTable.java:
##########
@@ -0,0 +1,93 @@
+/*
+ * 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.virtual;
+
+import java.util.List;
+import java.util.Map;
+import java.util.Set;
+import javax.management.openmbean.TabularData;
+import javax.management.openmbean.TabularDataSupport;
+import com.google.common.collect.ImmutableMap;
+
+import org.apache.cassandra.db.SnapshotDetailsTabularData;
+import org.apache.cassandra.db.marshal.BooleanType;
+import org.apache.cassandra.db.marshal.LongType;
+import org.apache.cassandra.db.marshal.UTF8Type;
+import org.apache.cassandra.dht.LocalPartitioner;
+import org.apache.cassandra.schema.TableMetadata;
+import org.apache.cassandra.service.StorageService;
+
+public class SnapshotsTable extends AbstractVirtualTable
+{
+    private static final String SNAPSHOT_NAME = "snapshot_name";
+    private static final String KEYSPACE_NAME = "keyspace_name";
+    private static final String COLUMNFAMILY_NAME = "columnfamily_name";
+    private static final String TRUE_SIZE = "true_size";
+    private static final String SIZE_ON_DISK = "size_on_disk";
+    private static final String CREATE_TIME = "created_at";
+    private static final String EXPIRATION_TIME = "expires_at";
+    private static final String EPHEMERAL = "ephemeral";
+    
+    SnapshotsTable(String keyspace)
+    {
+        super(TableMetadata.builder(keyspace, "snapshots")
+                           .comment("tables snapshots")
+                           .kind(TableMetadata.Kind.VIRTUAL)
+                           .partitioner(new LocalPartitioner(UTF8Type.instance))
+                           .addPartitionKeyColumn(SNAPSHOT_NAME, UTF8Type.instance)

Review Comment:
   @pauloricardomg different keyspace and table can have same snapshot id, so first time I use id as partition key and keyspace ,table as clustering key to used to distinguish each other .
   In my mind ,I think the scope of snapshot name is bigger than keyspace as different keyspace may have same snapshot name, so first time I treate id as partition key.But latter I am +1 on @smiklosovic , for the first perspective of a user may be the keyspace, and they want to know a certain snapshot under the keyspace when writing a cql.
   This is what I am thinking. 



-- 
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] Maxwell-Guo closed pull request #2117: Add a virtual table to list snapshots for CASSANDRA-18102

Posted by "Maxwell-Guo (via GitHub)" <gi...@apache.org>.
Maxwell-Guo closed pull request #2117: Add a virtual table to list snapshots for CASSANDRA-18102
URL: https://github.com/apache/cassandra/pull/2117


-- 
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] Maxwell-Guo commented on pull request #2117: Add a virtual table to list snapshots for CASSANDRA-18102

Posted by "Maxwell-Guo (via GitHub)" <gi...@apache.org>.
Maxwell-Guo commented on PR #2117:
URL: https://github.com/apache/cassandra/pull/2117#issuecomment-1461322802

   I am so sorry that this pr is discard due to an operation error, And I have create a new one . https://github.com/apache/cassandra/pull/2205


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