Skip to content

system.kafka_consumers table to monitor kafka consumers#50999

Merged
kssenii merged 36 commits intoClickHouse:masterfrom
ilejn:system_kafka_consumers
Aug 19, 2023
Merged

system.kafka_consumers table to monitor kafka consumers#50999
kssenii merged 36 commits intoClickHouse:masterfrom
ilejn:system_kafka_consumers

Conversation

@ilejn
Copy link
Copy Markdown
Contributor

@ilejn ilejn commented Jun 14, 2023

Changelog category (leave one):

  • New Feature

Changelog entry (a user-readable short description of the changes that goes to CHANGELOG.md):

system table to monitor kafka consumers

Documentation entry for user-facing changes

  • [x ] Documentation is written (mandatory for new features)

@ilejn
Copy link
Copy Markdown
Contributor Author

ilejn commented Jun 14, 2023

Implements #5128 (comment)

@robot-clickhouse robot-clickhouse added the pr-feature Pull request with new product feature label Jun 14, 2023
@robot-clickhouse
Copy link
Copy Markdown
Member

robot-clickhouse commented Jun 14, 2023

This is an automated comment for commit ade9c3d with description of existing statuses. It's updated for the latest CI running
The full report is available here
The overall status of the commit is 🔴 failure

Check nameDescriptionStatus
AST fuzzerRuns randomly generated queries to catch program errors. The build type is optionally given in parenthesis. If it fails, ask a maintainer for help🟢 success
CI runningA meta-check that indicates the running CI. Normally, it's in success or pending state. The failed status indicates some problems with the PR🟢 success
ClickHouse build checkBuilds ClickHouse in various configurations for use in further steps. You have to fix the builds that fail. Build logs often has enough information to fix the error, but you might have to reproduce the failure locally. The cmake options can be found in the build log, grepping for cmake. Use these options and follow the general build process🟢 success
Compatibility checkChecks that clickhouse binary runs on distributions with old libc versions. If it fails, ask a maintainer for help🟢 success
Docker image for serversThe check to build and optionally push the mentioned image to docker hub🟢 success
Docs CheckBuilds and tests the documentation🟢 success
Fast testNormally this is the first check that is ran for a PR. It builds ClickHouse and runs most of stateless functional tests, omitting some. If it fails, further checks are not started until it is fixed. Look at the report to see which tests fail, then reproduce the failure locally as described here🟢 success
Flaky testsChecks if new added or modified tests are flaky by running them repeatedly, in parallel, with more randomization. Functional tests are run 100 times with address sanitizer, and additional randomization of thread scheduling. Integrational tests are run up to 10 times. If at least once a new test has failed, or was too long, this check will be red. We don't allow flaky tests, read the doc🟢 success
Install packagesChecks that the built packages are installable in a clear environment🟢 success
Integration testsThe integration tests report. In parenthesis the package type is given, and in square brackets are the optional part/total tests🟢 success
Mergeable CheckChecks if all other necessary checks are successful🟢 success
Performance ComparisonMeasure changes in query performance. The performance test report is described in detail here. In square brackets are the optional part/total tests🔴 failure
Push to DockerhubThe check for building and pushing the CI related docker images to docker hub🟢 success
SQLancerFuzzing tests that detect logical bugs with SQLancer tool🟢 success
SqllogicRun clickhouse on the sqllogic test set against sqlite and checks that all statements are passed🟢 success
Stateful testsRuns stateful functional tests for ClickHouse binaries built in various configurations -- release, debug, with sanitizers, etc🟢 success
Stateless testsRuns stateless functional tests for ClickHouse binaries built in various configurations -- release, debug, with sanitizers, etc🟢 success
Stress testRuns stateless functional tests concurrently from several clients to detect concurrency-related errors🟢 success
Style CheckRuns a set of checks to keep the code style clean. If some of tests failed, see the related log from the report🟢 success
Unit testsRuns the unit tests for different release types🟢 success
Upgrade checkRuns stress tests on server version from last release and then tries to upgrade it to the version from the PR. It checks if the new server can successfully startup without any errors, crashes or sanitizer asserts🟢 success

@kssenii kssenii self-assigned this Jun 14, 2023
@ilejn
Copy link
Copy Markdown
Contributor Author

ilejn commented Jun 14, 2023

Hello @kssenii , thank you for assigning this PR.
[Just to let you know] that it is not ready for review yet.
Main gap - librdkafka_statistics is not covered, I hope to handle it soon.

