Skip to content
New issue

Have a question about this project? Sign up for a free GitHub account to open an issue and contact its maintainers and the community.

By clicking “Sign up for GitHub”, you agree to our terms of service and privacy statement. We’ll occasionally send you account related emails.

Already on GitHub? Sign in to your account

Global Aggregation over Kafka Stream #54870

Open
wants to merge 48 commits into
base: master
Choose a base branch
from

Conversation

chenziliang
Copy link
Contributor

@chenziliang chenziliang commented Sep 21, 2023

Changelog category (leave one):

  • New Feature

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

  • This feature enables streaming processing over Kafka table engine. User now can opt-in streaming query Kafka table engine directly by using EMIT STREAM clause. Aggregation over Kafka table engine is also supported.

Documentation entry for user-facing changes

  • Documentation is written (mandatory for new features)

Streaming aggregation over Kafka stream. This closes feature #54776 which contains some detail discussion regarding this feature.

Opt-in stream processing over Kafka storage engine via EMIT STREAM. We will need opt-in for backward compatibility with the existing behaviors of Kafka table engine.

Examples :

SELECT * FROM kafka_topic EMIT STREAM;  -- Streaming tail 

SELECT count() FROM kafka_topic EMIT STREAM PERIODIC 1s;  -- emit intermediate result every 1s.

Input formats which are supported by existing Kafka table engine keep as they are.

@CLAassistant
Copy link

CLAassistant commented Sep 21, 2023

CLA assistant check
All committers have signed the CLA.

@clickhouse-ci
Copy link

clickhouse-ci bot commented Sep 21, 2023

This is an automatic comment. The PR descriptions does not match the template.

Please, edit it accordingly.

The error is: Changelog entry required for category 'New Feature'

1 similar comment
@clickhouse-ci
Copy link

clickhouse-ci bot commented Sep 21, 2023

This is an automatic comment. The PR descriptions does not match the template.

Please, edit it accordingly.

The error is: Changelog entry required for category 'New Feature'

@KochetovNicolai KochetovNicolai added the can be tested Allows running workflows for external contributors label Sep 26, 2023
@robot-ch-test-poll1 robot-ch-test-poll1 added the pr-feature Pull request with new product feature label Sep 26, 2023
@robot-ch-test-poll1
Copy link
Contributor

robot-ch-test-poll1 commented Sep 26, 2023

This is an automated comment for commit 7710a92 with description of existing statuses. It's updated for the latest CI running

❌ Click here to open a full report in a separate page

