Hello, I just stumbled upon this library we are using for getting estimations of the number of partitions in a SSTable which are used e.g. in EstimatedPartitionCount metric. (1)
A user reported in (1) that it is an expensive operation. When one looks into what it is doing, it calls SSTableReader.getApproximateKeyCount() (6) which basically goes to disk every single time, it loads all Stats components and it looks into CompactionMetadata where the cardinality estimator is located. We are serializing the hyperloglog to disk as part of a SSTable and we deserialize it back in runtime for every SSTable in a CF and we merge them all to one cardinality again. I do not think there is a way around this because of the nature of how a cardinality estimator works (hyperloglog). We can not "cache it", it would work only in case we are adding SSTables only - hence we would just merge again - but if we remove an SSTable as part of the compaction, we can not "unmerge" it. That being said, we are currently using this library for hyperloglog (1) which was archived in summer 2020 and nothing was contributed to that for 6 years. That lib is dead. There is very nice replacement of that (2) directly from Apache (!!!) and they are even giving the detailed and in-depth comparison of hyperloglog implementation found in stream-lib we happen to use (3) (stream-lib = Clearspring) where they are saying that updating is way faster and it is also giving better estimations in general. I have implemented the usage of both cardinality estimators (4), (5). The reason we need to keep the old one around is that we may have old SSTables around and we need to work with them too. That translates to a new SSTable version (ob) which uses new implementation and for versions < ob, it uses the old one. When SSTables are upgraded from oa to ob, the old estimator will not be used anymore. There is also a case of a user not upgrading his oa SSTables, turning a node on and creating new SSTables with ob version. When this happens and we ask what is the cardinality (e.g via nodetool tablestats), I am checking if all SSTables are on the same version or not. If they are, they will use either an old or new estimator. (we can not merge estimations from two different hyperloglog implementations). If they are not, it will compute that from index summaries. (The computation for index summaries was already in place (6) as a fail-over in case the estimation computation failed / was not present). Does this all make sense to drive further to the completion and eventually merge this work to trunk? Worth to add that Apache Datasketches are just two dependencies for us, it has zero external dependencies. (1) https://github.com/addthis/stream-lib (2) https://datasketches.apache.org/ (3) https://datasketches.apache.org/docs/HLL/Hll_vs_CS_Hllpp.html (4) https://issues.apache.org/jira/browse/CASSANDRA-13338 (5) https://github.com/apache/cassandra/pull/3767 (6) https://github.com/apache/cassandra/blob/trunk/src/java/org/apache/cassandra/io/sstable/format/SSTableReader.java#L284-L338 Regards