[
https://issues.apache.org/jira/browse/CASSANDRA-8180?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=15118892#comment-15118892
]
Stefania commented on CASSANDRA-8180:
-------------------------------------
The latest commit is ready for review
[here|https://github.com/stef1927/cassandra/commit/fb1a4d93dc101143eb3d4062d577e643e8824c73].
CI is still pending however.
bq. Isn't the static row is outside the iteration?
That's a very good point, thanks for pointing this out. So we don't need to
worry about this.
bq. So we can't really trust hasTombstones()? This may be a significant bug.
This method was introduced in this patch but currently the same logic is used
in {{SinglePartitionReadCommand.queryMemtableAndDiskInternal()}} and
{{queryMemtableAndSSTablesInTimestampOrder}} to determine which non overlapping
sstables contain no tombstones at all, see especially the comment in the second
method:
{code}
// This mean that nothing queried by the filter can be in the sstable.
One exception is the top-level partition deletion
// however: if it is set, it impacts everything and must be included.
Getting that top-level partition deletion costs us
// some seek in general however (unless the partition is indexed and is
in the key cache), so we first check if the sstable
// has any tombstone at all as a shortcut.
if (sstable.getSSTableMetadata().maxLocalDeletionTime ==
Integer.MAX_VALUE)
continue; // Means no tombstone at all, we can skip that sstable
{code}
The trouble is that in other places we assume the opposite, for example
{{CompactionController.getFullyExpiredSSTables()}} requires {{maxDeletionTime}}
to be {{Integer.MAX_VALUE}} in the presence of live cells or else it might
expire the sstable. I think the code in {{SinglePartitionReadCommand}} is wrong
and we should have looked at the {{minDeletionTime}} instead, so I've updated
the new method accordingly.
I've had some failures with upgrade dtests, due to the clustering values of old
sstables ("jb" format) containing more values than the comparator types. I've
added a check on {{Version.hasNewStatsFile()}} to prevent this and some
assertions to spot it sooner, and this seems to have fixed the failing tests.
To wrap up the discussion on upgrading sstable stats, this would allow applying
this optimization also to sstables with tombstones and that are smaller than
{{DatabaseDescriptor.column_index_size_in_kb}}, 64kb by default. For larger
sstables with tombstones, the lower bound comes from the partition index and
this will be read anyway in the presence of tombstones when
{{partitionDeletion()}} is called. This requires also initializing the
iterator, which will cause a disk seek to the partition start if the partition
index is not available (sstables smaller than 64kb) or there are static columns
in the query, see the {{AbstractSSTableIterator}} constructor.
Therefore I don't see the point in upgrading the stats, but should we wish to
do it, here is the work required:
- upgrade the sstable version from "ma" to "mb" (unsure what this entails
exactly, other than changing {{BigVersion}} and {{BigFormat}})
- introduce a serializer for clustering prefixes that is able to distinguish
between clusterings and bounds by using functionality similar to what's in
{{UnfilteredSerializer}}
- pass the clustering types, or ideally the {{SerializationHeader}}, to the
{{StatsMetadata}} serialize and deserialize methods - or alternatively add the
min and max clustering prefixes to {{SerializationHeader.Component}}, which is
also a metadata component. However in this case we'd have to do some more work
to pass to it the min and max clustering prefixes, which can instead be easily
collected for {{StatsMetadata}} by using the existing method
{{MetadataCollector.updateClusteringValues()}}. I suppose they could also go
into {{EncodingStats}}, which is serialized in the header, but I am not sure if
this would increase the size of partitions excessively because it seems we also
use {{EncodingStats}} for partitions.
> Optimize disk seek using min/max column name meta data when the LIMIT clause
> is used
> ------------------------------------------------------------------------------------
>
> Key: CASSANDRA-8180
> URL: https://issues.apache.org/jira/browse/CASSANDRA-8180
> Project: Cassandra
> Issue Type: Improvement
> Components: Local Write-Read Paths
> Environment: Cassandra 2.0.10
> Reporter: DOAN DuyHai
> Assignee: Stefania
> Priority: Minor
> Fix For: 3.x
>
> Attachments: 8180_001.yaml, 8180_002.yaml
>
>
> I was working on an example of sensor data table (timeseries) and face a use
> case where C* does not optimize read on disk.
> {code}
> cqlsh:test> CREATE TABLE test(id int, col int, val text, PRIMARY KEY(id,col))
> WITH CLUSTERING ORDER BY (col DESC);
> cqlsh:test> INSERT INTO test(id, col , val ) VALUES ( 1, 10, '10');
> ...
> >nodetool flush test test
> ...
> cqlsh:test> INSERT INTO test(id, col , val ) VALUES ( 1, 20, '20');
> ...
> >nodetool flush test test
> ...
> cqlsh:test> INSERT INTO test(id, col , val ) VALUES ( 1, 30, '30');
> ...
> >nodetool flush test test
> {code}
> After that, I activate request tracing:
> {code}
> cqlsh:test> SELECT * FROM test WHERE id=1 LIMIT 1;
> activity |
> timestamp | source | source_elapsed
> ---------------------------------------------------------------------------+--------------+-----------+----------------
> execute_cql3_query |
> 23:48:46,498 | 127.0.0.1 | 0
> Parsing SELECT * FROM test WHERE id=1 LIMIT 1; |
> 23:48:46,498 | 127.0.0.1 | 74
> Preparing statement |
> 23:48:46,499 | 127.0.0.1 | 253
> Executing single-partition query on test |
> 23:48:46,499 | 127.0.0.1 | 930
> Acquiring sstable references |
> 23:48:46,499 | 127.0.0.1 | 943
> Merging memtable tombstones |
> 23:48:46,499 | 127.0.0.1 | 1032
> Key cache hit for sstable 3 |
> 23:48:46,500 | 127.0.0.1 | 1160
> Seeking to partition beginning in data file |
> 23:48:46,500 | 127.0.0.1 | 1173
> Key cache hit for sstable 2 |
> 23:48:46,500 | 127.0.0.1 | 1889
> Seeking to partition beginning in data file |
> 23:48:46,500 | 127.0.0.1 | 1901
> Key cache hit for sstable 1 |
> 23:48:46,501 | 127.0.0.1 | 2373
> Seeking to partition beginning in data file |
> 23:48:46,501 | 127.0.0.1 | 2384
> Skipped 0/3 non-slice-intersecting sstables, included 0 due to tombstones |
> 23:48:46,501 | 127.0.0.1 | 2768
> Merging data from memtables and 3 sstables |
> 23:48:46,501 | 127.0.0.1 | 2784
> Read 2 live and 0 tombstoned cells |
> 23:48:46,501 | 127.0.0.1 | 2976
> Request complete |
> 23:48:46,501 | 127.0.0.1 | 3551
> {code}
> We can clearly see that C* hits 3 SSTables on disk instead of just one,
> although it has the min/max column meta data to decide which SSTable contains
> the most recent data.
> Funny enough, if we add a clause on the clustering column to the select, this
> time C* optimizes the read path:
> {code}
> cqlsh:test> SELECT * FROM test WHERE id=1 AND col > 25 LIMIT 1;
> activity |
> timestamp | source | source_elapsed
> ---------------------------------------------------------------------------+--------------+-----------+----------------
> execute_cql3_query |
> 23:52:31,888 | 127.0.0.1 | 0
> Parsing SELECT * FROM test WHERE id=1 AND col > 25 LIMIT 1; |
> 23:52:31,888 | 127.0.0.1 | 60
> Preparing statement |
> 23:52:31,888 | 127.0.0.1 | 277
> Executing single-partition query on test |
> 23:52:31,889 | 127.0.0.1 | 961
> Acquiring sstable references |
> 23:52:31,889 | 127.0.0.1 | 971
> Merging memtable tombstones |
> 23:52:31,889 | 127.0.0.1 | 1020
> Key cache hit for sstable 3 |
> 23:52:31,889 | 127.0.0.1 | 1108
> Seeking to partition beginning in data file |
> 23:52:31,889 | 127.0.0.1 | 1117
> Skipped 2/3 non-slice-intersecting sstables, included 0 due to tombstones |
> 23:52:31,889 | 127.0.0.1 | 1611
> Merging data from memtables and 1 sstables |
> 23:52:31,890 | 127.0.0.1 | 1624
> Read 1 live and 0 tombstoned cells |
> 23:52:31,890 | 127.0.0.1 | 1700
> Request complete |
> 23:52:31,890 | 127.0.0.1 | 2140
> {code}
--
This message was sent by Atlassian JIRA
(v6.3.4#6332)