Successful checks
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
ClickBenchRuns [ClickBench](https://github.com/ClickHouse/ClickBench/) with instant-attach table✅ 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
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✅ success
SQLTestThere's no description for the check yet, please add it to tests/ci/ci_config.py:CHECK_DESCRIPTIONS✅ 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
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
Check nameDescriptionStatus
Integration testsThe integration tests report. In parenthesis the package type is given, and in square brackets are the optional part/total tests❌ failure
Stateless testsRuns stateless functional tests for ClickHouse binaries built in various configurations -- release, debug, with sanitizers, etc❌ failure
Stress testRuns stateless functional tests concurrently from several clients to detect concurrency-related errors❌ failure
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❌ failure

@qijun-niu-timeplus qijun-niu-timeplus force-pushed the feature/issue-54776-aggr-kafka-stream branch 5 times, most recently from ab75a79 to c36c139 Compare October 7, 2023 13:18
@qijun-niu-timeplus qijun-niu-timeplus force-pushed the feature/issue-54776-aggr-kafka-stream branch 2 times, most recently from aaa32a5 to 80d13be Compare October 16, 2023 13:20
@clickhouse-ci
Copy link

clickhouse-ci bot commented Oct 18, 2023

This is an automatic comment. The PR descriptions does not match the template.

Please, edit it accordingly.

The error is: Changelog entry required for category 'New Feature'

@chenziliang chenziliang marked this pull request as ready for review October 18, 2023 05:08
@clickhouse-ci
Copy link

clickhouse-ci bot commented Oct 18, 2023

This is an automatic comment. The PR descriptions does not match the template.

Please, edit it accordingly.

The error is: Changelog entry required for category 'New Feature'

@chenziliang
Copy link
Contributor Author

chenziliang commented Oct 18, 2023

No idea why docs check failed. @KochetovNicolai, may you help point out the missing part regarding the doc ?

@chenziliang
Copy link
Contributor Author

No idea why docs check failed. @KochetovNicolai, may you help point out the missing part regarding the doc ?

Ok, need add doc in the source code as well

@@ -0,0 +1,1014 @@
#pragma once
Copy link
Contributor Author

Choose a reason for hiding this comment

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

We choose to copy and revise the Aggregator because we like to avoid impacting the efficiency / performantce for historical data processing and also keep itself performant / efficient for streaming processing. And aggregator will have these major different tweaks / behaviors / revisions for now and moving forward.

  1. Keep around states after intermediate aggregation emit. (We have this in the current PR)
  2. Garbage collect aggregate states when watermark progress for streaming window process (in future PR)
  3. Interface changes -> Block => Chunk to make it more lightweight / efficient for streaming processing. (future PR)
  4. State checkpointing and restore. In streaming processing, when enable checkpoint, we will need checkpoint the in-memory state and restore it when the query gets recovered. (future PR)

Copy link
Member

@antaljanosbenjamin antaljanosbenjamin left a comment

Choose a reason for hiding this comment

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

Just a few comments as I started to go through the code.

src/Columns/ColumnAggregateFunction.h Outdated Show resolved Hide resolved
src/Common/ErrorCodes.cpp Show resolved Hide resolved
src/Common/HashTable/TwoLevelHashTable.h Outdated Show resolved Hide resolved
src/Core/Settings.h Outdated Show resolved Hide resolved
src/Interpreters/Streaming/WindowCommon.h Outdated Show resolved Hide resolved
src/Interpreters/Streaming/WindowCommon.h Outdated Show resolved Hide resolved
src/Interpreters/Streaming/WindowCommon.cpp Outdated Show resolved Hide resolved
throw Exception(ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT, "{}: the max interval kind supported is DAY.", msg);
}
}
throw Exception(ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT, "{}", msg);
Copy link
Member

Choose a reason for hiding this comment

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

Please ha a specific message here. We have a test that checks that we have meaningful message patterns.

Copy link
Contributor Author

Choose a reason for hiding this comment

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

Will tackle and revise it.

src/Interpreters/Streaming/WindowCommon.cpp Outdated Show resolved Hide resolved
src/Interpreters/Streaming/EmitInterpreter.h Show resolved Hide resolved
@qijun-niu-timeplus qijun-niu-timeplus force-pushed the feature/issue-54776-aggr-kafka-stream branch 2 times, most recently from b8bc02a to b9ea0ef Compare December 20, 2023 08:56
@qijun-niu-timeplus qijun-niu-timeplus force-pushed the feature/issue-54776-aggr-kafka-stream branch from 43f18b3 to 7e6ad75 Compare December 23, 2023 12:55
@qijun-niu-timeplus qijun-niu-timeplus force-pushed the feature/issue-54776-aggr-kafka-stream branch from 41fc28f to 7710a92 Compare December 23, 2023 13:36
@yokofly
Copy link
Contributor

yokofly commented Dec 24, 2023

stateless test fail: 01459_manual_write_to_replicas_quorum
this is flaky right?

@antaljanosbenjamin antaljanosbenjamin self-assigned this Jan 4, 2024
Copy link
Member

@antaljanosbenjamin antaljanosbenjamin left a comment

Choose a reason for hiding this comment

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

In general I like the changes (I briefly checked out GlobalAggregatingTransform and AggregatingTransform), however what is expected regarding to watermarks in unclear. Could you please write either a comment in the code or a comment here on GitHub about watermarks:

  • What can we expect regarding them? They produced in a monotonic way, but can they be mixed/shuffled in the pipeline and if so, is that a problem?
  • What extra guarantees or functionality do they give? Do they have any guarantees or just best effort?
  • Maybe if you have in general any good documentation for Proton/article or something about a nice description of watermarks you have implemented, that is also very welcome.

@@ -63,6 +63,10 @@ struct TreeRewriterResult
/// Note: not used further.
NameToNameMap array_join_name_to_alias;

bool streaming = true;
Copy link
Member

Choose a reason for hiding this comment

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

I would say this is a bit dangerous default value. Let's set it to false by default.

Copy link
Contributor Author

