Hello,

I wanted to throw some ideas and a vision in terms of metrics,
trancing and the adoption of new integrations, particularly
OpenTelemetry. I personally feel that the more integrations we have,
the better the adoption of Cassandra as a database will be. With
OpenTelemetry, users could have a better "first experience", so I'm +1
here.

I have two concerns with the way we currently handle such integrations:

1. The first is how do we manage all these integrations, because
according to the CEP we are adding new dependencies and interfaces [1]
to the project and adding new configuration values, this is not bad in
itself. However, it also means that as the number of integrations
increases, so does the maintenance of the project and config (the
vision - is to have minimal extra deps in the core and the smallest
config).

2. Exporting metrics/logs should not affect the node itself (adjusting
the JVM params [2] of the node to make the integration work tells us
that we are doing something wrong) and the JVM process that does the
main work with the data by handling user requests. The priority of
serving metrics/logs is lower than a user request. The current
approach of adding new metric exporters and/or instrumenting JVM
agents could affect the stability and performance of the node, the
bugs could prevent the node from serving user requests as well (e.g.
calculating instead of exporting raw histograms [3] causing gcs and
impacts the node).



With all that, the alternate solution and the vision I'm trying to
highlight here is that we should just rely on the native protocol and
"incorporate" these things into the native protocol itself and CQL as
its part.
That way, Cassandra Sidecar and other sub-projects interested in the
internal state of the node can rely only on the protocol specification
and the query syntax.

Specifically, querying the node's internal state (basically metrics
and logs) is being done using two paradigms: "poll" and "push".

1. The "poll" is the simplest part, we already have all we need - lots
of virtual tables. A new virtual keyspace "system_metrics" [4]
contains all the internal metrics in the Prometheus format that
Cassandra exposes in JMX, and can be queried by any other system (e.g.
the Cassandra Sidecar that has established a local connection via Unix
Domain Socket to query the metrics) to expose them via the REST API or
other interfaces they need. The efficiency of exposing these metrics
is the best we can offer in terms of performance (I hope).

2. The "push" is currently and unfortunately is not implemented - but
normally is used and designed to export logs and internal events. The
vision is - to register a continuous query to listen for the log
updates on the node, which is also a part of the Sidecar. Such a
feature would be useful in itself, regardless of the fact that in our
case we are going to use it to listen to internal events and log
updates. From my point of view, other database vendors offer something
similar that Cassandra lacks:

https://cloud.google.com/bigquery/docs/continuous-queries-introduction
https://docs.influxdata.com/influxdb/v1/query_language/continuous_queries/

The syntax could be:

CREATE CONTINUOUS QUERY cq_log_updates ON system_queries
BEGIN
  SELECT timestamp, text FROM system_logs
END

These two paradigms can be collected under the umbrella of the
Cassandra Sidecar to support the OpenTelemetry without having extra
config and extra dependency and leaving the Cassandra core with no
additional dependencies and free to focus on the native protocol and
CQL.



By far, it was all about metrics and logs. I also took a look at the
opentelemetry-demo project to see what a trace might look like. The
example [5] shows a good picture of a tracing request up to the point
where the request reaches the core, and from that point on the trace
picture becomes foggy, showing almost nothing about Cassndra
internals. I think this part should be improved by adding traceable
types for each of the internal components we want to trace (messaging
between nodes, the waiting time in a pool, the time to parse a request
etc.). For instance, here is an example [6] of what the granularity
could look like.

With that, and since we would somehow have specific trace types for
Cassandra to view on internal components, I think the point where we
can bind Cassandra traces to the OpenTelemetry trace context is the
Sidecar (the best candidate from my point of view). Theoretically, we
could use the opentelemetry-java-instrumentation [7] to get an
overview of the internals, but without custom Cassandra trancing types
it won't give us good granularity and a clear picture of the node
internals.


Please don't take this as a criticism or -1 on my part, just wanted to
share an alternative way it could have been seen. The amount of extra
dependencies scares me so much :-)


