clohfink commented on code in PR #3756:
URL: https://github.com/apache/cassandra/pull/3756#discussion_r1902270821


##########
src/java/org/apache/cassandra/db/virtual/PrimaryIdTable.java:
##########
@@ -0,0 +1,350 @@
+/*
+ * 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.io.IOException;
+import java.math.BigInteger;
+import java.nio.ByteBuffer;
+import java.util.List;
+import java.util.concurrent.CopyOnWriteArrayList;
+import java.util.function.Consumer;
+
+import com.google.common.annotations.VisibleForTesting;
+import com.google.common.collect.Lists;
+
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import org.apache.cassandra.cql3.Operator;
+import org.apache.cassandra.db.Clustering;
+import org.apache.cassandra.db.ColumnFamilyStore;
+import org.apache.cassandra.db.DataRange;
+import org.apache.cassandra.db.DecoratedKey;
+import org.apache.cassandra.db.DeletionTime;
+import org.apache.cassandra.db.PartitionPosition;
+import org.apache.cassandra.db.Slice;
+import org.apache.cassandra.db.Slices;
+import org.apache.cassandra.db.context.CounterContext;
+import org.apache.cassandra.db.filter.ClusteringIndexFilter;
+import org.apache.cassandra.db.filter.ColumnFilter;
+import org.apache.cassandra.db.filter.RowFilter;
+import org.apache.cassandra.db.marshal.CompositeType;
+import org.apache.cassandra.db.marshal.CounterColumnType;
+import org.apache.cassandra.db.marshal.IntegerType;
+import org.apache.cassandra.db.marshal.UTF8Type;
+import org.apache.cassandra.db.partitions.PartitionUpdate;
+import org.apache.cassandra.db.partitions.SingletonUnfilteredPartitionIterator;
+import org.apache.cassandra.db.partitions.UnfilteredPartitionIterator;
+import org.apache.cassandra.db.rows.AbstractUnfilteredRowIterator;
+import org.apache.cassandra.db.rows.BTreeRow;
+import org.apache.cassandra.db.rows.BufferCell;
+import org.apache.cassandra.db.rows.Cell;
+import org.apache.cassandra.db.rows.EncodingStats;
+import org.apache.cassandra.db.rows.Row;
+import org.apache.cassandra.db.rows.Rows;
+import org.apache.cassandra.db.rows.Unfiltered;
+import org.apache.cassandra.db.rows.UnfilteredRowIterator;
+import org.apache.cassandra.db.rows.UnfilteredRowIterators;
+import org.apache.cassandra.dht.AbstractBounds;
+import org.apache.cassandra.dht.Bounds;
+import org.apache.cassandra.dht.LocalPartitioner;
+import org.apache.cassandra.dht.Range;
+import org.apache.cassandra.dht.Token;
+import org.apache.cassandra.exceptions.InvalidRequestException;
+import org.apache.cassandra.io.sstable.KeyReader;
+import org.apache.cassandra.io.sstable.format.SSTableReader;
+import org.apache.cassandra.schema.ColumnMetadata;
+import org.apache.cassandra.schema.KeyspaceMetadata;
+import org.apache.cassandra.schema.Schema;
+import org.apache.cassandra.schema.TableMetadata;
+
+import static 
org.apache.cassandra.cql3.statements.RequestValidations.invalidRequest;
+
+/**
+ * A virtual table for querying primary IDs of SSTables in a specific keyspace.
+ *
+ * <p>This table is implemented as a virtual table in Cassandra, meaning it 
does not
+ * store data persistently on disk but instead derives its data from live 
metadata.
+ *
+ * <p>The CQL equivalent of this virtual table is:
+ * <pre>
+ * CREATE TABLE system_views.primary_ids (
+ *     keyspace_name TEXT,
+ *     table_name TEXT,
+ *     token_value INT,
+ *     key TEXT,
+ *     size_estimate COUNTER,
+ *     sstables COUNTER,
+ *     PRIMARY KEY ((keyspace_name, table_name), token_value, key)
+ * );
+ * </pre>
+ *
+ * <p>Note:
+ * <ul>
+ *     <li>The `size_estimate` and `sstables` columns represent aggregate 
information about SSTable sizes and counts.</li>
+ *     <li>Range queries across multiple tables and updates are not supported 
as this is a read-only table.</li>
+ * </ul>
+ */
+public class PrimaryIdTable implements VirtualTable
+{
+    private static final Logger logger = 
LoggerFactory.getLogger(PrimaryIdTable.class);
+    public static final String NAME = "primary_ids";
+
+    private static final String TABLE_READ_ONLY_ERROR = "The specified table 
is read-only.";
+    private static final String UNSUPPORTED_RANGE_QUERY_ERROR = "Range queries 
are not supported. Please provide both a keyspace and a table name.";
+    private static final String REVERSED_QUERY_ERROR = "Reversed queries are 
not supported.";
+    private static final String KEYSPACE_NOT_EXIST_ERROR = "The keyspace '%s' 
does not exist.";
+    private static final String TABLE_NOT_EXIST_ERROR = "The table '%s' does 
not exist in the keyspace '%s'.";
+    private static final String KEY_ONLY_EQUALS_ERROR = "The 'key' column can 
only be used in an equality query for this virtual table.";
+    private static final String KEY_NOT_WITHIN_BOUNDS_ERROR = "The specified 
'key' is not within the provided token value bounds.";
+
+    private static final String COLUMN_KEYSPACE_NAME = "keyspace_name";
+    private static final String COLUMN_TABLE_NAME = "table_name";
+    private static final String COLUMN_TOKEN_VALUE = "token_value";
+    private static final String COLUMN_KEY = "key";
+    private static final String COLUMN_SIZE_ESTIMATE = "size_estimate";
+    private static final String COLUMN_SSTABLES = "sstables";
+
+    private final TableMetadata metadata;
+    private final ColumnMetadata sizeEstimateColumn;
+    private final ColumnMetadata sstablesColumn;
+
+    @VisibleForTesting
+    final CopyOnWriteArrayList<Consumer<DecoratedKey>> readListener = new 
CopyOnWriteArrayList<>();
+
+    public PrimaryIdTable(String keyspace)
+    {
+        this.metadata = TableMetadata.builder(keyspace, NAME)
+                                     .kind(TableMetadata.Kind.VIRTUAL)
+                                     .partitioner(new 
LocalPartitioner(CompositeType.getInstance(UTF8Type.instance, 
UTF8Type.instance)))
+                                     
.addPartitionKeyColumn(COLUMN_KEYSPACE_NAME, UTF8Type.instance)
+                                     .addPartitionKeyColumn(COLUMN_TABLE_NAME, 
UTF8Type.instance)
+                                     .addClusteringColumn(COLUMN_TOKEN_VALUE, 
IntegerType.instance)
+                                     .addClusteringColumn(COLUMN_KEY, 
UTF8Type.instance)
+                                     .addRegularColumn(COLUMN_SIZE_ESTIMATE, 
CounterColumnType.instance)
+                                     .addRegularColumn(COLUMN_SSTABLES, 
CounterColumnType.instance)

Review Comment:
   it doesnt actually use counters in normal sense, its because if theres 2 
sstables, the cell resolver for the UnfilteredRowIterator will merge them by 
adding them together instead of replacing with lww



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

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


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

Reply via email to