Choose a reason for hiding this comment

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

Agree. T+ Proton is streaming by default. ClickHouse shall be different.

Comment on lines +430 to +434
if (settings.optimize_uniq_to_count)
{
RewriteUniqToCountMatcher::Data data_rewrite_uniq_count;
RewriteUniqToCountVisitor(data_rewrite_uniq_count).visit(query_ptr);
}
Copy link
Member

Choose a reason for hiding this comment

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

Isn't this a duplicate of lines 447-451?

Copy link
Contributor Author

Choose a reason for hiding this comment

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

Yes, i think this one is a cherry pick (from lots of changes) and diff mistake. We will remove these lines.

Comment on lines +29 to +30
/// Optional `STREAM` keyword
ParserKeyword("STREAM").ignore(pos, expected);
Copy link
Member

Choose a reason for hiding this comment

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

Why do we have STREAM if we don't do anything with it? Is this some kind of comformance thing?

Copy link
Contributor Author

Choose a reason for hiding this comment

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

Yes, it is more like a conformance thing. We can remove it. https://arxiv.org/pdf/1905.12133.pdf

Comment on lines +39 to +40
if (periodic_interval)
throw Exception(ErrorCodes::SYNTAX_ERROR, "Can not use repeat 'PERIODIC' in EMIT clause");
Copy link
Member

Choose a reason for hiding this comment

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

How this can be true with a local variable?

Copy link
Contributor Author

Choose a reason for hiding this comment

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

CherryPick partial commit caused this issue. https://github.com/timeplus-io/proton/blob/develop/src/Parsers/Streaming/ParserEmitQuery.cpp#L50-L60

We can revise this.

/// 'months' - 'M'
/// 'quarter' - 'q'
/// 'year' - 'y'
class ParserIntervalAliasExpression : public IParserBase
Copy link
Member

Choose a reason for hiding this comment

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

I think before making this feature not experimental, we have to unify the parsing of these expressions and other interval expressions (e.g.: how TTL can be specified). But This is not a blocker to merge this PR.

Copy link
Contributor Author

Choose a reason for hiding this comment

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

This is a good idea. We shall have a consistent way to handle interval alias like 5s vs INTERVAL 5 SECONDS and there are other alias like JSON shortcuts like jsonstring:a.b.c::int vs jsonExtract(jsonstring, "a.b.c") etc

@@ -275,6 +280,24 @@ ASTPtr StorageView::restoreViewName(ASTSelectQuery & select_query, const ASTPtr
return subquery->children[0];
}

bool StorageView::isStreamingQuery(ContextPtr query_context) const
Copy link
Member

Choose a reason for hiding this comment

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

@KochetovNicolai is it okay to depend on the new Analyzer in the storage layer?

Copy link
Member

Choose a reason for hiding this comment

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

Yes, I think it is ok.
We can support streaming with only a new analyzer if needed.

Comment on lines +29 to +31
NONE,
TAIL,
PERIODIC
Copy link
Member

Choose a reason for hiding this comment

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

Are you planning to introduce more emit modes? If not, then as far as I understood only PERIODIC does anything, TAIL and NONE doesn't do anything, so it would be better to remove them. Furthermore, I think based on the InterpreterSelectQuery::buildStreamingProcessingQueryPlanAfterJoin function we only use the watermarks in case of PERIODIC queries, therefore WatermarkStamper won't be used with NONE or TAIL (if I am wrong, please explain how it can be used).

Copy link
Contributor Author

Choose a reason for hiding this comment

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

yes, we will introduce more emit modes like EMIT CHANGELOG, EMIT AFTER WATERMARK, EMIT AFTER WATERMARK AND DELAY 5s. So more WatermarkStampers will chime in later PRs.

#include <Columns/IColumn.h>
#include <unordered_map>