@ilejn ilejn force-pushed the system_kafka_consumers branch from 13a3e88 to f9653a3 Compare July 1, 2023 22:00
Copy link
Copy Markdown
Contributor

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

No sure how expensive is that, but i doubt it comes 'for free'. I think that every few seconds - should be enough (but it would be nice to measure the impact). You can try this one https://github.com/filimonov/ch-kafka-consume-perftest

Copy link
Copy Markdown
Contributor Author

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

I've tweaked 0025-timers.c test https://github.com/ilejn/librdkafka/blob/prof_stat_cb/tests/0025-timers.c
No visible delays even with 1 millisecond interval.
Although

  • json is abridged for several first calls (IOW for 2-4 milliseconds)
  • I do not have consumers, so the test is not really meaningful (creating many topics did not change anything).

Copy link
Copy Markdown
Contributor

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

abs(dateDiff('second', d, now())) < 20

risky. CI/CD sometimes works superslow... Maybe we can push there a fixed timestamp (begining of the test) and compare to that?

Copy link
Copy Markdown
Contributor Author

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

Actually comparing to now() seems more load-agnostic than comparing to a time that had obtained before.
20 seconds should be enough, but ok, I'll change it to 30.

@ilejn ilejn force-pushed the system_kafka_consumers branch from 446fc05 to 539f127 Compare July 11, 2023 16:29
@ilejn ilejn force-pushed the system_kafka_consumers branch from c8219b9 to 4d60f18 Compare July 19, 2023 08:43
@ilejn ilejn marked this pull request as ready for review July 20, 2023 13:10
@ilejn
Copy link
Copy Markdown
Contributor Author

ilejn commented Jul 20, 2023

I believe that test failures are not caused by my changes.

@ilejn ilejn changed the title WIP system.kafka_consumers system.kafka_consumers table to monitor kafka consumers Jul 20, 2023
Copy link
Copy Markdown
Contributor

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

shouldn't it be used?

Copy link
Copy Markdown
Contributor

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

shouldn't that be per consumer? (instead of global per table).

Copy link
Copy Markdown
Contributor Author

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

The statistic is global (it is complete for every consumer), I do not see why it is better to have individual copy of the same data in all consumers.
@filimonov Do you have concerns behind this suggestion?

Copy link
Copy Markdown
Contributor Author

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

Have to admit that Mikhail is right, statistic is not global [enough].

