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: [email protected] For queries about this service, please contact Infrastructure at: [email protected] --------------------------------------------------------------------- To unsubscribe, e-mail: [email protected] For additional commands, e-mail: [email protected]