namespace DB
{

struct ChunkContext
Copy link
Member

Choose a reason for hiding this comment

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

I think in our codebase Context is very overloaded, from ChunkContext I associate to the Context object which is used in different places as global/session/query context. Maybe ChunkWatermarkInfo or something without Context would be a better name.

This is a very minor thing, feel free to ignore.

Copy link
Contributor Author

Choose a reason for hiding this comment

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

We have considered similar thing in Proton code base as well, but as time goes, we stuff more things there like checkpoint context, so finally we decided using ChunkContext. Agree, context is overloaded.


bool updated = false;
auto new_watermark = chunk.getWatermark();
if (new_watermark > input_with_data.watermark || (input_with_data.watermark == TIMEOUT_WATERMARK && new_watermark >= aligned_watermark))
Copy link
Member

Choose a reason for hiding this comment

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

How the new_watermark > input_with_data.watermark can be false?

Copy link
Contributor Author

Choose a reason for hiding this comment

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

When we do alignment, watermarks can be from different data sources, so it is possible new_watermark can be from a fast input.

assert(num_inputs > 0);
}

IProcessor::Status ShrinkResizeProcessor::prepare(const PortNumbers & updated_inputs, const PortNumbers & /*updated_outputs*/)
Copy link
Member

Choose a reason for hiding this comment

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

I need a bit of explanation about watermarks to understand this and updateAndAlignWatermark functions.

  • Is it expected that the different inputs has the same source of watermarks, so a single WatermarkStamper before an ExpandResizeProcessor?
  • Can data from different watermarks be mixed? E.g. we have a 5s periodic emit query. Can chunks before and after a water be sent to the output without sending the watermark? Isn't the goal of watermarks to keep the data and the outputted result "in sync"?

Copy link
Contributor Author

Choose a reason for hiding this comment

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

I need a bit of explanation about watermarks to understand this and updateAndAlignWatermark functions.

  • Is it expected that the different inputs has the same source of watermarks, so a single WatermarkStamper before an ExpandResizeProcessor?

Different inputs have different source of watermarks (timestamps). Watermarks are stamped (observed) independently for each of time in the code. However, when we reach a Resize, Aggregate, Join operators, we will need align them (which fundamentally is trying to align the timestamps - what time it is now). After alignment, we established a timestamp for the the down stream query pipeline.

  • Can data from different watermarks be mixed? E.g. we have a 5s periodic emit query. Can chunks before and after a water be sent to the output without sending the watermark? Isn't the goal of watermarks to keep the data and the outputted result "in sync"?

Watermark can be mixed and will need re-alignment when we hit a Resize etc operator like mentioned above. The sole goal of watermark is answering what time is it and all of the aggr, join emit, state GC etc are based on the watermarks.

@chenziliang
Copy link
Contributor Author

In general I like the changes (I briefly checked out GlobalAggregatingTransform and AggregatingTransform), however what is expected regarding to watermarks in unclear. Could you please write either a comment in the code or a comment here on GitHub about watermarks:

  • What can we expect regarding them? They produced in a monotonic way, but can they be mixed/shuffled in the pipeline and if so, is that a problem?

We will need do watermark alignment when mixed and shuffled. The current strategy is std::min(watermarks_from_inputs) of a processor.

For idle input or slow input, we can choose either 1) wait which stalls the pipeline or 2) a heartbeat watermark which pushes the watermark progress. In this PR, we have heartbeat watermark for each source (which is the periodic one)

  • What extra guarantees or functionality do they give? Do they have any guarantees or just best effort?
    For the watermark, we observe the timestamp of the sources or we generate by ourselves (periodical one). If watermark is based on timestamp of the source, it is accurate and for periodical watermark, we have 50 milliseconds (which is configurable in Proton/T+) accuracy by default (we don't want hearbeat too often and we don't want heartbeat to frequently neither).

We will have some mechanism to handle late / out of order events like using EMIT AFTER WATERMARK DELAY 5s to honor our of order or late events in 5 seconds threshold. And these will be different PRs.

  • Maybe if you have in general any good documentation for Proton/article or something about a nice description of watermarks you have implemented, that is also very welcome.

Watermarks which is timestamp fundamentally are one of the fundamental thing in streaming query processing / analytics. Proton borrowed lots of concepts from Flink but implemented in a very different way. There are quite a few academic papers which impact Proton's implementation. You can check this blog post regarding the details https://www.timeplus.com/post/unify-streaming-and-historical-data-processing

Thanks Kovi for the review and comments again. Answers inlined here

Sign up for free to join this conversation on GitHub. Already have an account? Sign in to comment
Labels
can be tested Allows running workflows for external contributors pr-feature Pull request with new product feature
Projects
None yet
Development

Successfully merging this pull request may close these issues.

None yet

8 participants