ilejn@hetzner:~$ echo "select rdkafka_stat from system.kafka_consumers where table='another_kafka';" | PATH=$PATH:$HOME/projects/ClickHouse_master/build/programs/  clickhouse-client | jq | grep topic
          "topic": "another",
          "topic": "another",
  "topics": {
      "topic": "another",
          "topic": "another",
          "topic": "another",
  "topics": {
      "topic": "another",
ilejn@hetzner:~$ echo "select rdkafka_stat from system.kafka_consumers where table='kafka';" | PATH=$PATH:$HOME/projects/ClickHouse_master/build/programs/  clickhouse-client | jq | grep topic
          "topic": "extop",
          "topic": "extop",
  "topics": {
      "topic": "extop",
          "topic": "extop",
          "topic": "extop",
  "topics": {
      "topic": "extop",

So I am moving it to consumers.

@ilejn ilejn force-pushed the system_kafka_consumers branch from 41ce6b3 to ade9c3d Compare August 10, 2023 23:22
@ilejn
Copy link
Copy Markdown
Contributor Author

ilejn commented Aug 11, 2023

Hello Mikhail @filimonov , could you have a look, please.
A perf test degradation is not related.

Copy link
Copy Markdown
Contributor

@filimonov filimonov left a comment

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

LGFM.

One case which is not well covered here is when the exception in the materialized view. At that stage there is no good way to attribute the problem with a particular consumer / topic / offset so it's hard to push that information to kafka_consumers table (well may be push that issue to all the consumers iteratively, but that is also non-perfect).

Maybe we can leave it like that for now. Materialized view errors should be visible in other places (system.errors, system.query_view_log etc) so it's less important

@ilejn
Copy link
Copy Markdown
Contributor Author

ilejn commented Aug 16, 2023

Hello @kssenii ,
master is currently not green, performance comparison failures (mostly aarch64 though) are common.
Should I try to rebase nevertheless?

@kssenii kssenii merged commit c0b8d7e into ClickHouse:master Aug 19, 2023
@Yerkon
Copy link
Copy Markdown

Yerkon commented Sep 26, 2023

Hello @ilejn,
Currently updated clickhouse to ClickHouse (ver. 23.8.2.7). But the table system.kafka_consumers currently is empty, despite to kafka engine consumers are working.
In system.events we have kafka statistics

@ilejn
Copy link
Copy Markdown
Contributor Author

ilejn commented Sep 26, 2023

Hello @ilejn, Currently updated clickhouse to ClickHouse (ver. 23.8.2.7). But the table system.kafka_consumers currently is empty, despite to kafka engine consumers are working. In system.events we have kafka statistics

Hello @Yerkon ,
1.Ensure that system.kafka_consumers and tables with Kafka engine are on the same host and system.kafka_consumers is accessible (e.g. no row level security)
2. Provide additional information about your scenario. Ideally complete reproduction.
3. Integration test which is the part of the PR may be a useful starting point.

@Yerkon

This comment was marked as outdated.

@Slach
Copy link
Copy Markdown
Contributor

Slach commented Sep 27, 2023

@Yerkon
could you share
ls -la /var/lib/clickhouse/metadata/system/
grep -C 10 -i kafka -r /var/lib/clickhouse/preprocessed_configs

@Yerkon
Copy link
Copy Markdown

Yerkon commented Sep 27, 2023

Hello @Slach, problem was already solved. It was some missing grants


/*
* Needed until
* https://github.com/mfontanini/cppkafka/pull/309
Copy link
Copy Markdown
Member

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

Looks like this got merged mfontanini/cppkafka#309
And it worth to update submodule and remove this function.

Copy link
Copy Markdown
Contributor Author

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

In process.

@alexey-milovidov
Copy link
Copy Markdown
Member

@ilejn, you can see that the Kafka library has data races and memory leaks.
Now this feature is in question...

@ilejn
Copy link
Copy Markdown
Contributor Author

ilejn commented Mar 7, 2024

@ilejn, you can see that the Kafka library has data races and memory leaks. Now this feature is in question...

Hello @alexey-milovidov , librdkafka is not the best library in the world indeed, but I am not sure that I got your point.
Are we speaking about librdkafka in general or about system.kafka_consumers?
Do we have a plan to replace librdkafka?

system.kafka_consumers supposed to help customers to find out and mitigate [librd]kafka issues. If librdkafka does have memory leaks, extending system.kafka_consumers to cover these types of issues may be a good starting point to fixing them.
If we are aware of issues with system.kafka_consumers , I willingly fix them.

@filimonov
Copy link
Copy Markdown
Contributor

filimonov commented Mar 8, 2024

that the Kafka library has data races and memory leaks.

Your criticism regarding librdkafka might not be unfounded, yet it's essential to recognize that Kafka's architecture adheres to the principle of "simple broker, complex consumer." This design philosophy necessitates a multitude of sophisticated features that need to be supported on the client side. The complexity inherent in these features, especially given their multi-threaded implementation in pure C, and attempts to overlay a simple external API atop them, naturally give rise to various issues. While librdkafka may not be flawless, it is the official client, supporting all (or nearly all) Kafka features.

As of my knowledge, there are no significantly better alternatives now in c++ world. The temptation to "write your own kafka client with blackjack etc" does exist, yet this usually dissipates upon delving into the details and grasping the scale of the challenge at hand.

Creating something from scratch, like a client for the Kafka 0.8 protocol without complex authentication methods, is feasible within a reasonable timeframe. However, developing a universal client that supports Kafka protocol 1.0 features and beyond is a task that spans months or years, requiring ongoing maintenance thereafter.

To give you an idea of the scale:

  • The librdkafka library itself comprises about 190K lines of code (LOC).
  • The Go client, sarama by IBM, has around 50K LOC.
  • The kafka-python library is approximately 25K LOC.
  • tulios/kafkajs, JavaScript client, totals about 50K LOC.
  • rskafka by influxdata, described as 'A minimal Rust client for Apache Kafka', is around 15K LOC.
  • The Rust client for Apache Kafka, kafka-rust, has about 10K LOC.

(LOC stats from https://ghloc.vercel.app/)

The client implementations with fewer lines of code usually lack support for some important or modern features of Apache Kafka.

Sign up for free to join this conversation on GitHub. Already have an account? Sign in to comment

Labels

pr-feature Pull request with new product feature

Projects

None yet

Development

Successfully merging this pull request may close these issues.

8 participants