[1] 
https://cwiki.apache.org/confluence/display/CASSANDRA/CEP-32%3A+%28DRAFT%29+OpenTelemetry+integration#CEP32:(DRAFT)OpenTelemetryintegration-NeworChangedPublicInterfaces
[2] 
https://cwiki.apache.org/confluence/display/CASSANDRA/CEP-32%3A+%28DRAFT%29+OpenTelemetry+integration#CEP32:(DRAFT)OpenTelemetryintegration-Necessaryconfigchange
[3] https://issues.apache.org/jira/browse/CASSANDRA-8052
[4] 
https://github.com/apache/cassandra/blob/trunk/doc/modules/cassandra/pages/managing/operating/virtualtables.adoc#virtual-tables-system_metrics-keyspace
[5] 
https://github.com/yukim/cassandra-opentelemetry-demo/blob/main/cassandra_tracing_in_lightstep.png
[6] 
https://github.com/Mmuzaf/ignite/blob/master/modules/core/src/main/java/org/apache/ignite/internal/processors/tracing/SpanType.java#L75
[7] https://github.com/open-telemetry/opentelemetry-java-instrumentation

On Tue, 22 Oct 2024 at 10:32, Michael Burman <y...@iki.fi> wrote:
>
> Hi,
>
> > I'd really, really like to see us ship a Prom compatible metrics endpoint 
> > out of the box in C* that has low overhead.  All the current OSS metrics 
> > exporters that I've seen have massive overhead.  I'm specifically looking 
> > for sub-10s collection on clusters with a thousand nodes and 500+ tables.  
> > That means going directly to DropWizard and skipping JMX.
>
> This is what we're doing in the management-api metrics endpoint. We poll the 
> DropWizard metrics directly and then modify the values to a standard 
> Prometheus output. The design goals included fast performance and near-zero 
> GC load. I tested the implementation using 8000 tables and on my old laptop I 
> was able to read some ~24 million datapoints per second. At that point, the 
> constraints are on the network side (even with compression which we support) 
> and what Prometheus / Mimir / Thanos / etc are able to receive.
>
> In reality, the TSDB is always going to be the limiting side, not what we can 
> parse from Cassandra. Since filtering on the Prometheus polling side was too 
> slow and would require first transferring all the data there, we opted to 
> implement also the replacements directive of Prometheus on the server side, 
> so one can for example filter out all the table metrics before anything is 
> transferred. That helps in our testing with the processing load on the 
> Prometheus side (since large amounts of metrics would overwhelm the 
> Prometheus).
>
> The implementation is available here:
>
> https://github.com/k8ssandra/management-api-for-apache-cassandra/tree/master/management-api-agent-common/src/main/java/io/k8ssandra/metrics
>
> There's no need to use the rest of the management-api features if you don't 
> want to, simply deploy the agent to get access to this output (it will answer 
> on localhost:9000/metrics).
>
> We do implement some extra metrics that are not available in Cassandra 
> DropWizard also (such as per compaction / per streaming process status), but 
> these are separated under namespace org_apache_cassandra_metrics_extended_*
>
>   - Micke
>
> On Thu, 3 Oct 2024 at 17:59, Patrick McFadin <pmcfa...@gmail.com> wrote:
>>
>> Sooooo. That's a +1 from you, Jon? Just want to make sure.
>>
>> On Thu, Oct 3, 2024 at 7:17 AM Jon Haddad <j...@rustyrazorblade.com> wrote:
>>>
>>> I love that we're having a discussion about observability.  A HUGE thank 
>>> you to anyone willing to invest time improving it in Cassandra.
>>>
>>> I'd really, really like to see us ship a Prom compatible metrics endpoint 
>>> out of the box in C* that has low overhead.  All the current OSS metrics 
>>> exporters that I've seen have massive overhead.  I'm specifically looking 
>>> for sub-10s collection on clusters with a thousand nodes and 500+ tables.  
>>> That means going directly to DropWizard and skipping JMX.
>>>
>>> I put together a POC of it a while ago here: 
>>> https://github.com/rustyrazorblade/cassandra-prometheus-exporter.  Please 
>>> use commit 434be099d5983d537e2c70aad745194e575bc49a as a reference.  I 
>>> wasn't expecting anyone to actually care about the repo and the last commit 
>>> broke it.  There's some optimizations that could be done to further improve 
>>> the exporter, I was working on that when I broke the repo :/
>>>
>>> For industry comparison the following DBs either ship entire monitoring 
>>> stacks or provide strong recommendations / solutions:
>>>
>>> * ScyllaDB: https://www.scylladb.com/product/scylladb-monitoring-stack/
>>> * Cockroach: https://www.cockroachlabs.com/docs/v24.2/ui-overview-dashboard
>>> * Aerospike: 
>>> https://aerospike.com/docs/monitorstack/new/components-of-monitoring-stack
>>> * MongoDB: https://www.mongodb.com/products/platform/atlas-charts/dashboard
>>> * Elastic: 
>>> https://www.elastic.co/guide/en/elasticsearch/reference/8.15/monitoring-production.html
>>> * Redis: https://grafana.com/grafana/dashboards/12776-redis/
>>>
>>> Re: Logs - I wouldn't write off OTel logging [1].  OTel logs can be tagged 
>>> with metadata including the span allowing you to do some really useful 
>>> diagnostics.  It's a significant improvement over standard logging.
>>>
>>> Anyways - I don't have a strong opinion on how the CEPs are done.  
>>> Different ones or together, whichever works.  I hope we can finally get a 
>>> good metrics solution because that's an area of significant pain for end 
>>> users.  A lot of teams don't even have Cassandra dashboards because we 
>>> currently provide zero direction.
>>>
>>> Jon
>>>
>>> [1] https://opentelemetry.io/docs/specs/otel/logs/
>>>
>>> Logs can be correlated with the rest of observability data in a few 
>>> dimensions:
>>>
>>> * By the time of execution. Logs, traces and metrics can record the moment 
>>> of time or the range of time the execution took place. This is the most 
>>> basic form of correlation.
>>>
>>>  * By the execution context, also known as the trace context. It is a 
>>> standard practice to record the execution context (trace and span ids as 
>>> well as user-defined context) in the spans. OpenTelemetry extends this 
>>> practice to logs where possible by including TraceId and SpanId in the 
>>> LogRecords. This allows to directly correlate logs and traces that 
>>> correspond to the same execution context. It also allows to correlate logs 
>>> from different components of a distributed system that participated in the 
>>> particular request execution.
>>>
>>>   * By the origin of the telemetry, also known as the Resource context. 
>>> OpenTelemetry traces and metrics contain information about the Resource 
>>> they come from. We extend this practice to logs by including the Resource 
>>> in LogRecords.
>>>
>>>
>>>
>>> On Thu, Oct 3, 2024 at 6:11 AM João Reis <joaor...@apache.org> wrote:
>>>>
>>>> Reducing the scope of CEP-32 to OpenTelemetry Tracing is a good idea (or 
>>>> creating a new one). We recently added OpenTelemetry Tracing support to 
>>>> the C# driver [1] and we also decided to not include Metrics and Logs in 
>>>> this initiative because the driver already provides a way to collect 
>>>> metrics and logs so it's not as important.
>>>>
>>>> I believe there's also efforts to add OpenTelemetry support to the java 
>>>> driver but I'm not sure if it's limited to Tracing or if they include 
>>>> metrics and logs.
>>>>
>>>> [1] 
>>>> https://github.com/datastax/csharp-driver/tree/master/doc/features/opentelemetry#readme
>>>>
>>>> Yuki Morishita <mor.y...@gmail.com> escreveu (terça, 1/10/2024 à(s) 07:13):
>>>>>
>>>>> Hi,
>>>>>
>>>>> Since I have limited time working on the CEP-32, I'd appreciate the 
>>>>> collaboration to make this CEP the reality.
>>>>>
>>>>> Another thing I'm thinking of is to reduce its scope to only the 
>>>>> OpenTelemetry configuration and the way to work only with OpenTelemetry 
>>>>> Tracing.
>>>>>
>>>>> If it's possible to create sub CEPs, I will create the one for tracing, 
>>>>> metrics and logs. Otherwise, I can rewrite the current CEP-32 to only 
>>>>> focus on OpenTelemetry Tracing.
>>>>> Or maybe scrap CEP-32 and create a new one for Tracing.
>>>>>
>>>>>
>>>>> On Mon, Sep 23, 2024 at 11:47 AM Saranya Krishnakumar 
>>>>> <saran.krishna...@gmail.com> wrote:
>>>>>>
>>>>>> Hi Patrick,
>>>>>>
>>>>>> I am interested in working on this CEP collaborating with Yuki. I 
>>>>>> recently worked on adding metrics framework in Apache Cassandra Sidecar 
>>>>>> project.
>>>>>>
>>>>>> Best,
>>>>>> Saranya Krishnakumar
>>>>>>
>>>>>> On Thu, Sep 19, 2024 at 10:57 AM Patrick McFadin <pmcfa...@gmail.com> 
>>>>>> wrote:
>>>>>>>
>>>>>>> Here's another stalled CEP. In this case, no discuss thread or Jira.
>>>>>>>
>>>>>>> Yuki (or anyone else) know the status of this CEP?
>>>>>>>
>>>>>>> https://cwiki.apache.org/confluence/display/CASSANDRA/CEP-32%3A+%28DRAFT%29+OpenTelemetry+integration
>>>>>>>
>>>>>>> Patrick

Reply via email to