From d3566a220d5a4680be73db126a789fd0bd613112 Mon Sep 17 00:00:00 2001 From: Jeroen van Disseldorp Date: Fri, 28 Jun 2024 11:47:27 +0200 Subject: [PATCH] Update docs (#134) * First set of documentation and corresponding code updates * Update toTopicNameExtractor * Syntax and reference updates * Update metric documentation and several related subjects * Remove stores declaration from operations, which was a duplicate of functions declaring state store names * Only parse store names for functions that can use state stores * Update function documentation --- README.md | 2 +- docs/_data/menu.yaml | 2 +- docs/functions.md | 361 +++++++- docs/index.md | 32 +- docs/introduction.md | 127 ++- docs/ksml-language-spec.json | 2 +- docs/ksml-language-spec.md | 201 ++--- docs/notations.md | 56 +- docs/operations.md | 830 +++++++++++------- docs/pipelines.md | 6 +- docs/{getting-started.md => quick-start.md} | 16 +- docs/release-notes.md | 87 +- docs/runners.md | 44 +- docs/streams.md | 3 + docs/types.md | 68 +- .../java/io/axual/ksml/runner/KsmlInfo.java | 4 +- .../runner/prometheus/PrometheusExport.java | 4 +- ksml/NOTICE.txt | 3 +- ksml/pom.xml | 6 + .../parser/FunctionDefinitionParser.java | 14 +- .../parser/PipelineDefinitionParser.java | 2 +- .../parser/PredicateDefinitionParser.java | 2 +- .../main/java/io/axual/ksml/dsl/KSMLDSL.java | 2 +- .../metric/{KSMLMetrics.java => Metrics.java} | 4 +- .../ksml/operation/AggregateOperation.java | 44 +- .../axual/ksml/operation/BaseOperation.java | 15 +- .../ksml/operation/CogroupOperation.java | 2 +- .../ksml/operation/FilterNotOperation.java | 2 +- .../axual/ksml/operation/FilterOperation.java | 2 +- .../axual/ksml/operation/JoinOperation.java | 3 +- .../axual/ksml/operation/OperationConfig.java | 6 +- .../axual/ksml/operation/PeekOperation.java | 2 +- .../axual/ksml/operation/ReduceOperation.java | 9 +- .../ksml/operation/RepartitionOperation.java | 6 +- .../ksml/operation/StoreOperationConfig.java | 6 +- .../ksml/operation/TransformKeyOperation.java | 2 +- .../operation/TransformKeyValueOperation.java | 2 +- ...nsformKeyValueToKeyValueListOperation.java | 2 +- ...TransformKeyValueToValueListOperation.java | 2 +- .../operation/TransformMetadataOperation.java | 2 +- .../operation/TransformValueOperation.java | 3 +- .../parser/FilterNotOperationParser.java | 5 +- .../parser/FilterOperationParser.java | 5 +- .../parser/ForEachOperationParser.java | 3 +- .../operation/parser/JoinOperationParser.java | 12 +- .../parser/LeftJoinOperationParser.java | 6 +- .../operation/parser/OperationParser.java | 8 +- .../operation/parser/PeekOperationParser.java | 3 +- .../parser/PrintOperationParser.java | 2 +- .../parser/ReduceOperationParser.java | 45 +- .../parser/RepartitionOperationParser.java | 5 +- .../parser/StoreOperationParser.java | 6 +- .../parser/SuppressOperationParser.java | 69 +- .../parser/TransformKeyOperationParser.java | 3 +- .../TransformKeyValueOperationParser.java | 3 +- ...KeyValueToKeyValueListOperationParser.java | 3 +- ...ormKeyValueToValueListOperationParser.java | 3 +- .../TransformMetadataOperationParser.java | 3 +- .../parser/TransformValueOperationParser.java | 3 +- .../parser/WindowByTimeOperationParser.java | 101 ++- .../io/axual/ksml/parser/ChoiceParser.java | 1 - .../axual/ksml/parser/DefinitionParser.java | 20 +- .../java/io/axual/ksml/python/Invoker.java | 8 +- .../io/axual/ksml/python/PythonContext.java | 6 +- .../axual/ksml/python/MetricsBridgeTest.java | 4 +- 65 files changed, 1458 insertions(+), 857 deletions(-) rename docs/{getting-started.md => quick-start.md} (91%) rename ksml/src/main/java/io/axual/ksml/metric/{KSMLMetrics.java => Metrics.java} (95%) diff --git a/README.md b/README.md index 332fa023..77350ce8 100644 --- a/README.md +++ b/README.md @@ -89,7 +89,7 @@ To run the KSML demo, we provide a Docker compose file which will start Kafka, c container with a demo producer. You can then start the runner you generated in the previous step, passing in a KSML configuration of your choice. -See [Getting started](docs/getting-started.md) or [Runners](docs/runners.md) for details. +See [Getting started](docs/quick-start) or [Runners](docs/runners.md) for details. To run the demo, Docker 19.x is required. diff --git a/docs/_data/menu.yaml b/docs/_data/menu.yaml index fa538ffd..9866ca7b 100644 --- a/docs/_data/menu.yaml +++ b/docs/_data/menu.yaml @@ -24,4 +24,4 @@ items: url: runners.html - title: Language Specification - url: specification.html \ No newline at end of file + url: ksml-language-spec.html \ No newline at end of file diff --git a/docs/functions.md b/docs/functions.md index 24c9d5d2..43eb170e 100644 --- a/docs/functions.md +++ b/docs/functions.md @@ -9,6 +9,7 @@ * [Function Types](#function-types) * [Function parameters](#function-parameters) * [Logger](#logger) + * [Metrics](#metrics) * [State stores](#state-stores) ## Introduction @@ -29,7 +30,7 @@ functions: - name: secondParam type: int globalCode: | - import something from somepackage + import something from package globalVar = 3 code: | print('Hello there!') @@ -88,43 +89,286 @@ KSML is able to automatically convert between types. Examples are: ## Function Types -| Type | Returns | Parameter | Value Type | Description | -|:------------------------------------|:-------------------|:------------------|:-----------|:------------------------------------------| -| `aggregator` | _any_ | `key` | _any_ | The key of the message | -| | | `value` | _any_ | The value of the message | -| | | `aggregatedValue` | _any_ | The aggregated value thus far. | -| `forEach` | _none_ | `key` | _any_ | The key of the message | -| | | `value` | _any_ | The value of the message | -| `foreignKeyExtractor` | _any_ | `value` | _any_ | The value to extract the foreign key from | -| `initializer` | _any_ | _none_ | | | -| `keyTransformer` | _any_ | `key` | _any_ | The key of the message | -| | | `value` | _any_ | The value of the message | -| `keyValuePrinter` | `string` | `key` | _any_ | The key of the message | -| | | `value` | _any_ | The value of the message | -| `keyValueToKeyValueListTransformer` | [ (_any_, _any_) ] | `key` | _any_ | The key of the message | -| | | `value` | _any_ | The value of the message | -| `keyValueToValueListTransformer` | [ _any_ ] | `key` | _any_ | The key of the message | -| | | `value` | _any_ | The value of the message | -| `keyValueTransformer` | (_any_, _any_) | `key` | _any_ | The key of the message | -| | | `value` | _any_ | The value of the message | -| `merger` | _any_ | `key` | _any_ | The key of the message | -| | | `value1` | _any_ | The first value to be merged | -| | | `value2` | _any_ | The second value to be merged | -| `predicate` | `boolean` | `key` | _any_ | The key of the message | -| | | `value` | _any_ | The value of the message | -| `reducer` | _any_ | `value1` | _any_ | The first value to be reduced | -| | | `value2` | _any_ | The second value to be reduced | -| `streamPartitioner` | `int` | `topic` | `String` | The topic of the message | -| | | `key` | _any_ | The key of the message | -| | | `value` | _any_ | The value of the message | -| | | `numPartitions` | `int` | The number of partitions on the topic | -| `topicNameExtractor` | `string` | `key` | _any_ | The key of the message | -| | | `value` | _any_ | The value of the message | -| `valueJoiner` | _any_ | `key` | _any_ | The key of both messages | -| | | `value1` | _any_ | The first value to join | -| | | `value2` | _any_ | The second value to join | -| `valueTransformer` | _any_ | `key` | _any_ | The key of the message | -| | | `value` | _any_ | The value of the message | +Functions in KSML always have a `type`. When no type is specified, the function type is inferred from the context, or it +defaults back to `generic`. This section discusses the purpose of every function type, and what fixed arguments every +call gets passed in. + +### Aggregator + +An `aggregator` incrementally integrates a new keu/value into an aggregatedValue. It is called for every new message +that +becomes part of the aggregated result. + +The following highlights which calls are made to which function type during a regular aggregation, in this case for +counting the number of messages: + +``` +# Aggregation starts +initializer() -> 0 +msg1: aggregator(msg1.key, msg1.value, 0) -> 1 +msg2: aggregator(msg2.key, msg2.value, 1) -> 1 +msg3: aggregator(msg3.key, msg3.value, 2) -> 3 +``` + +The result in this example is 3. + +Aggregators get the following fixed arguments: + +| Parameter | Value Type | Description | +|:------------------|:-----------|:---------------------------------------------------------------------------| +| `key` | _any_ | The key of the message to be included in the aggregated result thus far. | +| `value` | _any_ | The value of the message to be included in the aggregated result thus far. | +| `aggregatedValue` | _any_ | The aggregated value thus far. | +| returns | _any_ | The new aggregated result, which includes the latest message. | + +### ForEach + +A `forEach` function is called for every message in a stream. When part of a `forEach` operation at the end of a +pipeline, the function is the last one called for every message. When this function is called during `peek` operations, +it may look at the messages and cause side effects (e.g. printing the message to stdout), and the pipeline will continue +with the unmodified message after doing so. + +ForEach functions get the following fixed arguments: + +| Parameter | Value Type | Description | +|:----------|:-----------|:--------------------------| +| `key` | _any_ | The key of the message. | +| `value` | _any_ | The value of the message. | +| returns | _none_ | Nothing is returned. | + +### ForeignKeyExtractor + +A `foreignKeyExtractor` is a function used during (left) joins of two tables. The function translates a value from "this +table" and translates it into a key of the "other table" that is joined with. + +ForEach functions get the following fixed arguments: + +| Parameter | Value Type | Description | +|:----------|:-----------|:--------------------------------------------| +| `value` | _any_ | The value of the message. | +| returns | _any_ | The key looked up in the table joined with. | + +### Generator + +A `generator` is a function that generates new messages out of thin air. It is most often used to generate mock data for +testing purposes. + +Generators get no arguments, and return messages to be sent to the output stream. + +| Parameter | Value Type | Description | +|:----------|:----------------|:--------------------------------------------------------------| +| returns | `(_any_, _any`) | The key/value of the message to be sent to the output stream. | + +### Generic + +A `generic` function can be used for generic purposes. It can be used for any operation, as long as its parameters match +the expected types of the operation's function. + +Generic functions get any arguments, and may return anything. + +| Parameter | Value Type | Description | +|:-------------|:-----------|:------------------------------------------| +| self-defined | _any_ | Self-defined parameters can be passed in. | +| returns | _any_ | Can return any value. | + +### Initializer + +An `initializer` is called upon the start of every (part of an) aggregation. It takes no arguments and should return an +initial value for the aggregation. + +| Parameter | Value Type | Description | +|:----------|:-----------|:------------------------------------------------------------------------------------| +| returns | _any_ | An initial value for the aggregation. In a counting aggregation, this would be `0`. | + +### KeyTransformer + +A `keyTransformer` is able to transform a key/value into a new key, which then gets combined with the original value as +a new message on the output stream. + +KeyTransformers get the following fixed arguments: + +| Parameter | Value Type | Description | +|:----------|:-----------|:-------------------------------| +| `key` | _any_ | The key of the message. | +| `value` | _any_ | The value of the message. | +| returns | _any_ | The key of the output message. | + +### KeyValuePrinter + +A `keyValuePrinter` takes a message and converts it to `string` before outputting it to a file or printing it to stdout. + +KeyValuePrinters get the following fixed arguments: + +| Parameter | Value Type | Description | +|:----------|:-----------|:--------------------------------------------| +| `key` | _any_ | The key of the message. | +| `value` | _any_ | The value of the message. | +| returns | `string` | The string to be written to file or stdout. | + +### KeyValueToKeyValueListTransformer + +A `keyValueToKeyValueListTransformer` takes one message and converts it into a list of output messages, which then get +sent to the output stream. An example for this type of function would be a message, which contains a list of items in +its `value` (e.g. `(k, [item])`. Using a `transformKeyValueToKeyValueList` operation, this message can be converted into +individual messages `(k,item1), (k,item2), ...` on the output stream. + +KeyValueToKeyValueListTransformers get the following fixed arguments: + +| Parameter | Value Type | Description | +|:----------|:-------------------|:------------------------------------------| +| `key` | _any_ | The key of the message. | +| `value` | _any_ | The value of the message. | +| returns | `[(_any_, _any_)]` | A list of messages for the output stream. | + +### KeyValueToValueListTransformer + +A `keyValueToValueListTransformer` takes one message and converts it into a list of output values, which then get +combined with the original key and sent to the output stream. An example for this type of function would be a message, +which contains a list of items in its `value` (e.g. `(k, [item])`. Using a `transformKeyValueToValueList` operation, +this message can be converted into a list of values `[item1, item2, ...]` which get combined with the key of the message +into `(k,item1), (k,item2), ...`on the output stream. + +KeyValueToValueListTransformers get the following fixed arguments: + +| Parameter | Value Type | Description | +|:----------|:-----------|:-------------------------------------------------------------------| +| `key` | _any_ | The key of the message. | +| `value` | _any_ | The value of the message. | +| returns | `[_any_]` | A list of values to be combined with the key on the output stream. | + +### KeyValueTransformer + +A `keyValueTransformer` takes one message and converts it into another message, which may have different key/value +types. + +KeyValueTransformers get the following fixed arguments: + +| Parameter | Value Type | Description | +|:----------|:-----------------|:--------------------------| +| `key` | _any_ | The key of the message. | +| `value` | _any_ | The value of the message. | +| returns | `(_any_, _any_)` | The transformed message. | + +### Merger + +A `merger` takes a key and two values, and merges those values together into a new value. That value is combined with +the original key and sent to the output stream. + +Mergers get the following fixed arguments: + +| Parameter | Value Type | Description | +|:----------|:-----------|:----------------------------------------| +| `key` | _any_ | The key of the message. | +| `value1` | _any_ | The value of the first message. | +| `value2` | _any_ | The value of the second message. | +| returns | _any_ | The merged value of the output message. | + +### MetadataTransformer + +A `metadataTransformer` can transform a message's metadata (headers and timestamp). + +MetadataTransformers get the following fixed arguments: + +| Parameter | Value Type | Description | +|:-----------|:-----------|:---------------------------------------------------------------------------------------------------------------------------------| +| `key` | _any_ | The key of the message. | +| `value` | _any_ | The value of the message. | +| `metadata` | `dict` | Contains the `headers` and `timestamp` of the message. | +| returns | `dict` | The (optionally) modified metadata for the output message. This structure should have the same type as the `metadata` passed in. | + +### Predicate + +A `predicate` is a function that takes the key/value of a message and returns `True` or `False`. It is used for +filtering and branching purposes (e.g. routing messages based on content). + +Predicates get the following fixed arguments: + +| Parameter | Value Type | Description | +|:----------|:-----------|:--------------------------| +| `key` | _any_ | The key of the message. | +| `value` | _any_ | The value of the message. | +| returns | `boolean` | `True` or `False`. | + +### Reducer + +A `reducer` is a function that combines two aggregated results into one. + +Reducers get the following fixed arguments: + +| Parameter | Value Type | Description | +|:----------|:-----------|:----------------------------------------------| +| `value1` | _any_ | The value of the first aggregation result. | +| `value2` | _any_ | The value of the second aggregation result. | +| returns | _any_ | The value of the combined aggregation result. | + +### StreamPartitioner + +A `streamPartitioner` is a function that can assign a partition number to every message. It is used to repartition Kafka +topics, based on message contents. + +StreamPartitioners get the following fixed arguments: + +| Parameter | Value Type | Description | +|:----------------|:-----------|:--------------------------------------------------------| +| `topic` | `string` | The topic of the message. | +| `key` | _any_ | The key of the message. | +| `value` | _any_ | The value of the message. | +| `numPartitions` | `integer` | The number of partitions available on the output topic. | +| returns | `integer` | The partition number to which this message gets sent. | + +### TimestampExtractor + +A `timestampExtractor` is a function which can determine a timestamp from a given input message, which is used for all +downstream processing. + +TimestampExtractors get the following fixed arguments: + +| Parameter | Value Type | Description | +|:--------------------|:-----------|:----------------------------------------------------------------------------------------------------------------------------------| +| `record` | `struct` | A dictionary containing the `timestamp`, `timestampType`, `key`, `value`, `topic`, `partition` and `offset` of the input message. | +| `previousTimestamp` | `long` | The timestamp of the last message (before this one). | +| returns | `long` | The timestamp to apply to this message. | + +### TopicNameExtractor + +A `topicNameExtractor` is a function which can derive a topic name from a message, for example by getting the customer +name from a message and deriving the topic name from that. It is used by `toTopicNameExtractor` operations to send +messages to individually determined topics. + +TopicNameExtractors get the following fixed arguments: + +| Parameter | Value Type | Description | +|:----------|:-----------|:-----------------------------------------------| +| `key` | _any_ | The key of the message. | +| `value` | _any_ | The value of the message. | +| returns | `string` | The name of the topic to send this message to. | + +### ValueJoiner + +A `valueJoiner` takes a key and two values, and combines the two values into one. That value is then combined with the +original key and sent to the output stream. + +ValueJoiners get the following fixed arguments: + +| Parameter | Value Type | Description | +|:----------|:-----------|:----------------------------------------| +| `key` | _any_ | The key of the message. | +| `value1` | _any_ | The value of the first message. | +| `value2` | _any_ | The value of the second message. | +| returns | _any_ | The joined value of the output message. | + +### ValueTransformer + +A `valueTransformer` takes a key/value and transforms it into a new value, which is combined with the original key and +sent to the output stream. + +ValueTransformers get the following fixed arguments: + +| Parameter | Value Type | Description | +|:----------|:-----------|:---------------------------------| +| `key` | _any_ | The key of the message. | +| `value` | _any_ | The value of the message. | +| returns | _any_ | The value of the output message. | ## Function parameters @@ -161,6 +405,47 @@ Output of the above statements looks like: [LOG TIMESTAMP] DEBUG function.name I'm printing five variables here: 1, 2, 3, text, {"json":"is cool"}. Lovely isn't it? ``` +### Metrics + +KSML supports metric collection and exposure through JMX and built-in Prometheus agent. Metrics for Python functions are +automatically generated and collected, but users can also specify their own metrics. For an example, +see `17-example-inspect-with-metrics.yaml` in the `examples` directory. + +KSML supports the following metric types: + +* Counter: an increasing integer, which counts for example the number of calls made to a Python function. +* Meter: used for periodically updating a measurement value. Preferred over Counter when don't care too much about exact + averages, but want to monitor trends instead. +* Timer: measures the time spent by processes or functions, that get called internally. + +Every Python function in KSML can use the `metrics` variable, which is made available by KSML. The object supports the +following methods to create your own metrics: + +* counter(name: str, tags: dict) -> Counter +* counter(name: str) -> Counter +* meter(name: str, tags: dict) -> Meter +* meter(name: str) -> Meter +* timer(name: str, tags: dict) -> Timer +* timer(name: str) -> Timer + +In turn these objects support the following: + +#### Counter + +* increment() +* increment(delta: int) + +#### Meter + +* mark() +* mark(nrOfEvents: int) + +#### Timer + +* updateSeconds(valueSeconds: int) +* updateMillis(valueMillis: int) +* updateNanos(valueNanos: int) + ### State stores Some functions are allowed to access local state stores. These functions specify the diff --git a/docs/index.md b/docs/index.md index 449b2f8d..e052cdf9 100644 --- a/docs/index.md +++ b/docs/index.md @@ -2,29 +2,25 @@ Welcome to the KSML documentation, Use the menu on the left to navigate through the various sections -## Introduction -KSML allows anyone to specify a powerful Kafka Streams application in just a few lines of YAML and Python snippets. +## Quick Start -## Prerequisites to run KSML -- Docker 19.03.0 or newer -- Kafka cluster 0.11 or newer -- Access to Docker Hub to pull images (internet connectivity) +If you want to get going quickly, go to the KSML Quickstart. -## Prerequisites to build KSML -- GraalVM JDK 21.0.2 with Java 21 -- Maven 3 - To build the project +## Introduction + +KSML allows anyone to specify a powerful Kafka Streams application in just a few lines of YAML and Python snippets. ## Contents 1. [Introduction](introduction.md) -1. [Stream Types](streams.md) -1. [Functions](functions.md) -1. [Pipelines](pipelines.md) -1. [Operations](operations.md) -1. [Data Types](types.md) -1. [Runners](runners.md) -1. [Language specification](ksml-language-spec) - -[Getting Started](getting-started.md) +2. [Stream Types](streams.md) +3. [Functions](functions.md) +4. [Pipelines](pipelines.md) +5. [Operations](operations.md) +6. [Data Types](types.md) +7. [Runners](runners.md) +8. [Language specification](ksml-language-spec.md) + +[Getting Started](quick-start) [Release Notes](release-notes.md) diff --git a/docs/introduction.md b/docs/introduction.md index e167c5f0..24d47c7d 100644 --- a/docs/introduction.md +++ b/docs/introduction.md @@ -1,28 +1,34 @@ # KSML: Kafka Streams for Low Code Environments ## Abstract -Kafka Streams has captured the hearts and minds of many developers that want to develop streaming applications on top of Kafka. But as powerful as the framework is, Kafka -Streams has had a hard time getting around the requirement of writing Java code and setting up build pipelines. There were some attempts to rebuild Kafka Streams, but up -until now popular languages like Python did not receive equally powerful (and maintained) stream processing frameworks. In this article we will present a new declarative -approach to unlock Kafka Streams, called KSML. By the time you finish reading this document, you will be able to write streaming applications yourself, using + +Kafka Streams has captured the hearts and minds of many developers that want to develop streaming applications on top of +Kafka. But as powerful as the framework is, Kafka +Streams has had a hard time getting around the requirement of writing Java code and setting up build pipelines. There +were some attempts to rebuild Kafka Streams, but up +until now popular languages like Python did not receive equally powerful (and maintained) stream processing frameworks. +In this article we will present a new declarative +approach to unlock Kafka Streams, called KSML. By the time you finish reading this document, you will be able to write +streaming applications yourself, using only a few simple basic rules and Python snippets. * [Setting up a test environment](#setting-up-a-test-environment) * [KSML in practice](#ksml-in-practice) - * [Example 1. Inspect data on a topic](#example-1-inspect-data-on-a-topic) - * [Example 2. Copying data to another topic](#example-2-copying-data-to-another-topic) - * [Example 3. Filtering data](#example-3-filtering-data) - * [Example 4. Branching messages](#example-4-branching-messages) - * [Example 5. Dynamic routing](#example-5-dynamic-routing) - * [Example 6. Multiple pipelines](#example-6-multiple-pipelines) - + * [Example 1. Inspect data on a topic](#example-1-inspect-data-on-a-topic) + * [Example 2. Copying data to another topic](#example-2-copying-data-to-another-topic) + * [Example 3. Filtering data](#example-3-filtering-data) + * [Example 4. Branching messages](#example-4-branching-messages) + * [Example 5. Dynamic routing](#example-5-dynamic-routing) + * [Example 6. Multiple pipelines](#example-6-multiple-pipelines) ## Setting up a test environment -To demonstrate KSML's capabilities, you will need a working Kafka cluster, or an Axual Platform/Cloud environment. Check out the [Runners](runners.md) page to configure KSML. +To demonstrate KSML's capabilities, you will need a working Kafka cluster, or an Axual Platform/Cloud environment. Check +out the [Runners](runners.md) page to configure KSML.
-We set up a test topic, called `ksml_sensordata_avro` with key/value types of `String`/`SensorData`. The [SensorData]({{ site.github.repository_url }}/tree/main/examples/SensorData.avsc) schema -was created for demo purposes only and contains several fields to demonstratie KSML capabilities: +We set up a test topic, called `ksml_sensordata_avro` with key/value types of `String`/`SensorData`. The [SensorData] +schema +was created for demo purposes only and contains several fields to demonstrate KSML capabilities: ```json { @@ -98,7 +104,8 @@ was created for demo purposes only and contains several fields to demonstratie K } ``` -For the rest of this document, we assume you have set up the `ksml_sensordata_avro` topic and populated it with some random data. +For the rest of this document, we assume you have set up the `ksml_sensordata_avro` topic and populated it with some +random data. So without any further delays, let's see how KSML allows us to process this data. @@ -133,13 +140,18 @@ pipelines: ``` -Let's disect this definition one element at a time. Before defining processing logic, we first define the streams used by the definition. In this case we define a stream named `sensor_source_avro` which reads from the topic `ksml_sensordata_avro`. The stream defines a `string` key and Avro `SensorData` values. +Let's analyze this definition one element at a time. Before defining processing logic, we first define the streams used +by the definition. In this case we define a stream named `sensor_source_avro` which reads from the +topic `ksml_sensordata_avro`. The stream defines a `string` key and Avro `SensorData` values. -Next is a list of functions that can be used by the processing logic. Here we define just one, `log_message`, which simply uses the provided logger to write the key, value and format of a message to the console. +Next is a list of functions that can be used by the processing logic. Here we define just one, `log_message`, which +simply uses the provided logger to write the key, value and format of a message to the console. -The third element `pipelines` defines the real processing logic. We define a pipeline called `consume_avro`, which takes messages from `ksml_sensordata_avro` and passes them to `print_message`. +The third element `pipelines` defines the real processing logic. We define a pipeline called `consume_avro`, which takes +messages from `ksml_sensordata_avro` and passes them to `print_message`. -The definition file is parsed by KSML and translated into a Kafka Streams topology, which is [described](https://kafka.apache.org/37/javadoc/org/apache/kafka/streams/Topology.html#describe--) as follows: +The definition file is parsed by KSML and translated into a Kafka Streams topology, which +is [described](https://kafka.apache.org/37/javadoc/org/apache/kafka/streams/Topology.html#describe--) as follows: ``` Topologies: @@ -162,11 +174,13 @@ And the output of the generated topology looks like this: 2024-03-06T18:31:59,412Z INFO ksml.functions.log_message Consumed AVRO message - key=sensor5, value={'city': 'Amsterdam', 'color': 'blue', 'name': 'sensor5', 'owner': 'Bob', 'timestamp': 1709749919409, 'type': 'LENGTH', 'unit': 'm', 'value': '658', '@type': 'SensorData', '@schema': { <>}} ``` -As you can see, the output of the application is exactly that what we defined it to be in the `log_message` function, namely a dump of all data found on the topic. +As you can see, the output of the application is exactly that what we defined it to be in the `log_message` function, +namely a dump of all data found on the topic. ### Example 2. Copying data to another topic -Now that we can see what data is on a topic, we will start to manipulate its routing. In this example we are copying unmodified data to a secondary topic: +Now that we can see what data is on a topic, we will start to manipulate its routing. In this example we are copying +unmodified data to a secondary topic: ```yaml streams: @@ -197,11 +211,17 @@ pipelines: to: sensor_copy ``` -You can see that we specified a second stream named `sensor_copy` in this example, which is backed by the topic `ksml_sensordata_copy` target topic. The `log_message` function is unchanged, but the pipeline did undergo some changes. Two new elements are introduced here, namely `via` and `to`. +You can see that we specified a second stream named `sensor_copy` in this example, which is backed by the +topic `ksml_sensordata_copy` target topic. The `log_message` function is unchanged, but the pipeline did undergo some +changes. Two new elements are introduced here, namely `via` and `to`. -The `via` tag allows users to define a series of operations executed on the data. In this case there is only one, namely a `peek` operation which does not modify any data, but simply outputs the data on stdout as a side-effect. +The `via` tag allows users to define a series of operations executed on the data. In this case there is only one, namely +a `peek` operation which does not modify any data, but simply outputs the data on stdout as a side effect. -The `to` operation is a so-called "sink operation". Sink operations are always last in a pipeline. Processing of the pipeline does not continue after it was delivered to a sink operation. Note that in the first example above `forEach` is also a sink operation, whereas in this example we achieve the same result by passing the `log_message` function as a parameter to the `peek` operation. +The `to` operation is a so-called "sink operation". Sink operations are always last in a pipeline. Processing of the +pipeline does not continue after it was delivered to a sink operation. Note that in the first example above `forEach` is +also a sink operation, whereas in this example we achieve the same result by passing the `log_message` function as a +parameter to the `peek` operation. When this definition is translated by KSML, the following Kafka Streams topology is created: @@ -222,7 +242,8 @@ The output is similar to that of example 1, but the same data can also be found ### Example 3. Filtering data -Now that we can read and write data, let's see if we can apply some logic to the processing as well. In this example we will be filtering data based on the contents of the value: +Now that we can read and write data, let's see if we can apply some logic to the processing as well. In this example we +will be filtering data based on the contents of the value: ```yaml # This example shows how to read from four simple streams and log all messages @@ -267,9 +288,13 @@ pipelines: to: sensor_filtered ``` -Again, first we define the streams and the functions involved in the processing. You can see we added a new function called `filter_message` which returns `true` or `false` based on the `color` field in the value of the message. This function is used below in the pipeline. +Again, first we define the streams and the functions involved in the processing. You can see we added a new function +called `filter_message` which returns `true` or `false` based on the `color` field in the value of the message. This +function is used below in the pipeline. -The pipeline is extended to include a `filter` operation, which takes a `predicate` function as parameter. That function is called for every input message. Only messages for which the function returns `true` are propagated. All other messages are discarded. +The pipeline is extended to include a `filter` operation, which takes a `predicate` function as parameter. That function +is called for every input message. Only messages for which the function returns `true` are propagated. All other +messages are discarded. Using this definition, KSML generates the following Kafka Streams topology: @@ -300,11 +325,14 @@ When it executes, we see the following output: 2024-03-06T18:45:12,008Z INFO ksml.functions.log_message Consumed AVRO message - key=sensor5, value={'city': 'Amsterdam', 'color': 'blue', 'name': 'sensor5', 'owner': 'Bob', 'timestamp': 1709749919409, 'type': 'LENGTH', 'unit': 'm', 'value': '658', '@type': 'SensorData', '@schema': { <>}} ``` -As you can see, the filter operation did its work. Only messages with field `color` set to `blue` are passed on to the `peek` operation, while other messages are discarded. +As you can see, the filter operation did its work. Only messages with field `color` set to `blue` are passed on to +the `peek` operation, while other messages are discarded. ### Example 4. Branching messages -Another way to filter messages is to use a `branch` operation. This is also a sink operation, which closes the processing of a pipeline. It is similar to `forEach` and `to` in that respect, but has a different definition and behaviour. +Another way to filter messages is to use a `branch` operation. This is also a sink operation, which closes the +processing of a pipeline. It is similar to `forEach` and `to` in that respect, but has a different definition and +behaviour. ```yaml streams: @@ -339,9 +367,15 @@ pipelines: code: log.warn("UNKNOWN COLOR - {}", value["color"]) ``` -The `branch` operation takes a list of branches as its parameters, which each specifies a processing pipeline of its own. Branches contain the keyword `if`, which take a predicate function that determines if a message will flow into that particular branch, or if it will be passed to the next branch(es). Every message will only end up in one branch, namely the first one in order where the `if` predcate function returns `true`. +The `branch` operation takes a list of branches as its parameters, which each specifies a processing pipeline of its +own. Branches contain the keyword `if`, which take a predicate function that determines if a message will flow into that +particular branch, or if it will be passed to the next branch(es). Every message will only end up in one branch, namely +the first one in order where the `if` predicate function returns `true`. -In the example we see that the first branch will be populated only with messages with `color` field set to `blue`. Once there, these messages will be written to `ksml_sensordata_blue`. The second branch will only contain messages with `color`=`red` and these messages will be written to `ksml_sensordata_red`. Finally, the last branch outputs a message that the color is unknown and ends any further processing. +In the example we see that the first branch will be populated only with messages with `color` field set to `blue`. Once +there, these messages will be written to `ksml_sensordata_blue`. The second branch will only contain messages +with `color`=`red` and these messages will be written to `ksml_sensordata_red`. Finally, the last branch outputs a +message that the color is unknown and ends any further processing. When translated by KSML the following Kafka Streams topology is set up: @@ -388,11 +422,15 @@ It is clear that the branch operation is integrated in this topology. Its output 2024-03-06T18:31:59,412Z INFO k.f.branch_pipelines_main_via_1_forEach SOURCE MESSAGE - key=sensor5, value={'city': 'Amsterdam', 'color': 'blue', 'name': 'sensor5', 'owner': 'Bob', 'timestamp': 1709749919409, 'type': 'LENGTH', 'unit': 'm', 'value': '658', '@type': 'SensorData', '@schema': { <>}} ``` -We see that every message processed by the pipeline is logged through the `k.f.branch_pipelines_main_via_1_forEach` logger. But the branch operation sorts the messages and sends messages with colors `blue` and `red` into their own branches. The only colors that show up as `UNKNOWN COLOR -` messages are non-blue and non-red and send through the `branch_pipelines_main_branch_3_forEach` logger. +We see that every message processed by the pipeline is logged through the `k.f.branch_pipelines_main_via_1_forEach` +logger. But the branch operation sorts the messages and sends messages with colors `blue` and `red` into their own +branches. The only colors that show up as `UNKNOWN COLOR -` messages are non-blue and non-red and send through +the `branch_pipelines_main_branch_3_forEach` logger. ### Example 5. Dynamic routing -Sometimes it is necessary to route a message to one stream or another based on the content of a message. This example shows how to route messages dynamically using a TopicNameExtractor. +Sometimes it is necessary to route a message to one stream or another based on the content of a message. This example +shows how to route messages dynamically using a TopicNameExtractor. ```yaml streams: @@ -419,7 +457,10 @@ pipelines: return 'ksml_sensordata_sensor0' ``` -The `topicNameExtractor` operation takes a function, which determines the routing of every message by returning a topic name string. In this case, when the key of a message is `sensor1` then the message will be sent to `ksml_sensordata_sensor1`. When it contains `sensor2` the message is sent to `ksml_sensordata_sensor2`. All other messages are sent to `ksml_sensordata_sensor0`. +The `topicNameExtractor` operation takes a function, which determines the routing of every message by returning a topic +name string. In this case, when the key of a message is `sensor1` then the message will be sent +to `ksml_sensordata_sensor1`. When it contains `sensor2` the message is sent to `ksml_sensordata_sensor2`. All other +messages are sent to `ksml_sensordata_sensor0`. The equivalent Kafka Streams topology looks like this: @@ -435,13 +476,16 @@ Topologies: <-- route_route_pipelines_main_via_1 ``` -The output does not show anything special compared to previous examples, since all messages are simply written by the logger. +The output does not show anything special compared to previous examples, since all messages are simply written by the +logger. ### Example 6. Multiple pipelines -In the previous examples there was always a single pipeline definition for processing data. KSML allows us to define multiple pipelines in a single file. +In the previous examples there was always a single pipeline definition for processing data. KSML allows us to define +multiple pipelines in a single file. -In this example we combine the filtering example with the routing example. We will also define new pipelines with the sole purpose of logging the routed messages. +In this example we combine the filtering example with the routing example. We will also define new pipelines with the +sole purpose of logging the routed messages. ```yaml # This example shows how to route messages to a dynamic topic. The target topic is the result of an executed function. @@ -517,13 +561,16 @@ pipelines: ``` In this definition we defined five pipelines: -1. `filtering` which filters out all sensor messages that don't have the color blue and sends it to the `sensor_filtered` stream. + +1. `filtering` which filters out all sensor messages that don't have the color blue and sends it to + the `sensor_filtered` stream. 2. `routing` which routes the data on the `sensor_filtered` stream to one of three target topics 3. `sensor0_peek` which writes the content of the `sensor_0` stream to the console 4. `sensor1_peek` which writes the content of the `sensor_1` stream to the console 5. `sensor2_peek` which writes the content of the `sensor_2` stream to the console The equivalent Kafka Streams topology looks like this: + ``` Topologies: Sub-topology: 0 @@ -566,7 +613,9 @@ Topologies: <-- ksml_sensordata_sensor2 ``` -And this is what the output would look something like this. The sensor peeks messages will not always be shown immediately after the Routing messages. This is because the pipelines are running in separate sub processes. +And this is what the output would look something like this. The sensor peeks messages will not always be shown +immediately after the Routing messages. This is because the pipelines are running in separate sub processes. + ``` 2024-03-06T20:11:39,520Z INFO k.f.route2_pipelines_routing_via_1_forEach Routing Blue sensor - key=sensor6, value={'city': 'Utrecht', 'color': 'blue', 'name': 'sensor6', 'owner': 'Charlie', 'timestamp': 1709755877401, 'type': 'LENGTH', 'unit': 'ft', 'value': '507', '@type': 'SensorData', '@schema': { <>}} 2024-03-06T20:11:39,523Z INFO k.f.route2_pipelines_sensor0_peek_forEach SENSOR0 - key=sensor6, value={'city': 'Utrecht', 'color': 'blue', 'name': 'sensor6', 'owner': 'Charlie', 'timestamp': 1709755877401, 'type': 'LENGTH', 'unit': 'ft', 'value': '507', '@type': 'SensorData', '@schema': { <>}} diff --git a/docs/ksml-language-spec.json b/docs/ksml-language-spec.json index 1cce31e7..08f1523f 100644 --- a/docs/ksml-language-spec.json +++ b/docs/ksml-language-spec.json @@ -1 +1 @@ -{"additionalProperties":false,"definitions":{"AggregateOperation":{"additionalProperties":false,"description":"An aggregate operation","properties":{"adder":{"anyOf":[{"type":"string"},{"$ref":"#/definitions/AggregatorDefinitionWithImplicitType"}],"description":"*(optional)* (GroupedTable) A function that adds a record to the aggregation result"},"aggregator":{"anyOf":[{"type":"string"},{"$ref":"#/definitions/AggregatorDefinitionWithImplicitType"}],"description":"*(optional)* (GroupedStream, SessionWindowedStream, TimeWindowedStream) The aggregator function, which combines a value with the previous aggregation result and outputs a new aggregation result"},"initializer":{"anyOf":[{"type":"string"},{"$ref":"#/definitions/InitializerDefinitionWithImplicitType"}],"description":"The initializer function, which generates an initial value for every set of aggregated records"},"merger":{"anyOf":[{"type":"string"},{"$ref":"#/definitions/MergerDefinitionWithImplicitType"}],"description":"*(optional)* (SessionWindowedStream, SessionWindowedCogroupedStream) A function that combines two aggregation results"},"name":{"description":"*(optional)* The name of the operation processor","type":"string"},"store":{"anyOf":[{"type":"string"},{"$ref":"#/definitions/WindowStateStoreDefinitionWithImplicitType"}],"description":"*(optional)* Materialized view of the result aggregation"},"subtractor":{"anyOf":[{"type":"string"},{"$ref":"#/definitions/AggregatorDefinitionWithImplicitType"}],"description":"*(optional)* (GroupedTable) A function that removes a record from the aggregation result"},"type":{"description":"The type of the operation","enum":["aggregate"]}},"required":["initializer","type"],"title":"AggregateOperation","type":"object"},"AggregatorDefinition":{"additionalProperties":false,"description":"Defines a aggregator function, that gets injected into the Kafka Streams topology","properties":{"code":{"anyOf":[{"type":"boolean"},{"type":"number"},{"type":"string"}],"description":"*(optional)* The (multiline) code of the aggregator"},"expression":{"anyOf":[{"type":"boolean"},{"type":"number"},{"type":"string"}],"description":"*(optional)* The expression returned by the aggregator. Only required for functions that return values."},"globalCode":{"anyOf":[{"type":"boolean"},{"type":"number"},{"type":"string"}],"description":"*(optional)* Global (multiline) code that gets loaded into the Python context outside of the aggregator. Can be used for defining eg. global variables."},"name":{"description":"*(optional)* The name of the aggregator. If this field is not defined, then the name is derived from the context.","type":"string"},"parameters":{"description":"*(optional)* A list of parameters to be passed into the aggregator","items":{"$ref":"#/definitions/ParameterDefinition"},"type":"array"},"resultType":{"description":"*(optional)* The data type returned by the aggregator. Only required for function types, which are not pre-defined.","type":"string"},"stores":{"description":"*(optional)* A list of store names that the aggregator uses. Only required if the function wants to use a state store.","items":{"type":"string"},"type":"array"},"type":{"description":"The type of the function","enum":["aggregator"]}},"required":["type"],"title":"AggregatorDefinition","type":"object"},"AggregatorDefinitionWithImplicitType":{"additionalProperties":false,"description":"Defines a aggregator function, that gets injected into the Kafka Streams topology","properties":{"code":{"anyOf":[{"type":"boolean"},{"type":"number"},{"type":"string"}],"description":"*(optional)* The (multiline) code of the aggregator"},"expression":{"anyOf":[{"type":"boolean"},{"type":"number"},{"type":"string"}],"description":"*(optional)* The expression returned by the aggregator. Only required for functions that return values."},"globalCode":{"anyOf":[{"type":"boolean"},{"type":"number"},{"type":"string"}],"description":"*(optional)* Global (multiline) code that gets loaded into the Python context outside of the aggregator. Can be used for defining eg. global variables."},"name":{"description":"*(optional)* The name of the aggregator. If this field is not defined, then the name is derived from the context.","type":"string"},"parameters":{"description":"*(optional)* A list of parameters to be passed into the aggregator","items":{"$ref":"#/definitions/ParameterDefinition"},"type":"array"},"resultType":{"description":"*(optional)* The data type returned by the aggregator. Only required for function types, which are not pre-defined.","type":"string"},"stores":{"description":"*(optional)* A list of store names that the aggregator uses. Only required if the function wants to use a state store.","items":{"type":"string"},"type":"array"}},"title":"AggregatorDefinitionWithImplicitType","type":"object"},"BranchDefinitionWithPipeline":{"additionalProperties":false,"description":"Defines a branch with sub-pipeline in a BranchOperation","properties":{"as":{"description":"*(optional)* The name to register the pipeline result under, which can be used as source by follow-up pipelines","type":"string"},"branch":{"description":"*(optional)* Defines a single branch, consisting of a condition and a pipeline to execute for messages that fulfil the predicate","items":{"$ref":"#/definitions/StringOrInlinePredicateDefinitionWithImplicitType"},"type":"array"},"forEach":{"anyOf":[{"type":"string"},{"$ref":"#/definitions/ForEachActionDefinitionWithImplicitType"}],"description":"*(optional)* A function that gets called for every message in the stream"},"if":{"anyOf":[{"type":"string"},{"$ref":"#/definitions/PredicateDefinitionWithImplicitType"}],"description":"*(optional)* Defines the condition under which messages get sent down this branch"},"name":{"description":"*(optional)* The name of the operation processor","type":"string"},"print":{"$ref":"#/definitions/PrintOperation","description":"*(optional)* The specification of where to print messages to"},"stores":{"description":"*(optional)* The names of all state stores used by the function","items":{"type":"string"},"type":"array"},"to":{"anyOf":[{"type":"string"},{"$ref":"#/definitions/ToTopicDefinition"}],"description":"*(optional)* Ends the pipeline by sending all messages to a stream, table or globalTable, or to an inline defined output topic and optional partitioner"},"toTopicNameExtractor":{"anyOf":[{"type":"string"},{"$ref":"#/definitions/ToTopicNameExtractorDefinition"}],"description":"*(optional)* Ends the pipeline by sending all messages to a topic provided by a pre-defined topic name extractor function, or to a topic provided by an inline defined topic name extractor and optional partitioner"},"via":{"description":"*(optional)* A series of operations performed on the input stream","items":{"anyOf":[{"$ref":"#/definitions/AggregateOperation"},{"$ref":"#/definitions/CogroupOperation"},{"$ref":"#/definitions/ConvertKeyOperation"},{"$ref":"#/definitions/ConvertKeyValueOperation"},{"$ref":"#/definitions/ConvertValueOperation"},{"$ref":"#/definitions/CountOperation"},{"$ref":"#/definitions/FilterNotOperation"},{"$ref":"#/definitions/FilterOperation"},{"$ref":"#/definitions/GroupByKeyOperation"},{"$ref":"#/definitions/GroupByOperation"},{"$ref":"#/definitions/JoinOperationWithGlobalTable"},{"$ref":"#/definitions/JoinOperationWithStream"},{"$ref":"#/definitions/JoinOperationWithTable"},{"$ref":"#/definitions/LeftJoinOperationWithGlobalTable"},{"$ref":"#/definitions/LeftJoinOperationWithStream"},{"$ref":"#/definitions/LeftJoinOperationWithTable"},{"$ref":"#/definitions/MergeOperation"},{"$ref":"#/definitions/OuterJoinOperationWithStream"},{"$ref":"#/definitions/OuterJoinOperationWithTable"},{"$ref":"#/definitions/PeekOperation"},{"$ref":"#/definitions/ReduceOperation"},{"$ref":"#/definitions/RepartitionOperation"},{"$ref":"#/definitions/SuppressOperation"},{"$ref":"#/definitions/ToStreamOperation"},{"$ref":"#/definitions/ToTableOperation"},{"$ref":"#/definitions/TransformKeyOperation"},{"$ref":"#/definitions/TransformKeyValueOperation"},{"$ref":"#/definitions/TransformKeyValueToKeyValueListOperation"},{"$ref":"#/definitions/TransformKeyValueToValueListOperation"},{"$ref":"#/definitions/TransformMetadataOperation"},{"$ref":"#/definitions/TransformValueOperation"},{"$ref":"#/definitions/WindowBySessionOperation"},{"$ref":"#/definitions/WindowByTimeOperation"}]},"type":"array"}},"title":"BranchDefinitionWithPipeline","type":"object"},"CogroupOperation":{"additionalProperties":false,"description":"A cogroup operation","properties":{"aggregator":{"anyOf":[{"type":"string"},{"$ref":"#/definitions/AggregatorDefinitionWithImplicitType"}],"description":"(GroupedStream, SessionWindowedStream, TimeWindowedStream) The aggregator function, which combines a value with the previous aggregation result and outputs a new aggregation result"},"name":{"description":"*(optional)* The name of the operation processor","type":"string"},"store":{"anyOf":[{"type":"string"},{"$ref":"#/definitions/WindowStateStoreDefinitionWithImplicitType"}],"description":"*(optional)* Materialized view of the co-grouped stream"},"type":{"description":"The type of the operation","enum":["cogroup"]}},"required":["aggregator","type"],"title":"CogroupOperation","type":"object"},"ConvertKeyOperation":{"additionalProperties":false,"description":"An operation to convert the stream key type to another type. Conversion is only syntactic, eg. from Avro to XML.","properties":{"into":{"description":"The type to convert the stream key into","type":"string"},"name":{"description":"*(optional)* The name of the operation processor","type":"string"},"type":{"description":"The type of the operation","enum":["convertKey"]}},"required":["into","type"],"title":"ConvertKeyOperation","type":"object"},"ConvertKeyValueOperation":{"additionalProperties":false,"description":"An operation to convert the stream key and value types to other types. Conversion is only syntactic, eg. from Avro to XML.","properties":{"into":{"description":"The tuple type to convert the stream key/value into","type":"string"},"name":{"description":"*(optional)* The name of the operation processor","type":"string"},"type":{"description":"The type of the operation","enum":["convertKeyValue"]}},"required":["into","type"],"title":"ConvertKeyValueOperation","type":"object"},"ConvertValueOperation":{"additionalProperties":false,"description":"An operation to convert the stream value type to another type. Conversion is only syntactic, eg. from Avro to XML.","properties":{"into":{"description":"The type to convert the stream value into","type":"string"},"name":{"description":"*(optional)* The name of the operation processor","type":"string"},"type":{"description":"The type of the operation","enum":["convertValue"]}},"required":["into","type"],"title":"ConvertValueOperation","type":"object"},"CountOperation":{"additionalProperties":false,"description":"Count the number of times a key is seen in a given window","properties":{"name":{"description":"*(optional)* The name of the operation processor","type":"string"},"store":{"anyOf":[{"type":"string"},{"$ref":"#/definitions/KeyValueStateStoreDefinition"},{"$ref":"#/definitions/SessionStateStoreDefinition"},{"$ref":"#/definitions/WindowStateStoreDefinition"}],"description":"*(optional)* Materialized view of the count operation's result"},"type":{"description":"The type of the operation","enum":["count"]}},"required":["type"],"title":"CountOperation","type":"object"},"FilterNotOperation":{"additionalProperties":false,"description":"Filter records based on the inverse result of a predicate function","properties":{"if":{"anyOf":[{"type":"string"},{"$ref":"#/definitions/PredicateDefinitionWithImplicitType"}],"description":"A function that returns \"false\" when records are accepted, \"true\" otherwise"},"name":{"description":"*(optional)* The name of the operation processor","type":"string"},"store":{"anyOf":[{"type":"string"},{"$ref":"#/definitions/KeyValueStateStoreDefinitionWithImplicitType"}],"description":"*(optional)* Materialized view of the filtered table (only applies to tables, ignored for streams)"},"stores":{"description":"*(optional)* The names of all state stores used by the function","items":{"type":"string"},"type":"array"},"type":{"description":"The type of the operation","enum":["filterNot"]}},"required":["if","type"],"title":"FilterNotOperation","type":"object"},"FilterOperation":{"additionalProperties":false,"description":"Filter records based on a predicate function","properties":{"if":{"anyOf":[{"type":"string"},{"$ref":"#/definitions/PredicateDefinitionWithImplicitType"}],"description":"A function that returns \"true\" when records are accepted, \"false\" otherwise"},"name":{"description":"*(optional)* The name of the operation processor","type":"string"},"store":{"anyOf":[{"type":"string"},{"$ref":"#/definitions/KeyValueStateStoreDefinitionWithImplicitType"}],"description":"*(optional)* Materialized view of the filtered table (only applies to tables, ignored for streams)"},"stores":{"description":"*(optional)* The names of all state stores used by the function","items":{"type":"string"},"type":"array"},"type":{"description":"The type of the operation","enum":["filter"]}},"required":["if","type"],"title":"FilterOperation","type":"object"},"ForEachActionDefinition":{"additionalProperties":false,"description":"Defines a foreach action function, that gets injected into the Kafka Streams topology","properties":{"code":{"anyOf":[{"type":"boolean"},{"type":"number"},{"type":"string"}],"description":"*(optional)* The (multiline) code of the foreach action"},"expression":{"anyOf":[{"type":"boolean"},{"type":"number"},{"type":"string"}],"description":"*(optional)* The expression returned by the foreach action. Only required for functions that return values."},"globalCode":{"anyOf":[{"type":"boolean"},{"type":"number"},{"type":"string"}],"description":"*(optional)* Global (multiline) code that gets loaded into the Python context outside of the foreach action. Can be used for defining eg. global variables."},"name":{"description":"*(optional)* The name of the foreach action. If this field is not defined, then the name is derived from the context.","type":"string"},"parameters":{"description":"*(optional)* A list of parameters to be passed into the foreach action","items":{"$ref":"#/definitions/ParameterDefinition"},"type":"array"},"resultType":{"description":"*(optional)* The data type returned by the foreach action. Only required for function types, which are not pre-defined.","type":"string"},"stores":{"description":"*(optional)* A list of store names that the foreach action uses. Only required if the function wants to use a state store.","items":{"type":"string"},"type":"array"},"type":{"description":"The type of the function","enum":["forEach"]}},"required":["type"],"title":"ForEachActionDefinition","type":"object"},"ForEachActionDefinitionWithImplicitType":{"additionalProperties":false,"description":"Defines a foreach action function, that gets injected into the Kafka Streams topology","properties":{"code":{"anyOf":[{"type":"boolean"},{"type":"number"},{"type":"string"}],"description":"*(optional)* The (multiline) code of the foreach action"},"expression":{"anyOf":[{"type":"boolean"},{"type":"number"},{"type":"string"}],"description":"*(optional)* The expression returned by the foreach action. Only required for functions that return values."},"globalCode":{"anyOf":[{"type":"boolean"},{"type":"number"},{"type":"string"}],"description":"*(optional)* Global (multiline) code that gets loaded into the Python context outside of the foreach action. Can be used for defining eg. global variables."},"name":{"description":"*(optional)* The name of the foreach action. If this field is not defined, then the name is derived from the context.","type":"string"},"parameters":{"description":"*(optional)* A list of parameters to be passed into the foreach action","items":{"$ref":"#/definitions/ParameterDefinition"},"type":"array"},"resultType":{"description":"*(optional)* The data type returned by the foreach action. Only required for function types, which are not pre-defined.","type":"string"},"stores":{"description":"*(optional)* A list of store names that the foreach action uses. Only required if the function wants to use a state store.","items":{"type":"string"},"type":"array"}},"title":"ForEachActionDefinitionWithImplicitType","type":"object"},"ForeignKeyExtractorDefinition":{"additionalProperties":false,"description":"Defines a foreign key extractor function, that gets injected into the Kafka Streams topology","properties":{"code":{"anyOf":[{"type":"boolean"},{"type":"number"},{"type":"string"}],"description":"*(optional)* The (multiline) code of the foreign key extractor"},"expression":{"anyOf":[{"type":"boolean"},{"type":"number"},{"type":"string"}],"description":"*(optional)* The expression returned by the foreign key extractor. Only required for functions that return values."},"globalCode":{"anyOf":[{"type":"boolean"},{"type":"number"},{"type":"string"}],"description":"*(optional)* Global (multiline) code that gets loaded into the Python context outside of the foreign key extractor. Can be used for defining eg. global variables."},"name":{"description":"*(optional)* The name of the foreign key extractor. If this field is not defined, then the name is derived from the context.","type":"string"},"parameters":{"description":"*(optional)* A list of parameters to be passed into the foreign key extractor","items":{"$ref":"#/definitions/ParameterDefinition"},"type":"array"},"resultType":{"description":"*(optional)* The data type returned by the foreign key extractor. Only required for function types, which are not pre-defined.","type":"string"},"stores":{"description":"*(optional)* A list of store names that the foreign key extractor uses. Only required if the function wants to use a state store.","items":{"type":"string"},"type":"array"},"type":{"description":"The type of the function","enum":["foreignKeyExtractor"]}},"required":["type"],"title":"ForeignKeyExtractorDefinition","type":"object"},"GeneratorDefinition":{"additionalProperties":false,"description":"Defines a message generator function, that gets injected into the Kafka Streams topology","properties":{"code":{"anyOf":[{"type":"boolean"},{"type":"number"},{"type":"string"}],"description":"*(optional)* The (multiline) code of the message generator"},"expression":{"anyOf":[{"type":"boolean"},{"type":"number"},{"type":"string"}],"description":"*(optional)* The expression returned by the message generator. Only required for functions that return values."},"globalCode":{"anyOf":[{"type":"boolean"},{"type":"number"},{"type":"string"}],"description":"*(optional)* Global (multiline) code that gets loaded into the Python context outside of the message generator. Can be used for defining eg. global variables."},"name":{"description":"*(optional)* The name of the message generator. If this field is not defined, then the name is derived from the context.","type":"string"},"parameters":{"description":"*(optional)* A list of parameters to be passed into the message generator","items":{"$ref":"#/definitions/ParameterDefinition"},"type":"array"},"resultType":{"description":"*(optional)* The data type returned by the message generator. Only required for function types, which are not pre-defined.","type":"string"},"stores":{"description":"*(optional)* A list of store names that the message generator uses. Only required if the function wants to use a state store.","items":{"type":"string"},"type":"array"},"type":{"description":"The type of the function","enum":["generator"]}},"required":["type"],"title":"GeneratorDefinition","type":"object"},"GeneratorDefinitionWithImplicitType":{"additionalProperties":false,"description":"Defines a message generator function, that gets injected into the Kafka Streams topology","properties":{"code":{"anyOf":[{"type":"boolean"},{"type":"number"},{"type":"string"}],"description":"*(optional)* The (multiline) code of the message generator"},"expression":{"anyOf":[{"type":"boolean"},{"type":"number"},{"type":"string"}],"description":"*(optional)* The expression returned by the message generator. Only required for functions that return values."},"globalCode":{"anyOf":[{"type":"boolean"},{"type":"number"},{"type":"string"}],"description":"*(optional)* Global (multiline) code that gets loaded into the Python context outside of the message generator. Can be used for defining eg. global variables."},"name":{"description":"*(optional)* The name of the message generator. If this field is not defined, then the name is derived from the context.","type":"string"},"parameters":{"description":"*(optional)* A list of parameters to be passed into the message generator","items":{"$ref":"#/definitions/ParameterDefinition"},"type":"array"},"resultType":{"description":"*(optional)* The data type returned by the message generator. Only required for function types, which are not pre-defined.","type":"string"},"stores":{"description":"*(optional)* A list of store names that the message generator uses. Only required if the function wants to use a state store.","items":{"type":"string"},"type":"array"}},"title":"GeneratorDefinitionWithImplicitType","type":"object"},"GenericFunctionDefinitionWithImplicitType":{"additionalProperties":false,"description":"Defines a generic function function, that gets injected into the Kafka Streams topology","properties":{"code":{"anyOf":[{"type":"boolean"},{"type":"number"},{"type":"string"}],"description":"*(optional)* The (multiline) code of the generic function"},"expression":{"anyOf":[{"type":"boolean"},{"type":"number"},{"type":"string"}],"description":"*(optional)* The expression returned by the generic function. Only required for functions that return values."},"globalCode":{"anyOf":[{"type":"boolean"},{"type":"number"},{"type":"string"}],"description":"*(optional)* Global (multiline) code that gets loaded into the Python context outside of the generic function. Can be used for defining eg. global variables."},"name":{"description":"*(optional)* The name of the generic function. If this field is not defined, then the name is derived from the context.","type":"string"},"parameters":{"description":"*(optional)* A list of parameters to be passed into the generic function","items":{"$ref":"#/definitions/ParameterDefinition"},"type":"array"},"resultType":{"description":"*(optional)* The data type returned by the generic function. Only required for function types, which are not pre-defined.","type":"string"},"stores":{"description":"*(optional)* A list of store names that the generic function uses. Only required if the function wants to use a state store.","items":{"type":"string"},"type":"array"},"type":{"description":"The type of the function","enum":["generic"]}},"title":"GenericFunctionDefinitionWithImplicitType","type":"object"},"GlobalTableDefinition":{"additionalProperties":false,"description":"Contains a definition of a GlobalTable, which can be referenced by producers and pipelines","properties":{"keyType":{"description":"*(optional)* The key type of the global table","type":"string"},"store":{"anyOf":[{"type":"string"},{"$ref":"#/definitions/KeyValueStateStoreDefinitionWithImplicitType"}],"description":"*(optional)* KeyValue state store definition"},"topic":{"description":"The name of the Kafka topic for this global table","type":"string"},"valueType":{"description":"*(optional)* The value type of the global table","type":"string"}},"required":["topic"],"title":"GlobalTableDefinition","type":"object"},"GlobalTableDefinitionSource":{"additionalProperties":false,"description":"Contains a definition of a GlobalTable, which can be referenced by producers and pipelines","properties":{"keyType":{"description":"The key type of the global table","type":"string"},"offsetResetPolicy":{"description":"*(optional)* Policy that determines what to do when there is no initial offset in Kafka, or if the current offset does not exist any more on the server (e.g. because that data has been deleted)","type":"string"},"store":{"anyOf":[{"type":"string"},{"$ref":"#/definitions/KeyValueStateStoreDefinitionWithImplicitType"}],"description":"*(optional)* KeyValue state store definition"},"timestampExtractor":{"anyOf":[{"type":"string"},{"$ref":"#/definitions/TimestampExtractorDefinitionWithImplicitType"}],"description":"*(optional)* A function extracts the event time from a consumed record"},"topic":{"description":"The name of the Kafka topic for this global table","type":"string"},"valueType":{"description":"The value type of the global table","type":"string"}},"required":["topic","keyType","valueType"],"title":"GlobalTableDefinitionSource","type":"object"},"GroupByKeyOperation":{"additionalProperties":false,"description":"Operation to group all messages with the same key together","properties":{"name":{"description":"*(optional)* The name of the operation processor","type":"string"},"store":{"anyOf":[{"type":"string"},{"$ref":"#/definitions/KeyValueStateStoreDefinitionWithImplicitType"}],"description":"*(optional)* Materialized view of the grouped stream"},"type":{"description":"The type of the operation","enum":["groupByKey"]}},"required":["type"],"title":"GroupByKeyOperation","type":"object"},"GroupByOperation":{"additionalProperties":false,"description":"Operation to group all messages with together based on a keying function","properties":{"mapper":{"anyOf":[{"type":"string"},{"$ref":"#/definitions/KeyValueMapperDefinitionWithImplicitType"}],"description":"Function to map records to a key they can be grouped on"},"name":{"description":"*(optional)* The name of the operation processor","type":"string"},"store":{"anyOf":[{"type":"string"},{"$ref":"#/definitions/KeyValueStateStoreDefinitionWithImplicitType"}],"description":"*(optional)* Materialized view of the grouped stream or table"},"type":{"description":"The type of the operation","enum":["groupBy"]}},"required":["mapper","type"],"title":"GroupByOperation","type":"object"},"InitializerDefinition":{"additionalProperties":false,"description":"Defines a initializer function, that gets injected into the Kafka Streams topology","properties":{"code":{"anyOf":[{"type":"boolean"},{"type":"number"},{"type":"string"}],"description":"*(optional)* The (multiline) code of the initializer"},"expression":{"anyOf":[{"type":"boolean"},{"type":"number"},{"type":"string"}],"description":"*(optional)* The expression returned by the initializer. Only required for functions that return values."},"globalCode":{"anyOf":[{"type":"boolean"},{"type":"number"},{"type":"string"}],"description":"*(optional)* Global (multiline) code that gets loaded into the Python context outside of the initializer. Can be used for defining eg. global variables."},"name":{"description":"*(optional)* The name of the initializer. If this field is not defined, then the name is derived from the context.","type":"string"},"parameters":{"description":"*(optional)* A list of parameters to be passed into the initializer","items":{"$ref":"#/definitions/ParameterDefinition"},"type":"array"},"resultType":{"description":"*(optional)* The data type returned by the initializer. Only required for function types, which are not pre-defined.","type":"string"},"stores":{"description":"*(optional)* A list of store names that the initializer uses. Only required if the function wants to use a state store.","items":{"type":"string"},"type":"array"},"type":{"description":"The type of the function","enum":["initializer"]}},"required":["type"],"title":"InitializerDefinition","type":"object"},"InitializerDefinitionWithImplicitType":{"additionalProperties":false,"description":"Defines a initializer function, that gets injected into the Kafka Streams topology","properties":{"code":{"anyOf":[{"type":"boolean"},{"type":"number"},{"type":"string"}],"description":"*(optional)* The (multiline) code of the initializer"},"expression":{"anyOf":[{"type":"boolean"},{"type":"number"},{"type":"string"}],"description":"*(optional)* The expression returned by the initializer. Only required for functions that return values."},"globalCode":{"anyOf":[{"type":"boolean"},{"type":"number"},{"type":"string"}],"description":"*(optional)* Global (multiline) code that gets loaded into the Python context outside of the initializer. Can be used for defining eg. global variables."},"name":{"description":"*(optional)* The name of the initializer. If this field is not defined, then the name is derived from the context.","type":"string"},"parameters":{"description":"*(optional)* A list of parameters to be passed into the initializer","items":{"$ref":"#/definitions/ParameterDefinition"},"type":"array"},"resultType":{"description":"*(optional)* The data type returned by the initializer. Only required for function types, which are not pre-defined.","type":"string"},"stores":{"description":"*(optional)* A list of store names that the initializer uses. Only required if the function wants to use a state store.","items":{"type":"string"},"type":"array"}},"title":"InitializerDefinitionWithImplicitType","type":"object"},"JoinOperationWithGlobalTable":{"additionalProperties":false,"description":"Operation to join with a table","properties":{"globalTable":{"anyOf":[{"type":"string"},{"$ref":"#/definitions/GlobalTableDefinition"}],"description":"A reference to the globalTable, or an inline definition of the globalTable to join with"},"mapper":{"anyOf":[{"type":"string"},{"$ref":"#/definitions/ValueJoinerDefinitionWithImplicitType"}],"description":"A function that maps the key value from the stream with the primary key of the globalTable"},"name":{"description":"*(optional)* The name of the operation processor","type":"string"},"store":{"anyOf":[{"type":"string"},{"$ref":"#/definitions/KeyValueStateStoreDefinition"},{"$ref":"#/definitions/SessionStateStoreDefinition"},{"$ref":"#/definitions/WindowStateStoreDefinition"}],"description":"*(optional)* Materialized view of the joined streams"},"type":{"description":"The type of the operation","enum":["join"]},"valueJoiner":{"anyOf":[{"type":"string"},{"$ref":"#/definitions/ValueJoinerDefinitionWithImplicitType"}],"description":"A function that joins two values"}},"required":["type","globalTable","mapper","valueJoiner","type"],"title":"JoinOperationWithGlobalTable","type":"object"},"JoinOperationWithStream":{"additionalProperties":false,"description":"Operation to join with a stream","properties":{"grace":{"anyOf":[{"type":"number"},{"type":"string"}],"description":"*(optional)* The window grace period (the time to admit out-of-order events after the end of the window)"},"name":{"description":"*(optional)* The name of the operation processor","type":"string"},"store":{"anyOf":[{"type":"string"},{"$ref":"#/definitions/KeyValueStateStoreDefinition"},{"$ref":"#/definitions/SessionStateStoreDefinition"},{"$ref":"#/definitions/WindowStateStoreDefinition"}],"description":"*(optional)* Materialized view of the joined streams"},"stream":{"anyOf":[{"type":"string"},{"$ref":"#/definitions/StreamDefinition"}],"description":"A reference to the Stream, or an inline definition of the stream to join with"},"timeDifference":{"anyOf":[{"type":"number"},{"type":"string"}],"description":"The maximum time difference for a join over two streams on the same key"},"type":{"description":"The type of the operation","enum":["join"]},"valueJoiner":{"anyOf":[{"type":"string"},{"$ref":"#/definitions/ValueJoinerDefinitionWithImplicitType"}],"description":"A function that joins two values"}},"required":["stream","valueJoiner","timeDifference","type"],"title":"JoinOperationWithStream","type":"object"},"JoinOperationWithTable":{"additionalProperties":false,"description":"Operation to join with a table","properties":{"foreignKeyExtractor":{"anyOf":[{"type":"string"},{"$ref":"#/definitions/ValueJoinerDefinitionWithImplicitType"}],"description":"*(optional)* A function that can translate the join table value to a primary key"},"grace":{"anyOf":[{"type":"number"},{"type":"string"}],"description":"*(optional)* The window grace period (the time to admit out-of-order events after the end of the window)"},"name":{"description":"*(optional)* The name of the operation processor","type":"string"},"otherPartitioner":{"anyOf":[{"type":"string"},{"$ref":"#/definitions/StreamPartitionerDefinitionWithImplicitType"}],"description":"*(optional)* A function that partitions the records on the join table"},"partitioner":{"anyOf":[{"type":"string"},{"$ref":"#/definitions/StreamPartitionerDefinitionWithImplicitType"}],"description":"*(optional)* A function that partitions the records on the primary table"},"store":{"anyOf":[{"type":"string"},{"$ref":"#/definitions/KeyValueStateStoreDefinition"},{"$ref":"#/definitions/SessionStateStoreDefinition"},{"$ref":"#/definitions/WindowStateStoreDefinition"}],"description":"*(optional)* Materialized view of the joined streams"},"table":{"anyOf":[{"type":"string"},{"$ref":"#/definitions/TableDefinition"}],"description":"A reference to the table, or an inline definition of the table to join with"},"type":{"description":"The type of the operation","enum":["join"]},"valueJoiner":{"anyOf":[{"type":"string"},{"$ref":"#/definitions/ValueJoinerDefinitionWithImplicitType"}],"description":"A function that joins two values"}},"required":["type","table","valueJoiner","type"],"title":"JoinOperationWithTable","type":"object"},"KeyTransformerDefinition":{"additionalProperties":false,"description":"Defines a key transformer function, that gets injected into the Kafka Streams topology","properties":{"code":{"anyOf":[{"type":"boolean"},{"type":"number"},{"type":"string"}],"description":"*(optional)* The (multiline) code of the key transformer"},"expression":{"anyOf":[{"type":"boolean"},{"type":"number"},{"type":"string"}],"description":"*(optional)* The expression returned by the key transformer. Only required for functions that return values."},"globalCode":{"anyOf":[{"type":"boolean"},{"type":"number"},{"type":"string"}],"description":"*(optional)* Global (multiline) code that gets loaded into the Python context outside of the key transformer. Can be used for defining eg. global variables."},"name":{"description":"*(optional)* The name of the key transformer. If this field is not defined, then the name is derived from the context.","type":"string"},"parameters":{"description":"*(optional)* A list of parameters to be passed into the key transformer","items":{"$ref":"#/definitions/ParameterDefinition"},"type":"array"},"resultType":{"description":"*(optional)* The data type returned by the key transformer. Only required for function types, which are not pre-defined.","type":"string"},"stores":{"description":"*(optional)* A list of store names that the key transformer uses. Only required if the function wants to use a state store.","items":{"type":"string"},"type":"array"},"type":{"description":"The type of the function","enum":["keyTransformer"]}},"required":["type"],"title":"KeyTransformerDefinition","type":"object"},"KeyTransformerDefinitionWithImplicitType":{"additionalProperties":false,"description":"Defines a key transformer function, that gets injected into the Kafka Streams topology","properties":{"code":{"anyOf":[{"type":"boolean"},{"type":"number"},{"type":"string"}],"description":"*(optional)* The (multiline) code of the key transformer"},"expression":{"anyOf":[{"type":"boolean"},{"type":"number"},{"type":"string"}],"description":"*(optional)* The expression returned by the key transformer. Only required for functions that return values."},"globalCode":{"anyOf":[{"type":"boolean"},{"type":"number"},{"type":"string"}],"description":"*(optional)* Global (multiline) code that gets loaded into the Python context outside of the key transformer. Can be used for defining eg. global variables."},"name":{"description":"*(optional)* The name of the key transformer. If this field is not defined, then the name is derived from the context.","type":"string"},"parameters":{"description":"*(optional)* A list of parameters to be passed into the key transformer","items":{"$ref":"#/definitions/ParameterDefinition"},"type":"array"},"resultType":{"description":"*(optional)* The data type returned by the key transformer. Only required for function types, which are not pre-defined.","type":"string"},"stores":{"description":"*(optional)* A list of store names that the key transformer uses. Only required if the function wants to use a state store.","items":{"type":"string"},"type":"array"}},"title":"KeyTransformerDefinitionWithImplicitType","type":"object"},"KeyValueMapperDefinition":{"additionalProperties":false,"description":"Defines a keyvalue mapper function, that gets injected into the Kafka Streams topology","properties":{"code":{"anyOf":[{"type":"boolean"},{"type":"number"},{"type":"string"}],"description":"*(optional)* The (multiline) code of the keyvalue mapper"},"expression":{"anyOf":[{"type":"boolean"},{"type":"number"},{"type":"string"}],"description":"*(optional)* The expression returned by the keyvalue mapper. Only required for functions that return values."},"globalCode":{"anyOf":[{"type":"boolean"},{"type":"number"},{"type":"string"}],"description":"*(optional)* Global (multiline) code that gets loaded into the Python context outside of the keyvalue mapper. Can be used for defining eg. global variables."},"name":{"description":"*(optional)* The name of the keyvalue mapper. If this field is not defined, then the name is derived from the context.","type":"string"},"parameters":{"description":"*(optional)* A list of parameters to be passed into the keyvalue mapper","items":{"$ref":"#/definitions/ParameterDefinition"},"type":"array"},"resultType":{"description":"*(optional)* The data type returned by the keyvalue mapper. Only required for function types, which are not pre-defined.","type":"string"},"stores":{"description":"*(optional)* A list of store names that the keyvalue mapper uses. Only required if the function wants to use a state store.","items":{"type":"string"},"type":"array"},"type":{"description":"The type of the function","enum":["keyValueMapper"]}},"required":["type"],"title":"KeyValueMapperDefinition","type":"object"},"KeyValueMapperDefinitionWithImplicitType":{"additionalProperties":false,"description":"Defines a keyvalue mapper function, that gets injected into the Kafka Streams topology","properties":{"code":{"anyOf":[{"type":"boolean"},{"type":"number"},{"type":"string"}],"description":"*(optional)* The (multiline) code of the keyvalue mapper"},"expression":{"anyOf":[{"type":"boolean"},{"type":"number"},{"type":"string"}],"description":"*(optional)* The expression returned by the keyvalue mapper. Only required for functions that return values."},"globalCode":{"anyOf":[{"type":"boolean"},{"type":"number"},{"type":"string"}],"description":"*(optional)* Global (multiline) code that gets loaded into the Python context outside of the keyvalue mapper. Can be used for defining eg. global variables."},"name":{"description":"*(optional)* The name of the keyvalue mapper. If this field is not defined, then the name is derived from the context.","type":"string"},"parameters":{"description":"*(optional)* A list of parameters to be passed into the keyvalue mapper","items":{"$ref":"#/definitions/ParameterDefinition"},"type":"array"},"resultType":{"description":"*(optional)* The data type returned by the keyvalue mapper. Only required for function types, which are not pre-defined.","type":"string"},"stores":{"description":"*(optional)* A list of store names that the keyvalue mapper uses. Only required if the function wants to use a state store.","items":{"type":"string"},"type":"array"}},"title":"KeyValueMapperDefinitionWithImplicitType","type":"object"},"KeyValuePrinterDefinition":{"additionalProperties":false,"description":"Defines a keyvalue printer function, that gets injected into the Kafka Streams topology","properties":{"code":{"anyOf":[{"type":"boolean"},{"type":"number"},{"type":"string"}],"description":"*(optional)* The (multiline) code of the keyvalue printer"},"expression":{"anyOf":[{"type":"boolean"},{"type":"number"},{"type":"string"}],"description":"*(optional)* The expression returned by the keyvalue printer. Only required for functions that return values."},"globalCode":{"anyOf":[{"type":"boolean"},{"type":"number"},{"type":"string"}],"description":"*(optional)* Global (multiline) code that gets loaded into the Python context outside of the keyvalue printer. Can be used for defining eg. global variables."},"name":{"description":"*(optional)* The name of the keyvalue printer. If this field is not defined, then the name is derived from the context.","type":"string"},"parameters":{"description":"*(optional)* A list of parameters to be passed into the keyvalue printer","items":{"$ref":"#/definitions/ParameterDefinition"},"type":"array"},"resultType":{"description":"*(optional)* The data type returned by the keyvalue printer. Only required for function types, which are not pre-defined.","type":"string"},"stores":{"description":"*(optional)* A list of store names that the keyvalue printer uses. Only required if the function wants to use a state store.","items":{"type":"string"},"type":"array"},"type":{"description":"The type of the function","enum":["keyValuePrinter"]}},"required":["type"],"title":"KeyValuePrinterDefinition","type":"object"},"KeyValuePrinterDefinitionWithImplicitType":{"additionalProperties":false,"description":"Defines a keyvalue printer function, that gets injected into the Kafka Streams topology","properties":{"code":{"anyOf":[{"type":"boolean"},{"type":"number"},{"type":"string"}],"description":"*(optional)* The (multiline) code of the keyvalue printer"},"expression":{"anyOf":[{"type":"boolean"},{"type":"number"},{"type":"string"}],"description":"*(optional)* The expression returned by the keyvalue printer. Only required for functions that return values."},"globalCode":{"anyOf":[{"type":"boolean"},{"type":"number"},{"type":"string"}],"description":"*(optional)* Global (multiline) code that gets loaded into the Python context outside of the keyvalue printer. Can be used for defining eg. global variables."},"name":{"description":"*(optional)* The name of the keyvalue printer. If this field is not defined, then the name is derived from the context.","type":"string"},"parameters":{"description":"*(optional)* A list of parameters to be passed into the keyvalue printer","items":{"$ref":"#/definitions/ParameterDefinition"},"type":"array"},"resultType":{"description":"*(optional)* The data type returned by the keyvalue printer. Only required for function types, which are not pre-defined.","type":"string"},"stores":{"description":"*(optional)* A list of store names that the keyvalue printer uses. Only required if the function wants to use a state store.","items":{"type":"string"},"type":"array"}},"title":"KeyValuePrinterDefinitionWithImplicitType","type":"object"},"KeyValueStateStoreDefinition":{"additionalProperties":false,"description":"Definition of a keyValue state store","properties":{"caching":{"description":"*(optional)* \"true\" if changed to the keyValue store need to be buffered and periodically released, \"false\" to emit all changes directly","type":"boolean"},"historyRetention":{"anyOf":[{"type":"number"},{"type":"string"}],"description":"*(optional)* (Versioned only) The duration for which old record versions are available for query (cannot be negative)"},"keyType":{"description":"*(optional)* The key type of the keyValue store","type":"string"},"logging":{"description":"*(optional)* \"true\" if a changelog topic should be set up on Kafka for this keyValue store, \"false\" otherwise","type":"boolean"},"name":{"description":"*(optional)* The name of the keyValue store. If this field is not defined, then the name is derived from the context.","type":"string"},"persistent":{"description":"*(optional)* \"true\" if this keyValue store needs to be stored on disk, \"false\" otherwise","type":"boolean"},"segmentInterval":{"anyOf":[{"type":"number"},{"type":"string"}],"description":"*(optional)* Size of segments for storing old record versions (must be positive). Old record versions for the same key in a single segment are stored (updated and accessed) together. The only impact of this parameter is performance. If segments are large and a workload results in many record versions for the same key being collected in a single segment, performance may degrade as a result. On the other hand, historical reads (which access older segments) and out-of-order writes may slow down if there are too many segments."},"timestamped":{"description":"*(optional)* \"true\" if elements in the store are timestamped, \"false\" otherwise","type":"boolean"},"type":{"description":"The type of the state store","enum":["keyValue"]},"valueType":{"description":"*(optional)* The value type of the keyValue store","type":"string"},"versioned":{"description":"*(optional)* \"true\" if elements in the store are versioned, \"false\" otherwise","type":"boolean"}},"required":["type"],"title":"KeyValueStateStoreDefinition","type":"object"},"KeyValueStateStoreDefinitionWithImplicitType":{"additionalProperties":false,"description":"Definition of a keyValue state store","properties":{"caching":{"description":"*(optional)* \"true\" if changed to the keyValue store need to be buffered and periodically released, \"false\" to emit all changes directly","type":"boolean"},"historyRetention":{"anyOf":[{"type":"number"},{"type":"string"}],"description":"*(optional)* (Versioned only) The duration for which old record versions are available for query (cannot be negative)"},"keyType":{"description":"*(optional)* The key type of the keyValue store","type":"string"},"logging":{"description":"*(optional)* \"true\" if a changelog topic should be set up on Kafka for this keyValue store, \"false\" otherwise","type":"boolean"},"name":{"description":"*(optional)* The name of the keyValue store. If this field is not defined, then the name is derived from the context.","type":"string"},"persistent":{"description":"*(optional)* \"true\" if this keyValue store needs to be stored on disk, \"false\" otherwise","type":"boolean"},"segmentInterval":{"anyOf":[{"type":"number"},{"type":"string"}],"description":"*(optional)* Size of segments for storing old record versions (must be positive). Old record versions for the same key in a single segment are stored (updated and accessed) together. The only impact of this parameter is performance. If segments are large and a workload results in many record versions for the same key being collected in a single segment, performance may degrade as a result. On the other hand, historical reads (which access older segments) and out-of-order writes may slow down if there are too many segments."},"timestamped":{"description":"*(optional)* \"true\" if elements in the store are timestamped, \"false\" otherwise","type":"boolean"},"type":{"description":"The type of the state store","enum":["keyValue"]},"valueType":{"description":"*(optional)* The value type of the keyValue store","type":"string"},"versioned":{"description":"*(optional)* \"true\" if elements in the store are versioned, \"false\" otherwise","type":"boolean"}},"title":"KeyValueStateStoreDefinitionWithImplicitType","type":"object"},"KeyValueToKeyValueListTransformerDefinition":{"additionalProperties":false,"description":"Defines a keyvalue-to-keyvaluelist transformer function, that gets injected into the Kafka Streams topology","properties":{"code":{"anyOf":[{"type":"boolean"},{"type":"number"},{"type":"string"}],"description":"*(optional)* The (multiline) code of the keyvalue-to-keyvaluelist transformer"},"expression":{"anyOf":[{"type":"boolean"},{"type":"number"},{"type":"string"}],"description":"*(optional)* The expression returned by the keyvalue-to-keyvaluelist transformer. Only required for functions that return values."},"globalCode":{"anyOf":[{"type":"boolean"},{"type":"number"},{"type":"string"}],"description":"*(optional)* Global (multiline) code that gets loaded into the Python context outside of the keyvalue-to-keyvaluelist transformer. Can be used for defining eg. global variables."},"name":{"description":"*(optional)* The name of the keyvalue-to-keyvaluelist transformer. If this field is not defined, then the name is derived from the context.","type":"string"},"parameters":{"description":"*(optional)* A list of parameters to be passed into the keyvalue-to-keyvaluelist transformer","items":{"$ref":"#/definitions/ParameterDefinition"},"type":"array"},"resultType":{"description":"*(optional)* The data type returned by the keyvalue-to-keyvaluelist transformer. Only required for function types, which are not pre-defined.","type":"string"},"stores":{"description":"*(optional)* A list of store names that the keyvalue-to-keyvaluelist transformer uses. Only required if the function wants to use a state store.","items":{"type":"string"},"type":"array"},"type":{"description":"The type of the function","enum":["keyValueToKeyValueListTransformer"]}},"required":["type"],"title":"KeyValueToKeyValueListTransformerDefinition","type":"object"},"KeyValueToKeyValueListTransformerDefinitionWithImplicitType":{"additionalProperties":false,"description":"Defines a keyvalue-to-keyvaluelist transformer function, that gets injected into the Kafka Streams topology","properties":{"code":{"anyOf":[{"type":"boolean"},{"type":"number"},{"type":"string"}],"description":"*(optional)* The (multiline) code of the keyvalue-to-keyvaluelist transformer"},"expression":{"anyOf":[{"type":"boolean"},{"type":"number"},{"type":"string"}],"description":"*(optional)* The expression returned by the keyvalue-to-keyvaluelist transformer. Only required for functions that return values."},"globalCode":{"anyOf":[{"type":"boolean"},{"type":"number"},{"type":"string"}],"description":"*(optional)* Global (multiline) code that gets loaded into the Python context outside of the keyvalue-to-keyvaluelist transformer. Can be used for defining eg. global variables."},"name":{"description":"*(optional)* The name of the keyvalue-to-keyvaluelist transformer. If this field is not defined, then the name is derived from the context.","type":"string"},"parameters":{"description":"*(optional)* A list of parameters to be passed into the keyvalue-to-keyvaluelist transformer","items":{"$ref":"#/definitions/ParameterDefinition"},"type":"array"},"resultType":{"description":"*(optional)* The data type returned by the keyvalue-to-keyvaluelist transformer. Only required for function types, which are not pre-defined.","type":"string"},"stores":{"description":"*(optional)* A list of store names that the keyvalue-to-keyvaluelist transformer uses. Only required if the function wants to use a state store.","items":{"type":"string"},"type":"array"}},"title":"KeyValueToKeyValueListTransformerDefinitionWithImplicitType","type":"object"},"KeyValueToValueListTransformerDefinition":{"additionalProperties":false,"description":"Defines a keyvalue-to-valuelist transformer function, that gets injected into the Kafka Streams topology","properties":{"code":{"anyOf":[{"type":"boolean"},{"type":"number"},{"type":"string"}],"description":"*(optional)* The (multiline) code of the keyvalue-to-valuelist transformer"},"expression":{"anyOf":[{"type":"boolean"},{"type":"number"},{"type":"string"}],"description":"*(optional)* The expression returned by the keyvalue-to-valuelist transformer. Only required for functions that return values."},"globalCode":{"anyOf":[{"type":"boolean"},{"type":"number"},{"type":"string"}],"description":"*(optional)* Global (multiline) code that gets loaded into the Python context outside of the keyvalue-to-valuelist transformer. Can be used for defining eg. global variables."},"name":{"description":"*(optional)* The name of the keyvalue-to-valuelist transformer. If this field is not defined, then the name is derived from the context.","type":"string"},"parameters":{"description":"*(optional)* A list of parameters to be passed into the keyvalue-to-valuelist transformer","items":{"$ref":"#/definitions/ParameterDefinition"},"type":"array"},"resultType":{"description":"*(optional)* The data type returned by the keyvalue-to-valuelist transformer. Only required for function types, which are not pre-defined.","type":"string"},"stores":{"description":"*(optional)* A list of store names that the keyvalue-to-valuelist transformer uses. Only required if the function wants to use a state store.","items":{"type":"string"},"type":"array"},"type":{"description":"The type of the function","enum":["keyValueToValueListTransformer"]}},"required":["type"],"title":"KeyValueToValueListTransformerDefinition","type":"object"},"KeyValueToValueListTransformerDefinitionWithImplicitType":{"additionalProperties":false,"description":"Defines a keyvalue-to-valuelist transformer function, that gets injected into the Kafka Streams topology","properties":{"code":{"anyOf":[{"type":"boolean"},{"type":"number"},{"type":"string"}],"description":"*(optional)* The (multiline) code of the keyvalue-to-valuelist transformer"},"expression":{"anyOf":[{"type":"boolean"},{"type":"number"},{"type":"string"}],"description":"*(optional)* The expression returned by the keyvalue-to-valuelist transformer. Only required for functions that return values."},"globalCode":{"anyOf":[{"type":"boolean"},{"type":"number"},{"type":"string"}],"description":"*(optional)* Global (multiline) code that gets loaded into the Python context outside of the keyvalue-to-valuelist transformer. Can be used for defining eg. global variables."},"name":{"description":"*(optional)* The name of the keyvalue-to-valuelist transformer. If this field is not defined, then the name is derived from the context.","type":"string"},"parameters":{"description":"*(optional)* A list of parameters to be passed into the keyvalue-to-valuelist transformer","items":{"$ref":"#/definitions/ParameterDefinition"},"type":"array"},"resultType":{"description":"*(optional)* The data type returned by the keyvalue-to-valuelist transformer. Only required for function types, which are not pre-defined.","type":"string"},"stores":{"description":"*(optional)* A list of store names that the keyvalue-to-valuelist transformer uses. Only required if the function wants to use a state store.","items":{"type":"string"},"type":"array"}},"title":"KeyValueToValueListTransformerDefinitionWithImplicitType","type":"object"},"KeyValueTransformerDefinition":{"additionalProperties":false,"description":"Defines a keyvalue transformer function, that gets injected into the Kafka Streams topology","properties":{"code":{"anyOf":[{"type":"boolean"},{"type":"number"},{"type":"string"}],"description":"*(optional)* The (multiline) code of the keyvalue transformer"},"expression":{"anyOf":[{"type":"boolean"},{"type":"number"},{"type":"string"}],"description":"*(optional)* The expression returned by the keyvalue transformer. Only required for functions that return values."},"globalCode":{"anyOf":[{"type":"boolean"},{"type":"number"},{"type":"string"}],"description":"*(optional)* Global (multiline) code that gets loaded into the Python context outside of the keyvalue transformer. Can be used for defining eg. global variables."},"name":{"description":"*(optional)* The name of the keyvalue transformer. If this field is not defined, then the name is derived from the context.","type":"string"},"parameters":{"description":"*(optional)* A list of parameters to be passed into the keyvalue transformer","items":{"$ref":"#/definitions/ParameterDefinition"},"type":"array"},"resultType":{"description":"*(optional)* The data type returned by the keyvalue transformer. Only required for function types, which are not pre-defined.","type":"string"},"stores":{"description":"*(optional)* A list of store names that the keyvalue transformer uses. Only required if the function wants to use a state store.","items":{"type":"string"},"type":"array"},"type":{"description":"The type of the function","enum":["keyValueTransformer"]}},"required":["type"],"title":"KeyValueTransformerDefinition","type":"object"},"KeyValueTransformerDefinitionWithImplicitType":{"additionalProperties":false,"description":"Defines a keyvalue transformer function, that gets injected into the Kafka Streams topology","properties":{"code":{"anyOf":[{"type":"boolean"},{"type":"number"},{"type":"string"}],"description":"*(optional)* The (multiline) code of the keyvalue transformer"},"expression":{"anyOf":[{"type":"boolean"},{"type":"number"},{"type":"string"}],"description":"*(optional)* The expression returned by the keyvalue transformer. Only required for functions that return values."},"globalCode":{"anyOf":[{"type":"boolean"},{"type":"number"},{"type":"string"}],"description":"*(optional)* Global (multiline) code that gets loaded into the Python context outside of the keyvalue transformer. Can be used for defining eg. global variables."},"name":{"description":"*(optional)* The name of the keyvalue transformer. If this field is not defined, then the name is derived from the context.","type":"string"},"parameters":{"description":"*(optional)* A list of parameters to be passed into the keyvalue transformer","items":{"$ref":"#/definitions/ParameterDefinition"},"type":"array"},"resultType":{"description":"*(optional)* The data type returned by the keyvalue transformer. Only required for function types, which are not pre-defined.","type":"string"},"stores":{"description":"*(optional)* A list of store names that the keyvalue transformer uses. Only required if the function wants to use a state store.","items":{"type":"string"},"type":"array"}},"title":"KeyValueTransformerDefinitionWithImplicitType","type":"object"},"LeftJoinOperationWithGlobalTable":{"additionalProperties":false,"description":"Operation to leftJoin with a globalTable","properties":{"globalTable":{"anyOf":[{"type":"string"},{"$ref":"#/definitions/GlobalTableDefinition"}],"description":"A reference to the globalTable, or an inline definition of the globalTable to join with"},"mapper":{"anyOf":[{"type":"string"},{"$ref":"#/definitions/ValueJoinerDefinitionWithImplicitType"}],"description":"A function that maps the key value from the stream with the primary key of the globalTable"},"name":{"description":"*(optional)* The name of the operation processor","type":"string"},"store":{"anyOf":[{"type":"string"},{"$ref":"#/definitions/KeyValueStateStoreDefinition"},{"$ref":"#/definitions/SessionStateStoreDefinition"},{"$ref":"#/definitions/WindowStateStoreDefinition"}],"description":"*(optional)* Materialized view of the leftJoined streams"},"type":{"description":"The type of the operation","enum":["leftJoin"]},"valueJoiner":{"anyOf":[{"type":"string"},{"$ref":"#/definitions/ValueJoinerDefinitionWithImplicitType"}],"description":"A function that joins two values"}},"required":["globalTable","mapper","valueJoiner","type"],"title":"LeftJoinOperationWithGlobalTable","type":"object"},"LeftJoinOperationWithStream":{"additionalProperties":false,"description":"Operation to leftJoin with a stream","properties":{"grace":{"anyOf":[{"type":"number"},{"type":"string"}],"description":"*(optional)* The window grace period (the time to admit out-of-order events after the end of the window)"},"name":{"description":"*(optional)* The name of the operation processor","type":"string"},"store":{"anyOf":[{"type":"string"},{"$ref":"#/definitions/KeyValueStateStoreDefinition"},{"$ref":"#/definitions/SessionStateStoreDefinition"},{"$ref":"#/definitions/WindowStateStoreDefinition"}],"description":"*(optional)* Materialized view of the joined streams"},"stream":{"anyOf":[{"type":"string"},{"$ref":"#/definitions/StreamDefinition"}],"description":"A reference to the stream, or an inline definition of the stream to leftJoin with"},"timeDifference":{"anyOf":[{"type":"number"},{"type":"string"}],"description":"The maximum time difference for a leftJoin over two streams on the same key"},"type":{"description":"The type of the operation","enum":["leftJoin"]},"valueJoiner":{"anyOf":[{"type":"string"},{"$ref":"#/definitions/ValueJoinerDefinitionWithImplicitType"}],"description":"A function that joins two values"}},"required":["stream","valueJoiner","timeDifference","type"],"title":"LeftJoinOperationWithStream","type":"object"},"LeftJoinOperationWithTable":{"additionalProperties":false,"description":"Operation to leftJoin with a table","properties":{"foreignKeyExtractor":{"anyOf":[{"type":"string"},{"$ref":"#/definitions/ValueJoinerDefinitionWithImplicitType"}],"description":"*(optional)* A function that can translate the join table value to a primary key"},"grace":{"anyOf":[{"type":"number"},{"type":"string"}],"description":"*(optional)* The window grace period (the time to admit out-of-order events after the end of the window)"},"name":{"description":"*(optional)* The name of the operation processor","type":"string"},"otherPartitioner":{"anyOf":[{"type":"string"},{"$ref":"#/definitions/StreamPartitionerDefinitionWithImplicitType"}],"description":"*(optional)* A function that partitions the records on the join table"},"partitioner":{"anyOf":[{"type":"string"},{"$ref":"#/definitions/StreamPartitionerDefinitionWithImplicitType"}],"description":"*(optional)* A function that partitions the records on the primary table"},"store":{"anyOf":[{"type":"string"},{"$ref":"#/definitions/KeyValueStateStoreDefinition"},{"$ref":"#/definitions/SessionStateStoreDefinition"},{"$ref":"#/definitions/WindowStateStoreDefinition"}],"description":"*(optional)* Materialized view of the joined streams"},"table":{"anyOf":[{"type":"string"},{"$ref":"#/definitions/TableDefinition"}],"description":"A reference to the Table, or an inline definition of the Table to join with"},"type":{"description":"The type of the operation","enum":["leftJoin"]},"valueJoiner":{"anyOf":[{"type":"string"},{"$ref":"#/definitions/ValueJoinerDefinitionWithImplicitType"}],"description":"A function that joins two values"}},"required":["table","valueJoiner","type"],"title":"LeftJoinOperationWithTable","type":"object"},"MergeOperation":{"additionalProperties":false,"description":"A merge operation to join two Streams","properties":{"name":{"description":"*(optional)* The name of the operation processor","type":"string"},"stream":{"anyOf":[{"type":"string"},{"$ref":"#/definitions/StreamDefinitionSource"}],"description":"The stream to merge with"},"type":{"description":"The type of the operation","enum":["merge"]}},"required":["stream","type"],"title":"MergeOperation","type":"object"},"MergerDefinition":{"additionalProperties":false,"description":"Defines a merger function, that gets injected into the Kafka Streams topology","properties":{"code":{"anyOf":[{"type":"boolean"},{"type":"number"},{"type":"string"}],"description":"*(optional)* The (multiline) code of the merger"},"expression":{"anyOf":[{"type":"boolean"},{"type":"number"},{"type":"string"}],"description":"*(optional)* The expression returned by the merger. Only required for functions that return values."},"globalCode":{"anyOf":[{"type":"boolean"},{"type":"number"},{"type":"string"}],"description":"*(optional)* Global (multiline) code that gets loaded into the Python context outside of the merger. Can be used for defining eg. global variables."},"name":{"description":"*(optional)* The name of the merger. If this field is not defined, then the name is derived from the context.","type":"string"},"parameters":{"description":"*(optional)* A list of parameters to be passed into the merger","items":{"$ref":"#/definitions/ParameterDefinition"},"type":"array"},"resultType":{"description":"*(optional)* The data type returned by the merger. Only required for function types, which are not pre-defined.","type":"string"},"stores":{"description":"*(optional)* A list of store names that the merger uses. Only required if the function wants to use a state store.","items":{"type":"string"},"type":"array"},"type":{"description":"The type of the function","enum":["merger"]}},"required":["type"],"title":"MergerDefinition","type":"object"},"MergerDefinitionWithImplicitType":{"additionalProperties":false,"description":"Defines a merger function, that gets injected into the Kafka Streams topology","properties":{"code":{"anyOf":[{"type":"boolean"},{"type":"number"},{"type":"string"}],"description":"*(optional)* The (multiline) code of the merger"},"expression":{"anyOf":[{"type":"boolean"},{"type":"number"},{"type":"string"}],"description":"*(optional)* The expression returned by the merger. Only required for functions that return values."},"globalCode":{"anyOf":[{"type":"boolean"},{"type":"number"},{"type":"string"}],"description":"*(optional)* Global (multiline) code that gets loaded into the Python context outside of the merger. Can be used for defining eg. global variables."},"name":{"description":"*(optional)* The name of the merger. If this field is not defined, then the name is derived from the context.","type":"string"},"parameters":{"description":"*(optional)* A list of parameters to be passed into the merger","items":{"$ref":"#/definitions/ParameterDefinition"},"type":"array"},"resultType":{"description":"*(optional)* The data type returned by the merger. Only required for function types, which are not pre-defined.","type":"string"},"stores":{"description":"*(optional)* A list of store names that the merger uses. Only required if the function wants to use a state store.","items":{"type":"string"},"type":"array"}},"title":"MergerDefinitionWithImplicitType","type":"object"},"MetadataTransformerDefinition":{"additionalProperties":false,"description":"Defines a metadata transformer function, that gets injected into the Kafka Streams topology","properties":{"code":{"anyOf":[{"type":"boolean"},{"type":"number"},{"type":"string"}],"description":"*(optional)* The (multiline) code of the metadata transformer"},"expression":{"anyOf":[{"type":"boolean"},{"type":"number"},{"type":"string"}],"description":"*(optional)* The expression returned by the metadata transformer. Only required for functions that return values."},"globalCode":{"anyOf":[{"type":"boolean"},{"type":"number"},{"type":"string"}],"description":"*(optional)* Global (multiline) code that gets loaded into the Python context outside of the metadata transformer. Can be used for defining eg. global variables."},"name":{"description":"*(optional)* The name of the metadata transformer. If this field is not defined, then the name is derived from the context.","type":"string"},"parameters":{"description":"*(optional)* A list of parameters to be passed into the metadata transformer","items":{"$ref":"#/definitions/ParameterDefinition"},"type":"array"},"resultType":{"description":"*(optional)* The data type returned by the metadata transformer. Only required for function types, which are not pre-defined.","type":"string"},"stores":{"description":"*(optional)* A list of store names that the metadata transformer uses. Only required if the function wants to use a state store.","items":{"type":"string"},"type":"array"},"type":{"description":"The type of the function","enum":["metadataTransformer"]}},"required":["type"],"title":"MetadataTransformerDefinition","type":"object"},"MetadataTransformerDefinitionWithImplicitType":{"additionalProperties":false,"description":"Defines a metadata transformer function, that gets injected into the Kafka Streams topology","properties":{"code":{"anyOf":[{"type":"boolean"},{"type":"number"},{"type":"string"}],"description":"*(optional)* The (multiline) code of the metadata transformer"},"expression":{"anyOf":[{"type":"boolean"},{"type":"number"},{"type":"string"}],"description":"*(optional)* The expression returned by the metadata transformer. Only required for functions that return values."},"globalCode":{"anyOf":[{"type":"boolean"},{"type":"number"},{"type":"string"}],"description":"*(optional)* Global (multiline) code that gets loaded into the Python context outside of the metadata transformer. Can be used for defining eg. global variables."},"name":{"description":"*(optional)* The name of the metadata transformer. If this field is not defined, then the name is derived from the context.","type":"string"},"parameters":{"description":"*(optional)* A list of parameters to be passed into the metadata transformer","items":{"$ref":"#/definitions/ParameterDefinition"},"type":"array"},"resultType":{"description":"*(optional)* The data type returned by the metadata transformer. Only required for function types, which are not pre-defined.","type":"string"},"stores":{"description":"*(optional)* A list of store names that the metadata transformer uses. Only required if the function wants to use a state store.","items":{"type":"string"},"type":"array"}},"title":"MetadataTransformerDefinitionWithImplicitType","type":"object"},"OuterJoinOperationWithStream":{"additionalProperties":false,"description":"Operation to outerJoin with a stream","properties":{"grace":{"anyOf":[{"type":"number"},{"type":"string"}],"description":"*(optional)* The window grace period (the time to admit out-of-order events after the end of the window)"},"name":{"description":"*(optional)* The name of the operation processor","type":"string"},"store":{"anyOf":[{"type":"string"},{"$ref":"#/definitions/KeyValueStateStoreDefinition"},{"$ref":"#/definitions/SessionStateStoreDefinition"},{"$ref":"#/definitions/WindowStateStoreDefinition"}],"description":"*(optional)* Materialized view of the outerJoined streams"},"stream":{"anyOf":[{"type":"string"},{"$ref":"#/definitions/StreamDefinition"}],"description":"A reference to the stream, or an inline definition of the stream to outerJoin with"},"timeDifference":{"anyOf":[{"type":"number"},{"type":"string"}],"description":"The maximum time difference for an outerJoin over two streams on the same key"},"type":{"description":"The type of the operation","enum":["outerJoin"]},"valueJoiner":{"anyOf":[{"type":"string"},{"$ref":"#/definitions/ValueJoinerDefinitionWithImplicitType"}],"description":"A function that joins two values"}},"required":["stream","valueJoiner","timeDifference","type"],"title":"OuterJoinOperationWithStream","type":"object"},"OuterJoinOperationWithTable":{"additionalProperties":false,"description":"Operation to outerJoin with a table","properties":{"name":{"description":"*(optional)* The name of the operation processor","type":"string"},"store":{"anyOf":[{"type":"string"},{"$ref":"#/definitions/KeyValueStateStoreDefinition"},{"$ref":"#/definitions/SessionStateStoreDefinition"},{"$ref":"#/definitions/WindowStateStoreDefinition"}],"description":"*(optional)* Materialized view of the outerJoined streams"},"table":{"anyOf":[{"type":"string"},{"$ref":"#/definitions/TableDefinition"}],"description":"A reference to the table, or an inline definition of the table to outerJoin with"},"type":{"description":"The type of the operation","enum":["outerJoin"]},"valueJoiner":{"anyOf":[{"type":"string"},{"$ref":"#/definitions/ValueJoinerDefinitionWithImplicitType"}],"description":"A function that joins two values"}},"required":["table","valueJoiner","type"],"title":"OuterJoinOperationWithTable","type":"object"},"ParameterDefinition":{"additionalProperties":false,"description":"Defines a parameter for a user function","properties":{"defaultValue":{"description":"*(optional)* The default value for the parameter","type":"string"},"name":{"description":"The name of the parameter","type":"string"},"type":{"description":"The type of the parameter","type":"string"}},"required":["name","type"],"title":"ParameterDefinition","type":"object"},"PeekOperation":{"additionalProperties":false,"description":"Operation to peek into a stream, without modifying the stream contents","properties":{"forEach":{"anyOf":[{"type":"string"},{"$ref":"#/definitions/ForEachActionDefinitionWithImplicitType"}],"description":"A function that gets called for every message in the stream"},"name":{"description":"*(optional)* The name of the operation processor","type":"string"},"stores":{"description":"*(optional)* The names of all state stores used by the function","items":{"type":"string"},"type":"array"},"type":{"description":"The type of the operation","enum":["peek"]}},"required":["forEach","type"],"title":"PeekOperation","type":"object"},"PipelineDefinition":{"additionalProperties":false,"description":"Defines a pipeline through a source, a series of operations to perform on it and a sink operation to close the stream with","properties":{"as":{"description":"*(optional)* The name to register the pipeline result under, which can be used as source by follow-up pipelines","type":"string"},"branch":{"description":"*(optional)* Defines a single branch, consisting of a condition and a pipeline to execute for messages that fulfil the predicate","items":{"$ref":"#/definitions/BranchDefinitionWithPipeline"},"type":"array"},"forEach":{"anyOf":[{"type":"string"},{"$ref":"#/definitions/ForEachActionDefinitionWithImplicitType"}],"description":"*(optional)* A function that gets called for every message in the stream"},"from":{"anyOf":[{"type":"string"},{"$ref":"#/definitions/TopicDefinitionSource"}],"description":"Pipeline source"},"name":{"description":"*(optional)* The name of the operation processor","type":"string"},"print":{"$ref":"#/definitions/PrintOperation","description":"*(optional)* The specification of where to print messages to"},"stores":{"description":"*(optional)* The names of all state stores used by the function","items":{"type":"string"},"type":"array"},"to":{"anyOf":[{"type":"string"},{"$ref":"#/definitions/ToTopicDefinition"}],"description":"*(optional)* Ends the pipeline by sending all messages to a stream, table or globalTable, or to an inline defined output topic and optional partitioner"},"toTopicNameExtractor":{"anyOf":[{"type":"string"},{"$ref":"#/definitions/ToTopicNameExtractorDefinition"}],"description":"*(optional)* Ends the pipeline by sending all messages to a topic provided by a pre-defined topic name extractor function, or to a topic provided by an inline defined topic name extractor and optional partitioner"},"via":{"description":"*(optional)* A series of operations performed on the input stream","items":{"anyOf":[{"$ref":"#/definitions/AggregateOperation"},{"$ref":"#/definitions/CogroupOperation"},{"$ref":"#/definitions/ConvertKeyOperation"},{"$ref":"#/definitions/ConvertKeyValueOperation"},{"$ref":"#/definitions/ConvertValueOperation"},{"$ref":"#/definitions/CountOperation"},{"$ref":"#/definitions/FilterNotOperation"},{"$ref":"#/definitions/FilterOperation"},{"$ref":"#/definitions/GroupByKeyOperation"},{"$ref":"#/definitions/GroupByOperation"},{"$ref":"#/definitions/JoinOperationWithGlobalTable"},{"$ref":"#/definitions/JoinOperationWithStream"},{"$ref":"#/definitions/JoinOperationWithTable"},{"$ref":"#/definitions/LeftJoinOperationWithGlobalTable"},{"$ref":"#/definitions/LeftJoinOperationWithStream"},{"$ref":"#/definitions/LeftJoinOperationWithTable"},{"$ref":"#/definitions/MergeOperation"},{"$ref":"#/definitions/OuterJoinOperationWithStream"},{"$ref":"#/definitions/OuterJoinOperationWithTable"},{"$ref":"#/definitions/PeekOperation"},{"$ref":"#/definitions/ReduceOperation"},{"$ref":"#/definitions/RepartitionOperation"},{"$ref":"#/definitions/SuppressOperation"},{"$ref":"#/definitions/ToStreamOperation"},{"$ref":"#/definitions/ToTableOperation"},{"$ref":"#/definitions/TransformKeyOperation"},{"$ref":"#/definitions/TransformKeyValueOperation"},{"$ref":"#/definitions/TransformKeyValueToKeyValueListOperation"},{"$ref":"#/definitions/TransformKeyValueToValueListOperation"},{"$ref":"#/definitions/TransformMetadataOperation"},{"$ref":"#/definitions/TransformValueOperation"},{"$ref":"#/definitions/WindowBySessionOperation"},{"$ref":"#/definitions/WindowByTimeOperation"}]},"type":"array"}},"required":["from"],"title":"PipelineDefinition","type":"object"},"PredicateDefinition":{"additionalProperties":false,"description":"Defines a Function that returns true or false based on key/value input function, that gets injected into the Kafka Streams topology","properties":{"code":{"anyOf":[{"type":"boolean"},{"type":"number"},{"type":"string"}],"description":"*(optional)* The (multiline) code of the Function that returns true or false based on key/value input"},"expression":{"anyOf":[{"type":"boolean"},{"type":"number"},{"type":"string"}],"description":"*(optional)* The expression returned by the Function that returns true or false based on key/value input. Only required for functions that return values."},"globalCode":{"anyOf":[{"type":"boolean"},{"type":"number"},{"type":"string"}],"description":"*(optional)* Global (multiline) code that gets loaded into the Python context outside of the Function that returns true or false based on key/value input. Can be used for defining eg. global variables."},"name":{"description":"*(optional)* The name of the Function that returns true or false based on key/value input. If this field is not defined, then the name is derived from the context.","type":"string"},"parameters":{"description":"*(optional)* A list of parameters to be passed into the Function that returns true or false based on key/value input","items":{"$ref":"#/definitions/ParameterDefinition"},"type":"array"},"resultType":{"description":"*(optional)* The data type returned by the Function that returns true or false based on key/value input. Only required for function types, which are not pre-defined.","type":"string"},"stores":{"description":"*(optional)* A list of store names that the Function that returns true or false based on key/value input uses. Only required if the function wants to use a state store.","items":{"type":"string"},"type":"array"},"type":{"description":"The type of the function","enum":["predicate"]}},"required":["type"],"title":"PredicateDefinition","type":"object"},"PredicateDefinitionWithImplicitType":{"additionalProperties":false,"description":"Defines a Function that returns true or false based on key/value input function, that gets injected into the Kafka Streams topology","properties":{"code":{"anyOf":[{"type":"boolean"},{"type":"number"},{"type":"string"}],"description":"*(optional)* The (multiline) code of the Function that returns true or false based on key/value input"},"expression":{"anyOf":[{"type":"boolean"},{"type":"number"},{"type":"string"}],"description":"*(optional)* The expression returned by the Function that returns true or false based on key/value input. Only required for functions that return values."},"globalCode":{"anyOf":[{"type":"boolean"},{"type":"number"},{"type":"string"}],"description":"*(optional)* Global (multiline) code that gets loaded into the Python context outside of the Function that returns true or false based on key/value input. Can be used for defining eg. global variables."},"name":{"description":"*(optional)* The name of the Function that returns true or false based on key/value input. If this field is not defined, then the name is derived from the context.","type":"string"},"parameters":{"description":"*(optional)* A list of parameters to be passed into the Function that returns true or false based on key/value input","items":{"$ref":"#/definitions/ParameterDefinition"},"type":"array"},"resultType":{"description":"*(optional)* The data type returned by the Function that returns true or false based on key/value input. Only required for function types, which are not pre-defined.","type":"string"},"stores":{"description":"*(optional)* A list of store names that the Function that returns true or false based on key/value input uses. Only required if the function wants to use a state store.","items":{"type":"string"},"type":"array"}},"title":"PredicateDefinitionWithImplicitType","type":"object"},"PrintOperation":{"additionalProperties":false,"description":"Operation to print the contents of a pipeline on the screen or to write them to a file","properties":{"filename":{"description":"*(optional)* The filename to output records to. If nothing is specified, then messages will be printed on stdout","type":"string"},"label":{"description":"*(optional)* A label to attach to the output records","type":"string"},"mapper":{"anyOf":[{"type":"string"},{"$ref":"#/definitions/KeyValuePrinterDefinitionWithImplicitType"}],"description":"*(optional)* A function to convert record into a string for output"},"name":{"description":"*(optional)* The name of the operation processor","type":"string"}},"title":"PrintOperation","type":"object"},"ProducerDefinition":{"additionalProperties":false,"description":"Definition of a Producer that regularly generates messages for a topic","properties":{"condition":{"anyOf":[{"type":"string"},{"$ref":"#/definitions/PredicateDefinitionWithImplicitType"}],"description":"*(optional)* A function that validates the generator's result message. Returns \"true\" when the message may be produced on the topic, \"false\" otherwise."},"count":{"description":"*(optional)* The number of messages to produce."},"generator":{"anyOf":[{"type":"string"},{"$ref":"#/definitions/GeneratorDefinitionWithImplicitType"}],"description":"The function that generates records"},"interval":{"anyOf":[{"type":"number"},{"type":"string"}],"description":"The interval with which the generator is called"},"to":{"anyOf":[{"type":"string"},{"$ref":"#/definitions/TopicDefinition"}],"description":"The topic to produce to"},"until":{"anyOf":[{"type":"string"},{"$ref":"#/definitions/PredicateDefinitionWithImplicitType"}],"description":"*(optional)* A predicate that returns true to indicate producing should stop."}},"required":["generator","interval","to"],"title":"ProducerDefinition","type":"object"},"ReduceOperation":{"additionalProperties":false,"description":"Operation to reduce a series of records into a single aggregate result","properties":{"adder":{"anyOf":[{"type":"string"},{"$ref":"#/definitions/ReducerDefinitionWithImplicitType"}],"description":"A function that adds a record to the aggregate result"},"name":{"description":"*(optional)* The name of the operation processor","type":"string"},"reducer":{"anyOf":[{"type":"string"},{"$ref":"#/definitions/ReducerDefinitionWithImplicitType"}],"description":"A function that computes a new aggregate result"},"store":{"anyOf":[{"type":"string"},{"$ref":"#/definitions/WindowStateStoreDefinitionWithImplicitType"}],"description":"*(optional)* Materialized view of the aggregation"},"subtractor":{"anyOf":[{"type":"string"},{"$ref":"#/definitions/ReducerDefinitionWithImplicitType"}],"description":"A function that removes a record from the aggregate result"},"type":{"description":"The type of the operation","enum":["reduce"]}},"required":["reducer","adder","subtractor","type"],"title":"ReduceOperation","type":"object"},"ReducerDefinition":{"additionalProperties":false,"description":"Defines a reducer function, that gets injected into the Kafka Streams topology","properties":{"code":{"anyOf":[{"type":"boolean"},{"type":"number"},{"type":"string"}],"description":"*(optional)* The (multiline) code of the reducer"},"expression":{"anyOf":[{"type":"boolean"},{"type":"number"},{"type":"string"}],"description":"*(optional)* The expression returned by the reducer. Only required for functions that return values."},"globalCode":{"anyOf":[{"type":"boolean"},{"type":"number"},{"type":"string"}],"description":"*(optional)* Global (multiline) code that gets loaded into the Python context outside of the reducer. Can be used for defining eg. global variables."},"name":{"description":"*(optional)* The name of the reducer. If this field is not defined, then the name is derived from the context.","type":"string"},"parameters":{"description":"*(optional)* A list of parameters to be passed into the reducer","items":{"$ref":"#/definitions/ParameterDefinition"},"type":"array"},"resultType":{"description":"*(optional)* The data type returned by the reducer. Only required for function types, which are not pre-defined.","type":"string"},"stores":{"description":"*(optional)* A list of store names that the reducer uses. Only required if the function wants to use a state store.","items":{"type":"string"},"type":"array"},"type":{"description":"The type of the function","enum":["reducer"]}},"required":["type"],"title":"ReducerDefinition","type":"object"},"ReducerDefinitionWithImplicitType":{"additionalProperties":false,"description":"Defines a reducer function, that gets injected into the Kafka Streams topology","properties":{"code":{"anyOf":[{"type":"boolean"},{"type":"number"},{"type":"string"}],"description":"*(optional)* The (multiline) code of the reducer"},"expression":{"anyOf":[{"type":"boolean"},{"type":"number"},{"type":"string"}],"description":"*(optional)* The expression returned by the reducer. Only required for functions that return values."},"globalCode":{"anyOf":[{"type":"boolean"},{"type":"number"},{"type":"string"}],"description":"*(optional)* Global (multiline) code that gets loaded into the Python context outside of the reducer. Can be used for defining eg. global variables."},"name":{"description":"*(optional)* The name of the reducer. If this field is not defined, then the name is derived from the context.","type":"string"},"parameters":{"description":"*(optional)* A list of parameters to be passed into the reducer","items":{"$ref":"#/definitions/ParameterDefinition"},"type":"array"},"resultType":{"description":"*(optional)* The data type returned by the reducer. Only required for function types, which are not pre-defined.","type":"string"},"stores":{"description":"*(optional)* A list of store names that the reducer uses. Only required if the function wants to use a state store.","items":{"type":"string"},"type":"array"}},"title":"ReducerDefinitionWithImplicitType","type":"object"},"RepartitionOperation":{"additionalProperties":false,"description":"Operation to (re)partition a stream","properties":{"name":{"description":"*(optional)* The name of the operation processor","type":"string"},"partitioner":{"anyOf":[{"type":"string"},{"$ref":"#/definitions/StreamPartitionerDefinitionWithImplicitType"}],"description":"A function that partitions stream records"},"type":{"description":"The type of the operation","enum":["repartition"]}},"required":["partitioner","type"],"title":"RepartitionOperation","type":"object"},"SessionStateStoreDefinition":{"additionalProperties":false,"description":"Definition of a session state store","properties":{"caching":{"description":"*(optional)* \"true\" if changed to the session store need to be buffered and periodically released, \"false\" to emit all changes directly","type":"boolean"},"keyType":{"description":"*(optional)* The key type of the session store","type":"string"},"logging":{"description":"*(optional)* \"true\" if a changelog topic should be set up on Kafka for this session store, \"false\" otherwise","type":"boolean"},"name":{"description":"*(optional)* The name of the session store. If this field is not defined, then the name is derived from the context.","type":"string"},"persistent":{"description":"*(optional)* \"true\" if this session store needs to be stored on disk, \"false\" otherwise","type":"boolean"},"retention":{"anyOf":[{"type":"number"},{"type":"string"}],"description":"*(optional)* The duration for which elements in the session store are retained"},"timestamped":{"description":"*(optional)* \"true\" if elements in the store are timestamped, \"false\" otherwise","type":"boolean"},"type":{"description":"The type of the state store","enum":["session"]},"valueType":{"description":"*(optional)* The value type of the session store","type":"string"}},"required":["type"],"title":"SessionStateStoreDefinition","type":"object"},"StreamDefinition":{"additionalProperties":false,"description":"Contains a definition of a Stream, which can be referenced by producers and pipelines","properties":{"keyType":{"description":"*(optional)* The key type of the stream","type":"string"},"topic":{"description":"The name of the Kafka topic for this stream","type":"string"},"valueType":{"description":"*(optional)* The value type of the stream","type":"string"}},"required":["topic"],"title":"StreamDefinition","type":"object"},"StreamDefinitionSource":{"additionalProperties":false,"description":"Contains a definition of a Stream, which can be referenced by producers and pipelines","properties":{"keyType":{"description":"The key type of the stream","type":"string"},"offsetResetPolicy":{"description":"*(optional)* Policy that determines what to do when there is no initial offset in Kafka, or if the current offset does not exist any more on the server (e.g. because that data has been deleted)","type":"string"},"timestampExtractor":{"anyOf":[{"type":"string"},{"$ref":"#/definitions/TimestampExtractorDefinitionWithImplicitType"}],"description":"*(optional)* A function extracts the event time from a consumed record"},"topic":{"description":"The name of the Kafka topic for this stream","type":"string"},"valueType":{"description":"The value type of the stream","type":"string"}},"required":["topic","keyType","valueType"],"title":"StreamDefinitionSource","type":"object"},"StreamPartitionerDefinition":{"additionalProperties":false,"description":"Defines a stream partitioner function, that gets injected into the Kafka Streams topology","properties":{"code":{"anyOf":[{"type":"boolean"},{"type":"number"},{"type":"string"}],"description":"*(optional)* The (multiline) code of the stream partitioner"},"expression":{"anyOf":[{"type":"boolean"},{"type":"number"},{"type":"string"}],"description":"*(optional)* The expression returned by the stream partitioner. Only required for functions that return values."},"globalCode":{"anyOf":[{"type":"boolean"},{"type":"number"},{"type":"string"}],"description":"*(optional)* Global (multiline) code that gets loaded into the Python context outside of the stream partitioner. Can be used for defining eg. global variables."},"name":{"description":"*(optional)* The name of the stream partitioner. If this field is not defined, then the name is derived from the context.","type":"string"},"parameters":{"description":"*(optional)* A list of parameters to be passed into the stream partitioner","items":{"$ref":"#/definitions/ParameterDefinition"},"type":"array"},"resultType":{"description":"*(optional)* The data type returned by the stream partitioner. Only required for function types, which are not pre-defined.","type":"string"},"stores":{"description":"*(optional)* A list of store names that the stream partitioner uses. Only required if the function wants to use a state store.","items":{"type":"string"},"type":"array"},"type":{"description":"The type of the function","enum":["streamPartitioner"]}},"required":["type"],"title":"StreamPartitionerDefinition","type":"object"},"StreamPartitionerDefinitionWithImplicitType":{"additionalProperties":false,"description":"Defines a stream partitioner function, that gets injected into the Kafka Streams topology","properties":{"code":{"anyOf":[{"type":"boolean"},{"type":"number"},{"type":"string"}],"description":"*(optional)* The (multiline) code of the stream partitioner"},"expression":{"anyOf":[{"type":"boolean"},{"type":"number"},{"type":"string"}],"description":"*(optional)* The expression returned by the stream partitioner. Only required for functions that return values."},"globalCode":{"anyOf":[{"type":"boolean"},{"type":"number"},{"type":"string"}],"description":"*(optional)* Global (multiline) code that gets loaded into the Python context outside of the stream partitioner. Can be used for defining eg. global variables."},"name":{"description":"*(optional)* The name of the stream partitioner. If this field is not defined, then the name is derived from the context.","type":"string"},"parameters":{"description":"*(optional)* A list of parameters to be passed into the stream partitioner","items":{"$ref":"#/definitions/ParameterDefinition"},"type":"array"},"resultType":{"description":"*(optional)* The data type returned by the stream partitioner. Only required for function types, which are not pre-defined.","type":"string"},"stores":{"description":"*(optional)* A list of store names that the stream partitioner uses. Only required if the function wants to use a state store.","items":{"type":"string"},"type":"array"}},"title":"StreamPartitionerDefinitionWithImplicitType","type":"object"},"StringOrInlinePredicateDefinitionWithImplicitType":{"additionalProperties":false,"description":"Defines the condition under which messages get sent down this branch","properties":{"if":{"anyOf":[{"type":"string"},{"$ref":"#/definitions/PredicateDefinitionWithImplicitType"}],"description":"*(optional)* Defines the condition under which messages get sent down this branch"}},"title":"StringOrInlinePredicateDefinitionWithImplicitType","type":"object"},"SuppressOperation":{"additionalProperties":false,"description":"Operation to suppress messages in the source stream until a certain limit is reached","properties":{"bufferFullStrategy":{"description":"*(optional)* What to do when the buffer is full, either \"emitEarlyWhenFull\", or \"shutdownWhenFull\"","type":"string"},"duration":{"anyOf":[{"type":"number"},{"type":"string"}],"description":"The duration for which messages are suppressed"},"maxBytes":{"description":"*(optional)* The maximum number of bytes in the buffer","type":"string"},"maxRecords":{"description":"*(optional)* The maximum number of records in the buffer","type":"string"},"name":{"description":"*(optional)* The name of the operation processor","type":"string"},"type":{"description":"The type of the operation","enum":["suppress"]},"until":{"description":"The method by which messages are held, either \"timeLimit\", or \"windowCloses\"","type":"string"}},"required":["until","duration","type"],"title":"SuppressOperation","type":"object"},"TableDefinition":{"additionalProperties":false,"description":"Contains a definition of a Table, which can be referenced by producers and pipelines","properties":{"keyType":{"description":"*(optional)* The key type of the table","type":"string"},"store":{"anyOf":[{"type":"string"},{"$ref":"#/definitions/KeyValueStateStoreDefinitionWithImplicitType"}],"description":"*(optional)* KeyValue state store definition"},"topic":{"description":"The name of the Kafka topic for this table","type":"string"},"valueType":{"description":"*(optional)* The value type of the table","type":"string"}},"required":["topic"],"title":"TableDefinition","type":"object"},"TableDefinitionSource":{"additionalProperties":false,"description":"Contains a definition of a Table, which can be referenced by producers and pipelines","properties":{"keyType":{"description":"The key type of the table","type":"string"},"offsetResetPolicy":{"description":"*(optional)* Policy that determines what to do when there is no initial offset in Kafka, or if the current offset does not exist any more on the server (e.g. because that data has been deleted)","type":"string"},"store":{"anyOf":[{"type":"string"},{"$ref":"#/definitions/KeyValueStateStoreDefinitionWithImplicitType"}],"description":"*(optional)* KeyValue state store definition"},"timestampExtractor":{"anyOf":[{"type":"string"},{"$ref":"#/definitions/TimestampExtractorDefinitionWithImplicitType"}],"description":"*(optional)* A function extracts the event time from a consumed record"},"topic":{"description":"The name of the Kafka topic for this table","type":"string"},"valueType":{"description":"The value type of the table","type":"string"}},"required":["topic","keyType","valueType"],"title":"TableDefinitionSource","type":"object"},"TimestampExtractorDefinition":{"additionalProperties":false,"description":"Defines a timestamp extractor function, that gets injected into the Kafka Streams topology","properties":{"code":{"anyOf":[{"type":"boolean"},{"type":"number"},{"type":"string"}],"description":"*(optional)* The (multiline) code of the timestamp extractor"},"expression":{"anyOf":[{"type":"boolean"},{"type":"number"},{"type":"string"}],"description":"*(optional)* The expression returned by the timestamp extractor. Only required for functions that return values."},"globalCode":{"anyOf":[{"type":"boolean"},{"type":"number"},{"type":"string"}],"description":"*(optional)* Global (multiline) code that gets loaded into the Python context outside of the timestamp extractor. Can be used for defining eg. global variables."},"name":{"description":"*(optional)* The name of the timestamp extractor. If this field is not defined, then the name is derived from the context.","type":"string"},"parameters":{"description":"*(optional)* A list of parameters to be passed into the timestamp extractor","items":{"$ref":"#/definitions/ParameterDefinition"},"type":"array"},"resultType":{"description":"*(optional)* The data type returned by the timestamp extractor. Only required for function types, which are not pre-defined.","type":"string"},"stores":{"description":"*(optional)* A list of store names that the timestamp extractor uses. Only required if the function wants to use a state store.","items":{"type":"string"},"type":"array"},"type":{"description":"The type of the function","enum":["timestampExtractor"]}},"required":["type"],"title":"TimestampExtractorDefinition","type":"object"},"TimestampExtractorDefinitionWithImplicitType":{"additionalProperties":false,"description":"Defines a timestamp extractor function, that gets injected into the Kafka Streams topology","properties":{"code":{"anyOf":[{"type":"boolean"},{"type":"number"},{"type":"string"}],"description":"*(optional)* The (multiline) code of the timestamp extractor"},"expression":{"anyOf":[{"type":"boolean"},{"type":"number"},{"type":"string"}],"description":"*(optional)* The expression returned by the timestamp extractor. Only required for functions that return values."},"globalCode":{"anyOf":[{"type":"boolean"},{"type":"number"},{"type":"string"}],"description":"*(optional)* Global (multiline) code that gets loaded into the Python context outside of the timestamp extractor. Can be used for defining eg. global variables."},"name":{"description":"*(optional)* The name of the timestamp extractor. If this field is not defined, then the name is derived from the context.","type":"string"},"parameters":{"description":"*(optional)* A list of parameters to be passed into the timestamp extractor","items":{"$ref":"#/definitions/ParameterDefinition"},"type":"array"},"resultType":{"description":"*(optional)* The data type returned by the timestamp extractor. Only required for function types, which are not pre-defined.","type":"string"},"stores":{"description":"*(optional)* A list of store names that the timestamp extractor uses. Only required if the function wants to use a state store.","items":{"type":"string"},"type":"array"}},"title":"TimestampExtractorDefinitionWithImplicitType","type":"object"},"ToStreamOperation":{"additionalProperties":false,"description":"Convert a Table into a Stream, optionally through a custom key transformer","properties":{"mapper":{"anyOf":[{"type":"string"},{"$ref":"#/definitions/KeyTransformerDefinitionWithImplicitType"}],"description":"*(optional)* A function that computes the output key for every record"},"name":{"description":"*(optional)* The name of the operation processor","type":"string"},"type":{"description":"The type of the operation","enum":["toStream"]}},"required":["type"],"title":"ToStreamOperation","type":"object"},"ToTableOperation":{"additionalProperties":false,"description":"Convert a Stream into a Table","properties":{"name":{"description":"*(optional)* The name of the operation processor","type":"string"},"store":{"anyOf":[{"type":"string"},{"$ref":"#/definitions/KeyValueStateStoreDefinitionWithImplicitType"}],"description":"*(optional)* Materialized view of the result table"},"type":{"description":"The type of the operation","enum":["toTable"]}},"required":["type"],"title":"ToTableOperation","type":"object"},"ToTopicDefinition":{"additionalProperties":false,"description":"Writes out pipeline messages to a topic","properties":{"keyType":{"description":"*(optional)* The key type of the topic","type":"string"},"partitioner":{"anyOf":[{"type":"string"},{"$ref":"#/definitions/StreamPartitionerDefinitionWithImplicitType"}],"description":"*(optional)* A function that partitions the records in the output topic"},"topic":{"description":"The name of the Kafka topic","type":"string"},"valueType":{"description":"*(optional)* The value type of the topic","type":"string"}},"required":["topic"],"title":"ToTopicDefinition","type":"object"},"ToTopicNameExtractorDefinition":{"additionalProperties":false,"description":"Writes out pipeline messages to a topic as given by a topic name extractor","properties":{"partitioner":{"anyOf":[{"type":"string"},{"$ref":"#/definitions/StreamPartitionerDefinitionWithImplicitType"}],"description":"*(optional)* A function that partitions the records in the output topic"},"topicNameExtractor":{"anyOf":[{"type":"string"},{"$ref":"#/definitions/TopicNameExtractorDefinitionWithImplicitType"}],"description":"Reference to a pre-defined topic name extractor, or an inline definition of a topic name extractor"}},"required":["topicNameExtractor"],"title":"ToTopicNameExtractorDefinition","type":"object"},"TopicDefinition":{"additionalProperties":false,"description":"Contains a definition of a Kafka topic, to be used by producers and pipelines","properties":{"keyType":{"description":"*(optional)* The key type of the topic","type":"string"},"topic":{"description":"The name of the Kafka topic","type":"string"},"valueType":{"description":"*(optional)* The value type of the topic","type":"string"}},"required":["topic"],"title":"TopicDefinition","type":"object"},"TopicDefinitionSource":{"additionalProperties":false,"description":"Contains a definition of a Kafka topic, to be used by producers and pipelines","properties":{"keyType":{"description":"The key type of the topic","type":"string"},"offsetResetPolicy":{"description":"*(optional)* Policy that determines what to do when there is no initial offset in Kafka, or if the current offset does not exist any more on the server (e.g. because that data has been deleted)","type":"string"},"timestampExtractor":{"anyOf":[{"type":"string"},{"$ref":"#/definitions/TimestampExtractorDefinitionWithImplicitType"}],"description":"*(optional)* A function extracts the event time from a consumed record"},"topic":{"description":"The name of the Kafka topic","type":"string"},"valueType":{"description":"The value type of the topic","type":"string"}},"required":["topic","keyType","valueType"],"title":"TopicDefinitionSource","type":"object"},"TopicNameExtractorDefinition":{"additionalProperties":false,"description":"Defines a topic name extractor function, that gets injected into the Kafka Streams topology","properties":{"code":{"anyOf":[{"type":"boolean"},{"type":"number"},{"type":"string"}],"description":"*(optional)* The (multiline) code of the topic name extractor"},"expression":{"anyOf":[{"type":"boolean"},{"type":"number"},{"type":"string"}],"description":"*(optional)* The expression returned by the topic name extractor. Only required for functions that return values."},"globalCode":{"anyOf":[{"type":"boolean"},{"type":"number"},{"type":"string"}],"description":"*(optional)* Global (multiline) code that gets loaded into the Python context outside of the topic name extractor. Can be used for defining eg. global variables."},"name":{"description":"*(optional)* The name of the topic name extractor. If this field is not defined, then the name is derived from the context.","type":"string"},"parameters":{"description":"*(optional)* A list of parameters to be passed into the topic name extractor","items":{"$ref":"#/definitions/ParameterDefinition"},"type":"array"},"resultType":{"description":"*(optional)* The data type returned by the topic name extractor. Only required for function types, which are not pre-defined.","type":"string"},"stores":{"description":"*(optional)* A list of store names that the topic name extractor uses. Only required if the function wants to use a state store.","items":{"type":"string"},"type":"array"},"type":{"description":"The type of the function","enum":["topicNameExtractor"]}},"required":["type"],"title":"TopicNameExtractorDefinition","type":"object"},"TopicNameExtractorDefinitionWithImplicitType":{"additionalProperties":false,"description":"Defines a topic name extractor function, that gets injected into the Kafka Streams topology","properties":{"code":{"anyOf":[{"type":"boolean"},{"type":"number"},{"type":"string"}],"description":"*(optional)* The (multiline) code of the topic name extractor"},"expression":{"anyOf":[{"type":"boolean"},{"type":"number"},{"type":"string"}],"description":"*(optional)* The expression returned by the topic name extractor. Only required for functions that return values."},"globalCode":{"anyOf":[{"type":"boolean"},{"type":"number"},{"type":"string"}],"description":"*(optional)* Global (multiline) code that gets loaded into the Python context outside of the topic name extractor. Can be used for defining eg. global variables."},"name":{"description":"*(optional)* The name of the topic name extractor. If this field is not defined, then the name is derived from the context.","type":"string"},"parameters":{"description":"*(optional)* A list of parameters to be passed into the topic name extractor","items":{"$ref":"#/definitions/ParameterDefinition"},"type":"array"},"resultType":{"description":"*(optional)* The data type returned by the topic name extractor. Only required for function types, which are not pre-defined.","type":"string"},"stores":{"description":"*(optional)* A list of store names that the topic name extractor uses. Only required if the function wants to use a state store.","items":{"type":"string"},"type":"array"}},"title":"TopicNameExtractorDefinitionWithImplicitType","type":"object"},"TransformKeyOperation":{"additionalProperties":false,"description":"Convert the key of every record in the stream to another key","properties":{"mapper":{"anyOf":[{"type":"string"},{"$ref":"#/definitions/KeyTransformerDefinitionWithImplicitType"}],"description":"A function that computes a new key for each record"},"name":{"description":"*(optional)* The name of the operation processor","type":"string"},"stores":{"description":"*(optional)* The names of all state stores used by the function","items":{"type":"string"},"type":"array"},"type":{"description":"The type of the operation","enum":["transformKey","mapKey","selectKey"]}},"required":["mapper","type"],"title":"TransformKeyOperation","type":"object"},"TransformKeyValueOperation":{"additionalProperties":false,"description":"Convert the key/value of every record in the stream to another key/value","properties":{"mapper":{"anyOf":[{"type":"string"},{"$ref":"#/definitions/KeyValueTransformerDefinitionWithImplicitType"}],"description":"A function that computes a new key/value for each record"},"name":{"description":"*(optional)* The name of the operation processor","type":"string"},"stores":{"description":"*(optional)* The names of all state stores used by the function","items":{"type":"string"},"type":"array"},"type":{"description":"The type of the operation","enum":["mapKeyValue","map","transformKeyValue"]}},"required":["mapper","type"],"title":"TransformKeyValueOperation","type":"object"},"TransformKeyValueToKeyValueListOperation":{"additionalProperties":false,"description":"Convert a stream by transforming every record into a list of derived records","properties":{"mapper":{"anyOf":[{"type":"string"},{"$ref":"#/definitions/KeyValueToKeyValueListTransformerDefinitionWithImplicitType"}],"description":"A function that converts every record of a stream to a list of output records."},"name":{"description":"*(optional)* The name of the operation processor","type":"string"},"stores":{"description":"*(optional)* The names of all state stores used by the function","items":{"type":"string"},"type":"array"},"type":{"description":"The type of the operation","enum":["transformKeyValueToKeyValueList","flatMap"]}},"required":["mapper","type"],"title":"TransformKeyValueToKeyValueListOperation","type":"object"},"TransformKeyValueToValueListOperation":{"additionalProperties":false,"description":"Convert every record in the stream to a list of output records with the same key","properties":{"mapper":{"anyOf":[{"type":"string"},{"$ref":"#/definitions/KeyValueToValueListTransformerDefinitionWithImplicitType"}],"description":"A function that converts every key/value into a list of result values, each of which will be combined with the original key to form a new message in the output stream"},"name":{"description":"*(optional)* The name of the operation processor","type":"string"},"stores":{"description":"*(optional)* The names of all state stores used by the function","items":{"type":"string"},"type":"array"},"type":{"description":"The type of the operation","enum":["transformKeyValueToValueList","flatMapValues"]}},"required":["mapper","type"],"title":"TransformKeyValueToValueListOperation","type":"object"},"TransformMetadataOperation":{"additionalProperties":false,"description":"Convert the metadata of every record in the stream","properties":{"mapper":{"anyOf":[{"type":"string"},{"$ref":"#/definitions/MetadataTransformerDefinitionWithImplicitType"}],"description":"A function that converts the metadata (Kafka headers, timestamp) of every record in the stream"},"name":{"description":"*(optional)* The name of the operation processor","type":"string"},"stores":{"description":"*(optional)* The names of all state stores used by the function","items":{"type":"string"},"type":"array"},"type":{"description":"The type of the operation","enum":["transformMetadata"]}},"required":["mapper","type"],"title":"TransformMetadataOperation","type":"object"},"TransformValueOperation":{"additionalProperties":false,"description":"Convert the value of every record in the stream to another value","properties":{"mapper":{"anyOf":[{"type":"string"},{"$ref":"#/definitions/ValueTransformerDefinitionWithImplicitType"}],"description":"A function that converts the value of every record into another value"},"name":{"description":"*(optional)* The name of the operation processor","type":"string"},"store":{"anyOf":[{"type":"string"},{"$ref":"#/definitions/KeyValueStateStoreDefinitionWithImplicitType"}],"description":"*(optional)* Materialized view of the transformed table (only applies to tables, ignored for streams)"},"stores":{"description":"*(optional)* The names of all state stores used by the function","items":{"type":"string"},"type":"array"},"type":{"description":"The type of the operation","enum":["mapValue","transformValue","mapValues"]}},"required":["mapper","type"],"title":"TransformValueOperation","type":"object"},"ValueJoinerDefinition":{"additionalProperties":false,"description":"Defines a value joiner function, that gets injected into the Kafka Streams topology","properties":{"code":{"anyOf":[{"type":"boolean"},{"type":"number"},{"type":"string"}],"description":"*(optional)* The (multiline) code of the value joiner"},"expression":{"anyOf":[{"type":"boolean"},{"type":"number"},{"type":"string"}],"description":"*(optional)* The expression returned by the value joiner. Only required for functions that return values."},"globalCode":{"anyOf":[{"type":"boolean"},{"type":"number"},{"type":"string"}],"description":"*(optional)* Global (multiline) code that gets loaded into the Python context outside of the value joiner. Can be used for defining eg. global variables."},"name":{"description":"*(optional)* The name of the value joiner. If this field is not defined, then the name is derived from the context.","type":"string"},"parameters":{"description":"*(optional)* A list of parameters to be passed into the value joiner","items":{"$ref":"#/definitions/ParameterDefinition"},"type":"array"},"resultType":{"description":"*(optional)* The data type returned by the value joiner. Only required for function types, which are not pre-defined.","type":"string"},"stores":{"description":"*(optional)* A list of store names that the value joiner uses. Only required if the function wants to use a state store.","items":{"type":"string"},"type":"array"},"type":{"description":"The type of the function","enum":["valueJoiner"]}},"required":["type"],"title":"ValueJoinerDefinition","type":"object"},"ValueJoinerDefinitionWithImplicitType":{"additionalProperties":false,"description":"Defines a value joiner function, that gets injected into the Kafka Streams topology","properties":{"code":{"anyOf":[{"type":"boolean"},{"type":"number"},{"type":"string"}],"description":"*(optional)* The (multiline) code of the value joiner"},"expression":{"anyOf":[{"type":"boolean"},{"type":"number"},{"type":"string"}],"description":"*(optional)* The expression returned by the value joiner. Only required for functions that return values."},"globalCode":{"anyOf":[{"type":"boolean"},{"type":"number"},{"type":"string"}],"description":"*(optional)* Global (multiline) code that gets loaded into the Python context outside of the value joiner. Can be used for defining eg. global variables."},"name":{"description":"*(optional)* The name of the value joiner. If this field is not defined, then the name is derived from the context.","type":"string"},"parameters":{"description":"*(optional)* A list of parameters to be passed into the value joiner","items":{"$ref":"#/definitions/ParameterDefinition"},"type":"array"},"resultType":{"description":"*(optional)* The data type returned by the value joiner. Only required for function types, which are not pre-defined.","type":"string"},"stores":{"description":"*(optional)* A list of store names that the value joiner uses. Only required if the function wants to use a state store.","items":{"type":"string"},"type":"array"}},"title":"ValueJoinerDefinitionWithImplicitType","type":"object"},"ValueTransformerDefinition":{"additionalProperties":false,"description":"Defines a value transformer function, that gets injected into the Kafka Streams topology","properties":{"code":{"anyOf":[{"type":"boolean"},{"type":"number"},{"type":"string"}],"description":"*(optional)* The (multiline) code of the value transformer"},"expression":{"anyOf":[{"type":"boolean"},{"type":"number"},{"type":"string"}],"description":"*(optional)* The expression returned by the value transformer. Only required for functions that return values."},"globalCode":{"anyOf":[{"type":"boolean"},{"type":"number"},{"type":"string"}],"description":"*(optional)* Global (multiline) code that gets loaded into the Python context outside of the value transformer. Can be used for defining eg. global variables."},"name":{"description":"*(optional)* The name of the value transformer. If this field is not defined, then the name is derived from the context.","type":"string"},"parameters":{"description":"*(optional)* A list of parameters to be passed into the value transformer","items":{"$ref":"#/definitions/ParameterDefinition"},"type":"array"},"resultType":{"description":"*(optional)* The data type returned by the value transformer. Only required for function types, which are not pre-defined.","type":"string"},"stores":{"description":"*(optional)* A list of store names that the value transformer uses. Only required if the function wants to use a state store.","items":{"type":"string"},"type":"array"},"type":{"description":"The type of the function","enum":["valueTransformer"]}},"required":["type"],"title":"ValueTransformerDefinition","type":"object"},"ValueTransformerDefinitionWithImplicitType":{"additionalProperties":false,"description":"Defines a value transformer function, that gets injected into the Kafka Streams topology","properties":{"code":{"anyOf":[{"type":"boolean"},{"type":"number"},{"type":"string"}],"description":"*(optional)* The (multiline) code of the value transformer"},"expression":{"anyOf":[{"type":"boolean"},{"type":"number"},{"type":"string"}],"description":"*(optional)* The expression returned by the value transformer. Only required for functions that return values."},"globalCode":{"anyOf":[{"type":"boolean"},{"type":"number"},{"type":"string"}],"description":"*(optional)* Global (multiline) code that gets loaded into the Python context outside of the value transformer. Can be used for defining eg. global variables."},"name":{"description":"*(optional)* The name of the value transformer. If this field is not defined, then the name is derived from the context.","type":"string"},"parameters":{"description":"*(optional)* A list of parameters to be passed into the value transformer","items":{"$ref":"#/definitions/ParameterDefinition"},"type":"array"},"resultType":{"description":"*(optional)* The data type returned by the value transformer. Only required for function types, which are not pre-defined.","type":"string"},"stores":{"description":"*(optional)* A list of store names that the value transformer uses. Only required if the function wants to use a state store.","items":{"type":"string"},"type":"array"}},"title":"ValueTransformerDefinitionWithImplicitType","type":"object"},"WindowBySessionOperation":{"additionalProperties":false,"description":"Operation to window messages by session, configured by an inactivity gap","properties":{"grace":{"anyOf":[{"type":"number"},{"type":"string"}],"description":"*(optional)* (Tumbling, Hopping) The grace period, during which out-of-order records can still be processed"},"inactivityGap":{"anyOf":[{"type":"number"},{"type":"string"}],"description":"The inactivity gap, below which two messages are considered to be of the same session"},"name":{"description":"*(optional)* The name of the operation processor","type":"string"},"type":{"description":"The type of the operation","enum":["windowBySession"]}},"required":["inactivityGap","type"],"title":"WindowBySessionOperation","type":"object"},"WindowByTimeOperation":{"additionalProperties":false,"description":"Operation to reduce a series of records into a single aggregate result","properties":{"advanceBy":{"anyOf":[{"type":"number"},{"type":"string"}],"description":"*(optional)* (Hopping) The amount of time to increase time windows by"},"duration":{"anyOf":[{"type":"number"},{"type":"string"}],"description":"*(optional)* (Tumbling) The duration of time windows"},"grace":{"anyOf":[{"type":"number"},{"type":"string"}],"description":"*(optional)* (Tumbling, Hopping, Sliding) The grace period, during which out-of-order records can still be processed"},"name":{"description":"*(optional)* The name of the operation processor","type":"string"},"timeDifference":{"anyOf":[{"type":"number"},{"type":"string"}],"description":"*(optional)* (Sliding) The maximum amount of time difference between two records"},"type":{"description":"The type of the operation","enum":["windowByTime"]},"windowType":{"description":"The type of the operation, either \"tumbling\", or \"hopping\", or \"sliding\"","type":"string"}},"required":["windowType","type"],"title":"WindowByTimeOperation","type":"object"},"WindowStateStoreDefinition":{"additionalProperties":false,"description":"Definition of a window state store","properties":{"caching":{"description":"*(optional)* \"true\" if changed to the window store need to be buffered and periodically released, \"false\" to emit all changes directly","type":"boolean"},"keyType":{"description":"*(optional)* The key type of the window store","type":"string"},"logging":{"description":"*(optional)* \"true\" if a changelog topic should be set up on Kafka for this window store, \"false\" otherwise","type":"boolean"},"name":{"description":"*(optional)* The name of the window store. If this field is not defined, then the name is derived from the context.","type":"string"},"persistent":{"description":"*(optional)* \"true\" if this window store needs to be stored on disk, \"false\" otherwise","type":"boolean"},"retainDuplicates":{"description":"*(optional)* Whether or not to retain duplicates","type":"boolean"},"retention":{"anyOf":[{"type":"number"},{"type":"string"}],"description":"*(optional)* The duration for which elements in the window store are retained"},"timestamped":{"description":"*(optional)* \"true\" if elements in the store are timestamped, \"false\" otherwise","type":"boolean"},"type":{"description":"The type of the state store","enum":["window"]},"valueType":{"description":"*(optional)* The value type of the window store","type":"string"},"windowSize":{"anyOf":[{"type":"number"},{"type":"string"}],"description":"*(optional)* Size of the windows (cannot be negative)"}},"required":["type"],"title":"WindowStateStoreDefinition","type":"object"},"WindowStateStoreDefinitionWithImplicitType":{"additionalProperties":false,"description":"Definition of a window state store","properties":{"caching":{"description":"*(optional)* \"true\" if changed to the window store need to be buffered and periodically released, \"false\" to emit all changes directly","type":"boolean"},"keyType":{"description":"*(optional)* The key type of the window store","type":"string"},"logging":{"description":"*(optional)* \"true\" if a changelog topic should be set up on Kafka for this window store, \"false\" otherwise","type":"boolean"},"name":{"description":"*(optional)* The name of the window store. If this field is not defined, then the name is derived from the context.","type":"string"},"persistent":{"description":"*(optional)* \"true\" if this window store needs to be stored on disk, \"false\" otherwise","type":"boolean"},"retainDuplicates":{"description":"*(optional)* Whether or not to retain duplicates","type":"boolean"},"retention":{"anyOf":[{"type":"number"},{"type":"string"}],"description":"*(optional)* The duration for which elements in the window store are retained"},"timestamped":{"description":"*(optional)* \"true\" if elements in the store are timestamped, \"false\" otherwise","type":"boolean"},"type":{"description":"The type of the state store","enum":["window"]},"valueType":{"description":"*(optional)* The value type of the window store","type":"string"},"windowSize":{"anyOf":[{"type":"number"},{"type":"string"}],"description":"*(optional)* Size of the windows (cannot be negative)"}},"title":"WindowStateStoreDefinitionWithImplicitType","type":"object"}},"description":"KSML definition","properties":{"functions":{"description":"*(optional)* Functions that can be referenced in producers and pipelines","patternProperties":{"^[a-zA-Z0-9_]+$":{"anyOf":[{"$ref":"#/definitions/AggregatorDefinition"},{"$ref":"#/definitions/ForEachActionDefinition"},{"$ref":"#/definitions/ForeignKeyExtractorDefinition"},{"$ref":"#/definitions/GeneratorDefinition"},{"$ref":"#/definitions/GenericFunctionDefinitionWithImplicitType"},{"$ref":"#/definitions/InitializerDefinition"},{"$ref":"#/definitions/KeyTransformerDefinition"},{"$ref":"#/definitions/KeyValueMapperDefinition"},{"$ref":"#/definitions/KeyValuePrinterDefinition"},{"$ref":"#/definitions/KeyValueToKeyValueListTransformerDefinition"},{"$ref":"#/definitions/KeyValueToValueListTransformerDefinition"},{"$ref":"#/definitions/KeyValueTransformerDefinition"},{"$ref":"#/definitions/MergerDefinition"},{"$ref":"#/definitions/MetadataTransformerDefinition"},{"$ref":"#/definitions/PredicateDefinition"},{"$ref":"#/definitions/ReducerDefinition"},{"$ref":"#/definitions/StreamPartitionerDefinition"},{"$ref":"#/definitions/TimestampExtractorDefinition"},{"$ref":"#/definitions/TopicNameExtractorDefinition"},{"$ref":"#/definitions/ValueJoinerDefinition"},{"$ref":"#/definitions/ValueTransformerDefinition"}]}},"type":"object"},"globalTables":{"description":"*(optional)* GlobalTables that can be referenced in producers and pipelines","patternProperties":{"^[a-zA-Z0-9_]+$":{"$ref":"#/definitions/GlobalTableDefinitionSource"}},"type":"object"},"pipelines":{"description":"*(optional)* Collection of named pipelines","patternProperties":{"^[a-zA-Z0-9_]+$":{"$ref":"#/definitions/PipelineDefinition"}},"type":"object"},"producers":{"description":"*(optional)* Collection of named producers","patternProperties":{"^[a-zA-Z0-9_]+$":{"$ref":"#/definitions/ProducerDefinition"}},"type":"object"},"stores":{"description":"*(optional)* State stores that can be referenced in producers and pipelines","patternProperties":{"^[a-zA-Z0-9_]+$":{"anyOf":[{"$ref":"#/definitions/KeyValueStateStoreDefinition"},{"$ref":"#/definitions/SessionStateStoreDefinition"},{"$ref":"#/definitions/WindowStateStoreDefinition"}]}},"type":"object"},"streams":{"description":"*(optional)* Streams that can be referenced in producers and pipelines","patternProperties":{"^[a-zA-Z0-9_]+$":{"$ref":"#/definitions/StreamDefinitionSource"}},"type":"object"},"tables":{"description":"*(optional)* Tables that can be referenced in producers and pipelines","patternProperties":{"^[a-zA-Z0-9_]+$":{"$ref":"#/definitions/TableDefinitionSource"}},"type":"object"}},"title":"TopologyDefinition","type":"object"} +{"additionalProperties":false,"definitions":{"AggregateOperation":{"additionalProperties":false,"description":"An aggregate operation","properties":{"adder":{"anyOf":[{"type":"string"},{"$ref":"#/definitions/AggregatorDefinitionWithImplicitType"}],"description":"*(optional)* (GroupedTable) A function that adds a record to the aggregation result"},"aggregator":{"anyOf":[{"type":"string"},{"$ref":"#/definitions/AggregatorDefinitionWithImplicitType"}],"description":"*(optional)* (GroupedStream, SessionWindowedStream, TimeWindowedStream) The aggregator function, which combines a value with the previous aggregation result and outputs a new aggregation result"},"initializer":{"anyOf":[{"type":"string"},{"$ref":"#/definitions/InitializerDefinitionWithImplicitType"}],"description":"The initializer function, which generates an initial value for every set of aggregated records"},"merger":{"anyOf":[{"type":"string"},{"$ref":"#/definitions/MergerDefinitionWithImplicitType"}],"description":"*(optional)* (SessionWindowedStream, SessionWindowedCogroupedStream) A function that combines two aggregation results"},"name":{"description":"*(optional)* The name of the operation processor","type":"string"},"store":{"anyOf":[{"type":"string"},{"$ref":"#/definitions/WindowStateStoreDefinitionWithImplicitType"}],"description":"*(optional)* Materialized view of the result aggregation"},"subtractor":{"anyOf":[{"type":"string"},{"$ref":"#/definitions/AggregatorDefinitionWithImplicitType"}],"description":"*(optional)* (GroupedTable) A function that removes a record from the aggregation result"},"type":{"description":"The type of the operation","enum":["aggregate"]}},"required":["initializer","type"],"title":"AggregateOperation","type":"object"},"AggregatorDefinition":{"additionalProperties":false,"description":"Defines a aggregator function, that gets injected into the Kafka Streams topology","properties":{"code":{"anyOf":[{"type":"boolean"},{"type":"number"},{"type":"string"}],"description":"*(optional)* The (multiline) code of the aggregator"},"expression":{"anyOf":[{"type":"boolean"},{"type":"number"},{"type":"string"}],"description":"*(optional)* The expression returned by the aggregator. Only required for functions that return values."},"globalCode":{"anyOf":[{"type":"boolean"},{"type":"number"},{"type":"string"}],"description":"*(optional)* Global (multiline) code that gets loaded into the Python context outside of the aggregator. Can be used for defining eg. global variables."},"name":{"description":"*(optional)* The name of the aggregator. If this field is not defined, then the name is derived from the context.","type":"string"},"parameters":{"description":"*(optional)* A list of parameters to be passed into the aggregator","items":{"$ref":"#/definitions/ParameterDefinition"},"type":"array"},"resultType":{"description":"*(optional)* The data type returned by the aggregator. Only required for function types, which are not pre-defined.","type":"string"},"type":{"description":"The type of the function","enum":["aggregator"]}},"required":["type"],"title":"AggregatorDefinition","type":"object"},"AggregatorDefinitionWithImplicitType":{"additionalProperties":false,"description":"Defines a aggregator function, that gets injected into the Kafka Streams topology","properties":{"code":{"anyOf":[{"type":"boolean"},{"type":"number"},{"type":"string"}],"description":"*(optional)* The (multiline) code of the aggregator"},"expression":{"anyOf":[{"type":"boolean"},{"type":"number"},{"type":"string"}],"description":"*(optional)* The expression returned by the aggregator. Only required for functions that return values."},"globalCode":{"anyOf":[{"type":"boolean"},{"type":"number"},{"type":"string"}],"description":"*(optional)* Global (multiline) code that gets loaded into the Python context outside of the aggregator. Can be used for defining eg. global variables."},"name":{"description":"*(optional)* The name of the aggregator. If this field is not defined, then the name is derived from the context.","type":"string"},"parameters":{"description":"*(optional)* A list of parameters to be passed into the aggregator","items":{"$ref":"#/definitions/ParameterDefinition"},"type":"array"},"resultType":{"description":"*(optional)* The data type returned by the aggregator. Only required for function types, which are not pre-defined.","type":"string"}},"title":"AggregatorDefinitionWithImplicitType","type":"object"},"BranchDefinitionWithPipeline":{"additionalProperties":false,"description":"Defines a branch with sub-pipeline in a BranchOperation","properties":{"as":{"description":"*(optional)* The name to register the pipeline result under, which can be used as source by follow-up pipelines","type":"string"},"branch":{"description":"*(optional)* Defines a single branch, consisting of a condition and a pipeline to execute for messages that fulfil the predicate","items":{"$ref":"#/definitions/StringOrInlinePredicateDefinitionWithImplicitType"},"type":"array"},"forEach":{"anyOf":[{"type":"string"},{"$ref":"#/definitions/ForEachActionDefinitionWithImplicitType"}],"description":"*(optional)* A function that gets called for every message in the stream"},"if":{"anyOf":[{"type":"string"},{"$ref":"#/definitions/PredicateDefinitionWithImplicitType"}],"description":"*(optional)* Defines the condition under which messages get sent down this branch"},"name":{"description":"*(optional)* The name of the operation processor","type":"string"},"print":{"$ref":"#/definitions/PrintOperation","description":"*(optional)* The specification of where to print messages to"},"to":{"anyOf":[{"type":"string"},{"$ref":"#/definitions/ToTopicDefinition"}],"description":"*(optional)* Ends the pipeline by sending all messages to a stream, table or globalTable, or to an inline defined output topic and optional partitioner"},"toTopicNameExtractor":{"anyOf":[{"type":"string"},{"$ref":"#/definitions/ToTopicNameExtractorDefinition"}],"description":"*(optional)* Ends the pipeline by sending all messages to a topic provided by a pre-defined topic name extractor function, or to a topic provided by an inline defined topic name extractor and optional partitioner"},"via":{"description":"*(optional)* A series of operations performed on the input stream","items":{"anyOf":[{"$ref":"#/definitions/AggregateOperation"},{"$ref":"#/definitions/CogroupOperation"},{"$ref":"#/definitions/ConvertKeyOperation"},{"$ref":"#/definitions/ConvertKeyValueOperation"},{"$ref":"#/definitions/ConvertValueOperation"},{"$ref":"#/definitions/CountOperation"},{"$ref":"#/definitions/FilterNotOperation"},{"$ref":"#/definitions/FilterOperation"},{"$ref":"#/definitions/GroupByKeyOperation"},{"$ref":"#/definitions/GroupByOperation"},{"$ref":"#/definitions/JoinOperationWithGlobalTable"},{"$ref":"#/definitions/JoinOperationWithStream"},{"$ref":"#/definitions/JoinOperationWithTable"},{"$ref":"#/definitions/LeftJoinOperationWithGlobalTable"},{"$ref":"#/definitions/LeftJoinOperationWithStream"},{"$ref":"#/definitions/LeftJoinOperationWithTable"},{"$ref":"#/definitions/MergeOperation"},{"$ref":"#/definitions/OuterJoinOperationWithStream"},{"$ref":"#/definitions/OuterJoinOperationWithTable"},{"$ref":"#/definitions/PeekOperation"},{"$ref":"#/definitions/ReduceOperationWithAdderAndSubtractor"},{"$ref":"#/definitions/ReduceOperationWithReducer"},{"$ref":"#/definitions/RepartitionOperation"},{"$ref":"#/definitions/SuppressOperationUntilTimeLimit"},{"$ref":"#/definitions/SuppressOperationUntilWindowCloses"},{"$ref":"#/definitions/ToStreamOperation"},{"$ref":"#/definitions/ToTableOperation"},{"$ref":"#/definitions/TransformKeyOperation"},{"$ref":"#/definitions/TransformKeyValueOperation"},{"$ref":"#/definitions/TransformKeyValueToKeyValueListOperation"},{"$ref":"#/definitions/TransformKeyValueToValueListOperation"},{"$ref":"#/definitions/TransformMetadataOperation"},{"$ref":"#/definitions/TransformValueOperation"},{"$ref":"#/definitions/WindowBySessionOperation"},{"$ref":"#/definitions/WindowByTimeOperationWithHoppingWindow"},{"$ref":"#/definitions/WindowByTimeOperationWithSlidingWindow"},{"$ref":"#/definitions/WindowByTimeOperationWithTumblingWindow"}]},"type":"array"}},"title":"BranchDefinitionWithPipeline","type":"object"},"CogroupOperation":{"additionalProperties":false,"description":"A cogroup operation","properties":{"aggregator":{"anyOf":[{"type":"string"},{"$ref":"#/definitions/AggregatorDefinitionWithImplicitType"}],"description":"(GroupedStream, SessionWindowedStream, TimeWindowedStream) The aggregator function, which combines a value with the previous aggregation result and outputs a new aggregation result"},"name":{"description":"*(optional)* The name of the operation processor","type":"string"},"store":{"anyOf":[{"type":"string"},{"$ref":"#/definitions/WindowStateStoreDefinitionWithImplicitType"}],"description":"*(optional)* Materialized view of the co-grouped stream"},"type":{"description":"The type of the operation","enum":["cogroup"]}},"required":["aggregator","type"],"title":"CogroupOperation","type":"object"},"ConvertKeyOperation":{"additionalProperties":false,"description":"An operation to convert the stream key type to another type. Conversion is only syntactic, eg. from Avro to XML.","properties":{"into":{"description":"The type to convert the stream key into","type":"string"},"name":{"description":"*(optional)* The name of the operation processor","type":"string"},"type":{"description":"The type of the operation","enum":["convertKey"]}},"required":["into","type"],"title":"ConvertKeyOperation","type":"object"},"ConvertKeyValueOperation":{"additionalProperties":false,"description":"An operation to convert the stream key and value types to other types. Conversion is only syntactic, eg. from Avro to XML.","properties":{"into":{"description":"The tuple type to convert the stream key/value into","type":"string"},"name":{"description":"*(optional)* The name of the operation processor","type":"string"},"type":{"description":"The type of the operation","enum":["convertKeyValue"]}},"required":["into","type"],"title":"ConvertKeyValueOperation","type":"object"},"ConvertValueOperation":{"additionalProperties":false,"description":"An operation to convert the stream value type to another type. Conversion is only syntactic, eg. from Avro to XML.","properties":{"into":{"description":"The type to convert the stream value into","type":"string"},"name":{"description":"*(optional)* The name of the operation processor","type":"string"},"type":{"description":"The type of the operation","enum":["convertValue"]}},"required":["into","type"],"title":"ConvertValueOperation","type":"object"},"CountOperation":{"additionalProperties":false,"description":"Count the number of times a key is seen in a given window","properties":{"name":{"description":"*(optional)* The name of the operation processor","type":"string"},"store":{"anyOf":[{"type":"string"},{"$ref":"#/definitions/KeyValueStateStoreDefinition"},{"$ref":"#/definitions/SessionStateStoreDefinition"},{"$ref":"#/definitions/WindowStateStoreDefinition"}],"description":"*(optional)* Materialized view of the count operation's result"},"type":{"description":"The type of the operation","enum":["count"]}},"required":["type"],"title":"CountOperation","type":"object"},"FilterNotOperation":{"additionalProperties":false,"description":"Filter records based on the inverse result of a predicate function","properties":{"if":{"anyOf":[{"type":"string"},{"$ref":"#/definitions/PredicateDefinitionWithImplicitType"}],"description":"A function that returns \"false\" when records are accepted, \"true\" otherwise"},"name":{"description":"*(optional)* The name of the operation processor","type":"string"},"store":{"anyOf":[{"type":"string"},{"$ref":"#/definitions/KeyValueStateStoreDefinitionWithImplicitType"}],"description":"*(optional)* Materialized view of the filtered table (only applies to tables, ignored for streams)"},"type":{"description":"The type of the operation","enum":["filterNot"]}},"required":["if","type"],"title":"FilterNotOperation","type":"object"},"FilterOperation":{"additionalProperties":false,"description":"Filter records based on a predicate function","properties":{"if":{"anyOf":[{"type":"string"},{"$ref":"#/definitions/PredicateDefinitionWithImplicitType"}],"description":"A function that returns \"true\" when records are accepted, \"false\" otherwise"},"name":{"description":"*(optional)* The name of the operation processor","type":"string"},"store":{"anyOf":[{"type":"string"},{"$ref":"#/definitions/KeyValueStateStoreDefinitionWithImplicitType"}],"description":"*(optional)* Materialized view of the filtered table (only applies to tables, ignored for streams)"},"type":{"description":"The type of the operation","enum":["filter"]}},"required":["if","type"],"title":"FilterOperation","type":"object"},"ForEachActionDefinition":{"additionalProperties":false,"description":"Defines a foreach action function, that gets injected into the Kafka Streams topology","properties":{"code":{"anyOf":[{"type":"boolean"},{"type":"number"},{"type":"string"}],"description":"*(optional)* The (multiline) code of the foreach action"},"expression":{"anyOf":[{"type":"boolean"},{"type":"number"},{"type":"string"}],"description":"*(optional)* The expression returned by the foreach action. Only required for functions that return values."},"globalCode":{"anyOf":[{"type":"boolean"},{"type":"number"},{"type":"string"}],"description":"*(optional)* Global (multiline) code that gets loaded into the Python context outside of the foreach action. Can be used for defining eg. global variables."},"name":{"description":"*(optional)* The name of the foreach action. If this field is not defined, then the name is derived from the context.","type":"string"},"parameters":{"description":"*(optional)* A list of parameters to be passed into the foreach action","items":{"$ref":"#/definitions/ParameterDefinition"},"type":"array"},"resultType":{"description":"*(optional)* The data type returned by the foreach action. Only required for function types, which are not pre-defined.","type":"string"},"stores":{"description":"*(optional)* A list of store names that the foreach action uses. Only required if the function wants to use a state store.","items":{"type":"string"},"type":"array"},"type":{"description":"The type of the function","enum":["forEach"]}},"required":["type"],"title":"ForEachActionDefinition","type":"object"},"ForEachActionDefinitionWithImplicitType":{"additionalProperties":false,"description":"Defines a foreach action function, that gets injected into the Kafka Streams topology","properties":{"code":{"anyOf":[{"type":"boolean"},{"type":"number"},{"type":"string"}],"description":"*(optional)* The (multiline) code of the foreach action"},"expression":{"anyOf":[{"type":"boolean"},{"type":"number"},{"type":"string"}],"description":"*(optional)* The expression returned by the foreach action. Only required for functions that return values."},"globalCode":{"anyOf":[{"type":"boolean"},{"type":"number"},{"type":"string"}],"description":"*(optional)* Global (multiline) code that gets loaded into the Python context outside of the foreach action. Can be used for defining eg. global variables."},"name":{"description":"*(optional)* The name of the foreach action. If this field is not defined, then the name is derived from the context.","type":"string"},"parameters":{"description":"*(optional)* A list of parameters to be passed into the foreach action","items":{"$ref":"#/definitions/ParameterDefinition"},"type":"array"},"resultType":{"description":"*(optional)* The data type returned by the foreach action. Only required for function types, which are not pre-defined.","type":"string"},"stores":{"description":"*(optional)* A list of store names that the foreach action uses. Only required if the function wants to use a state store.","items":{"type":"string"},"type":"array"}},"title":"ForEachActionDefinitionWithImplicitType","type":"object"},"ForeignKeyExtractorDefinition":{"additionalProperties":false,"description":"Defines a foreign key extractor function, that gets injected into the Kafka Streams topology","properties":{"code":{"anyOf":[{"type":"boolean"},{"type":"number"},{"type":"string"}],"description":"*(optional)* The (multiline) code of the foreign key extractor"},"expression":{"anyOf":[{"type":"boolean"},{"type":"number"},{"type":"string"}],"description":"*(optional)* The expression returned by the foreign key extractor. Only required for functions that return values."},"globalCode":{"anyOf":[{"type":"boolean"},{"type":"number"},{"type":"string"}],"description":"*(optional)* Global (multiline) code that gets loaded into the Python context outside of the foreign key extractor. Can be used for defining eg. global variables."},"name":{"description":"*(optional)* The name of the foreign key extractor. If this field is not defined, then the name is derived from the context.","type":"string"},"parameters":{"description":"*(optional)* A list of parameters to be passed into the foreign key extractor","items":{"$ref":"#/definitions/ParameterDefinition"},"type":"array"},"resultType":{"description":"*(optional)* The data type returned by the foreign key extractor. Only required for function types, which are not pre-defined.","type":"string"},"type":{"description":"The type of the function","enum":["foreignKeyExtractor"]}},"required":["type"],"title":"ForeignKeyExtractorDefinition","type":"object"},"GeneratorDefinition":{"additionalProperties":false,"description":"Defines a message generator function, that gets injected into the Kafka Streams topology","properties":{"code":{"anyOf":[{"type":"boolean"},{"type":"number"},{"type":"string"}],"description":"*(optional)* The (multiline) code of the message generator"},"expression":{"anyOf":[{"type":"boolean"},{"type":"number"},{"type":"string"}],"description":"*(optional)* The expression returned by the message generator. Only required for functions that return values."},"globalCode":{"anyOf":[{"type":"boolean"},{"type":"number"},{"type":"string"}],"description":"*(optional)* Global (multiline) code that gets loaded into the Python context outside of the message generator. Can be used for defining eg. global variables."},"name":{"description":"*(optional)* The name of the message generator. If this field is not defined, then the name is derived from the context.","type":"string"},"parameters":{"description":"*(optional)* A list of parameters to be passed into the message generator","items":{"$ref":"#/definitions/ParameterDefinition"},"type":"array"},"resultType":{"description":"*(optional)* The data type returned by the message generator. Only required for function types, which are not pre-defined.","type":"string"},"type":{"description":"The type of the function","enum":["generator"]}},"required":["type"],"title":"GeneratorDefinition","type":"object"},"GeneratorDefinitionWithImplicitType":{"additionalProperties":false,"description":"Defines a message generator function, that gets injected into the Kafka Streams topology","properties":{"code":{"anyOf":[{"type":"boolean"},{"type":"number"},{"type":"string"}],"description":"*(optional)* The (multiline) code of the message generator"},"expression":{"anyOf":[{"type":"boolean"},{"type":"number"},{"type":"string"}],"description":"*(optional)* The expression returned by the message generator. Only required for functions that return values."},"globalCode":{"anyOf":[{"type":"boolean"},{"type":"number"},{"type":"string"}],"description":"*(optional)* Global (multiline) code that gets loaded into the Python context outside of the message generator. Can be used for defining eg. global variables."},"name":{"description":"*(optional)* The name of the message generator. If this field is not defined, then the name is derived from the context.","type":"string"},"parameters":{"description":"*(optional)* A list of parameters to be passed into the message generator","items":{"$ref":"#/definitions/ParameterDefinition"},"type":"array"},"resultType":{"description":"*(optional)* The data type returned by the message generator. Only required for function types, which are not pre-defined.","type":"string"}},"title":"GeneratorDefinitionWithImplicitType","type":"object"},"GenericFunctionDefinitionWithImplicitType":{"additionalProperties":false,"description":"Defines a generic function function, that gets injected into the Kafka Streams topology","properties":{"code":{"anyOf":[{"type":"boolean"},{"type":"number"},{"type":"string"}],"description":"*(optional)* The (multiline) code of the generic function"},"expression":{"anyOf":[{"type":"boolean"},{"type":"number"},{"type":"string"}],"description":"*(optional)* The expression returned by the generic function. Only required for functions that return values."},"globalCode":{"anyOf":[{"type":"boolean"},{"type":"number"},{"type":"string"}],"description":"*(optional)* Global (multiline) code that gets loaded into the Python context outside of the generic function. Can be used for defining eg. global variables."},"name":{"description":"*(optional)* The name of the generic function. If this field is not defined, then the name is derived from the context.","type":"string"},"parameters":{"description":"*(optional)* A list of parameters to be passed into the generic function","items":{"$ref":"#/definitions/ParameterDefinition"},"type":"array"},"resultType":{"description":"*(optional)* The data type returned by the generic function. Only required for function types, which are not pre-defined.","type":"string"},"type":{"description":"The type of the function","enum":["generic"]}},"title":"GenericFunctionDefinitionWithImplicitType","type":"object"},"GlobalTableDefinition":{"additionalProperties":false,"description":"Contains a definition of a GlobalTable, which can be referenced by producers and pipelines","properties":{"keyType":{"description":"*(optional)* The key type of the global table","type":"string"},"store":{"anyOf":[{"type":"string"},{"$ref":"#/definitions/KeyValueStateStoreDefinitionWithImplicitType"}],"description":"*(optional)* KeyValue state store definition"},"topic":{"description":"The name of the Kafka topic for this global table","type":"string"},"valueType":{"description":"*(optional)* The value type of the global table","type":"string"}},"required":["topic"],"title":"GlobalTableDefinition","type":"object"},"GlobalTableDefinitionSource":{"additionalProperties":false,"description":"Contains a definition of a GlobalTable, which can be referenced by producers and pipelines","properties":{"keyType":{"description":"The key type of the global table","type":"string"},"offsetResetPolicy":{"description":"*(optional)* Policy that determines what to do when there is no initial offset in Kafka, or if the current offset does not exist any more on the server (e.g. because that data has been deleted)","type":"string"},"store":{"anyOf":[{"type":"string"},{"$ref":"#/definitions/KeyValueStateStoreDefinitionWithImplicitType"}],"description":"*(optional)* KeyValue state store definition"},"timestampExtractor":{"anyOf":[{"type":"string"},{"$ref":"#/definitions/TimestampExtractorDefinitionWithImplicitType"}],"description":"*(optional)* A function extracts the event time from a consumed record"},"topic":{"description":"The name of the Kafka topic for this global table","type":"string"},"valueType":{"description":"The value type of the global table","type":"string"}},"required":["topic","keyType","valueType"],"title":"GlobalTableDefinitionSource","type":"object"},"GroupByKeyOperation":{"additionalProperties":false,"description":"Operation to group all messages with the same key together","properties":{"name":{"description":"*(optional)* The name of the operation processor","type":"string"},"store":{"anyOf":[{"type":"string"},{"$ref":"#/definitions/KeyValueStateStoreDefinitionWithImplicitType"}],"description":"*(optional)* Materialized view of the grouped stream"},"type":{"description":"The type of the operation","enum":["groupByKey"]}},"required":["type"],"title":"GroupByKeyOperation","type":"object"},"GroupByOperation":{"additionalProperties":false,"description":"Operation to group all messages with together based on a keying function","properties":{"mapper":{"anyOf":[{"type":"string"},{"$ref":"#/definitions/KeyValueMapperDefinitionWithImplicitType"}],"description":"Function to map records to a key they can be grouped on"},"name":{"description":"*(optional)* The name of the operation processor","type":"string"},"store":{"anyOf":[{"type":"string"},{"$ref":"#/definitions/KeyValueStateStoreDefinitionWithImplicitType"}],"description":"*(optional)* Materialized view of the grouped stream or table"},"type":{"description":"The type of the operation","enum":["groupBy"]}},"required":["mapper","type"],"title":"GroupByOperation","type":"object"},"InitializerDefinition":{"additionalProperties":false,"description":"Defines a initializer function, that gets injected into the Kafka Streams topology","properties":{"code":{"anyOf":[{"type":"boolean"},{"type":"number"},{"type":"string"}],"description":"*(optional)* The (multiline) code of the initializer"},"expression":{"anyOf":[{"type":"boolean"},{"type":"number"},{"type":"string"}],"description":"*(optional)* The expression returned by the initializer. Only required for functions that return values."},"globalCode":{"anyOf":[{"type":"boolean"},{"type":"number"},{"type":"string"}],"description":"*(optional)* Global (multiline) code that gets loaded into the Python context outside of the initializer. Can be used for defining eg. global variables."},"name":{"description":"*(optional)* The name of the initializer. If this field is not defined, then the name is derived from the context.","type":"string"},"parameters":{"description":"*(optional)* A list of parameters to be passed into the initializer","items":{"$ref":"#/definitions/ParameterDefinition"},"type":"array"},"resultType":{"description":"*(optional)* The data type returned by the initializer. Only required for function types, which are not pre-defined.","type":"string"},"type":{"description":"The type of the function","enum":["initializer"]}},"required":["type"],"title":"InitializerDefinition","type":"object"},"InitializerDefinitionWithImplicitType":{"additionalProperties":false,"description":"Defines a initializer function, that gets injected into the Kafka Streams topology","properties":{"code":{"anyOf":[{"type":"boolean"},{"type":"number"},{"type":"string"}],"description":"*(optional)* The (multiline) code of the initializer"},"expression":{"anyOf":[{"type":"boolean"},{"type":"number"},{"type":"string"}],"description":"*(optional)* The expression returned by the initializer. Only required for functions that return values."},"globalCode":{"anyOf":[{"type":"boolean"},{"type":"number"},{"type":"string"}],"description":"*(optional)* Global (multiline) code that gets loaded into the Python context outside of the initializer. Can be used for defining eg. global variables."},"name":{"description":"*(optional)* The name of the initializer. If this field is not defined, then the name is derived from the context.","type":"string"},"parameters":{"description":"*(optional)* A list of parameters to be passed into the initializer","items":{"$ref":"#/definitions/ParameterDefinition"},"type":"array"},"resultType":{"description":"*(optional)* The data type returned by the initializer. Only required for function types, which are not pre-defined.","type":"string"}},"title":"InitializerDefinitionWithImplicitType","type":"object"},"JoinOperationWithGlobalTable":{"additionalProperties":false,"description":"Operation to join with a table","properties":{"globalTable":{"anyOf":[{"type":"string"},{"$ref":"#/definitions/GlobalTableDefinition"}],"description":"A reference to the globalTable, or an inline definition of the globalTable to join with"},"mapper":{"anyOf":[{"type":"string"},{"$ref":"#/definitions/ValueJoinerDefinitionWithImplicitType"}],"description":"A function that maps the key value from the stream to the primary key type of the globalTable"},"name":{"description":"*(optional)* The name of the operation processor","type":"string"},"type":{"description":"The type of the operation","enum":["join"]},"valueJoiner":{"anyOf":[{"type":"string"},{"$ref":"#/definitions/ValueJoinerDefinitionWithImplicitType"}],"description":"A function that joins two values"}},"required":["globalTable","mapper","valueJoiner","type"],"title":"JoinOperationWithGlobalTable","type":"object"},"JoinOperationWithStream":{"additionalProperties":false,"description":"Operation to join with a stream","properties":{"grace":{"anyOf":[{"type":"number"},{"type":"string"}],"description":"*(optional)* The window grace period (the time to admit out-of-order events after the end of the window)"},"name":{"description":"*(optional)* The name of the operation processor","type":"string"},"store":{"anyOf":[{"type":"string"},{"$ref":"#/definitions/KeyValueStateStoreDefinition"},{"$ref":"#/definitions/SessionStateStoreDefinition"},{"$ref":"#/definitions/WindowStateStoreDefinition"}],"description":"*(optional)* Materialized view of the joined streams"},"stream":{"anyOf":[{"type":"string"},{"$ref":"#/definitions/StreamDefinition"}],"description":"A reference to the Stream, or an inline definition of the stream to join with"},"timeDifference":{"anyOf":[{"type":"number"},{"type":"string"}],"description":"The maximum time difference for a join over two streams on the same key"},"type":{"description":"The type of the operation","enum":["join"]},"valueJoiner":{"anyOf":[{"type":"string"},{"$ref":"#/definitions/ValueJoinerDefinitionWithImplicitType"}],"description":"A function that joins two values"}},"required":["stream","valueJoiner","timeDifference","type"],"title":"JoinOperationWithStream","type":"object"},"JoinOperationWithTable":{"additionalProperties":false,"description":"Operation to join with a table","properties":{"foreignKeyExtractor":{"anyOf":[{"type":"string"},{"$ref":"#/definitions/ValueJoinerDefinitionWithImplicitType"}],"description":"*(optional)* A function that can translate the join table value to a primary key"},"grace":{"anyOf":[{"type":"number"},{"type":"string"}],"description":"*(optional)* The window grace period (the time to admit out-of-order events after the end of the window)"},"name":{"description":"*(optional)* The name of the operation processor","type":"string"},"otherPartitioner":{"anyOf":[{"type":"string"},{"$ref":"#/definitions/StreamPartitionerDefinitionWithImplicitType"}],"description":"*(optional)* A function that partitions the records on the join table"},"partitioner":{"anyOf":[{"type":"string"},{"$ref":"#/definitions/StreamPartitionerDefinitionWithImplicitType"}],"description":"*(optional)* A function that partitions the records on the primary table"},"store":{"anyOf":[{"type":"string"},{"$ref":"#/definitions/KeyValueStateStoreDefinition"},{"$ref":"#/definitions/SessionStateStoreDefinition"},{"$ref":"#/definitions/WindowStateStoreDefinition"}],"description":"*(optional)* Materialized view of the joined streams"},"table":{"anyOf":[{"type":"string"},{"$ref":"#/definitions/TableDefinition"}],"description":"A reference to the table, or an inline definition of the table to join with"},"type":{"description":"The type of the operation","enum":["join"]},"valueJoiner":{"anyOf":[{"type":"string"},{"$ref":"#/definitions/ValueJoinerDefinitionWithImplicitType"}],"description":"A function that joins two values"}},"required":["table","valueJoiner","type"],"title":"JoinOperationWithTable","type":"object"},"KeyTransformerDefinition":{"additionalProperties":false,"description":"Defines a key transformer function, that gets injected into the Kafka Streams topology","properties":{"code":{"anyOf":[{"type":"boolean"},{"type":"number"},{"type":"string"}],"description":"*(optional)* The (multiline) code of the key transformer"},"expression":{"anyOf":[{"type":"boolean"},{"type":"number"},{"type":"string"}],"description":"*(optional)* The expression returned by the key transformer. Only required for functions that return values."},"globalCode":{"anyOf":[{"type":"boolean"},{"type":"number"},{"type":"string"}],"description":"*(optional)* Global (multiline) code that gets loaded into the Python context outside of the key transformer. Can be used for defining eg. global variables."},"name":{"description":"*(optional)* The name of the key transformer. If this field is not defined, then the name is derived from the context.","type":"string"},"parameters":{"description":"*(optional)* A list of parameters to be passed into the key transformer","items":{"$ref":"#/definitions/ParameterDefinition"},"type":"array"},"resultType":{"description":"*(optional)* The data type returned by the key transformer. Only required for function types, which are not pre-defined.","type":"string"},"stores":{"description":"*(optional)* A list of store names that the key transformer uses. Only required if the function wants to use a state store.","items":{"type":"string"},"type":"array"},"type":{"description":"The type of the function","enum":["keyTransformer"]}},"required":["type"],"title":"KeyTransformerDefinition","type":"object"},"KeyTransformerDefinitionWithImplicitType":{"additionalProperties":false,"description":"Defines a key transformer function, that gets injected into the Kafka Streams topology","properties":{"code":{"anyOf":[{"type":"boolean"},{"type":"number"},{"type":"string"}],"description":"*(optional)* The (multiline) code of the key transformer"},"expression":{"anyOf":[{"type":"boolean"},{"type":"number"},{"type":"string"}],"description":"*(optional)* The expression returned by the key transformer. Only required for functions that return values."},"globalCode":{"anyOf":[{"type":"boolean"},{"type":"number"},{"type":"string"}],"description":"*(optional)* Global (multiline) code that gets loaded into the Python context outside of the key transformer. Can be used for defining eg. global variables."},"name":{"description":"*(optional)* The name of the key transformer. If this field is not defined, then the name is derived from the context.","type":"string"},"parameters":{"description":"*(optional)* A list of parameters to be passed into the key transformer","items":{"$ref":"#/definitions/ParameterDefinition"},"type":"array"},"resultType":{"description":"*(optional)* The data type returned by the key transformer. Only required for function types, which are not pre-defined.","type":"string"},"stores":{"description":"*(optional)* A list of store names that the key transformer uses. Only required if the function wants to use a state store.","items":{"type":"string"},"type":"array"}},"title":"KeyTransformerDefinitionWithImplicitType","type":"object"},"KeyValueMapperDefinition":{"additionalProperties":false,"description":"Defines a keyvalue mapper function, that gets injected into the Kafka Streams topology","properties":{"code":{"anyOf":[{"type":"boolean"},{"type":"number"},{"type":"string"}],"description":"*(optional)* The (multiline) code of the keyvalue mapper"},"expression":{"anyOf":[{"type":"boolean"},{"type":"number"},{"type":"string"}],"description":"*(optional)* The expression returned by the keyvalue mapper. Only required for functions that return values."},"globalCode":{"anyOf":[{"type":"boolean"},{"type":"number"},{"type":"string"}],"description":"*(optional)* Global (multiline) code that gets loaded into the Python context outside of the keyvalue mapper. Can be used for defining eg. global variables."},"name":{"description":"*(optional)* The name of the keyvalue mapper. If this field is not defined, then the name is derived from the context.","type":"string"},"parameters":{"description":"*(optional)* A list of parameters to be passed into the keyvalue mapper","items":{"$ref":"#/definitions/ParameterDefinition"},"type":"array"},"resultType":{"description":"*(optional)* The data type returned by the keyvalue mapper. Only required for function types, which are not pre-defined.","type":"string"},"type":{"description":"The type of the function","enum":["keyValueMapper"]}},"required":["type"],"title":"KeyValueMapperDefinition","type":"object"},"KeyValueMapperDefinitionWithImplicitType":{"additionalProperties":false,"description":"Defines a keyvalue mapper function, that gets injected into the Kafka Streams topology","properties":{"code":{"anyOf":[{"type":"boolean"},{"type":"number"},{"type":"string"}],"description":"*(optional)* The (multiline) code of the keyvalue mapper"},"expression":{"anyOf":[{"type":"boolean"},{"type":"number"},{"type":"string"}],"description":"*(optional)* The expression returned by the keyvalue mapper. Only required for functions that return values."},"globalCode":{"anyOf":[{"type":"boolean"},{"type":"number"},{"type":"string"}],"description":"*(optional)* Global (multiline) code that gets loaded into the Python context outside of the keyvalue mapper. Can be used for defining eg. global variables."},"name":{"description":"*(optional)* The name of the keyvalue mapper. If this field is not defined, then the name is derived from the context.","type":"string"},"parameters":{"description":"*(optional)* A list of parameters to be passed into the keyvalue mapper","items":{"$ref":"#/definitions/ParameterDefinition"},"type":"array"},"resultType":{"description":"*(optional)* The data type returned by the keyvalue mapper. Only required for function types, which are not pre-defined.","type":"string"}},"title":"KeyValueMapperDefinitionWithImplicitType","type":"object"},"KeyValuePrinterDefinition":{"additionalProperties":false,"description":"Defines a keyvalue printer function, that gets injected into the Kafka Streams topology","properties":{"code":{"anyOf":[{"type":"boolean"},{"type":"number"},{"type":"string"}],"description":"*(optional)* The (multiline) code of the keyvalue printer"},"expression":{"anyOf":[{"type":"boolean"},{"type":"number"},{"type":"string"}],"description":"*(optional)* The expression returned by the keyvalue printer. Only required for functions that return values."},"globalCode":{"anyOf":[{"type":"boolean"},{"type":"number"},{"type":"string"}],"description":"*(optional)* Global (multiline) code that gets loaded into the Python context outside of the keyvalue printer. Can be used for defining eg. global variables."},"name":{"description":"*(optional)* The name of the keyvalue printer. If this field is not defined, then the name is derived from the context.","type":"string"},"parameters":{"description":"*(optional)* A list of parameters to be passed into the keyvalue printer","items":{"$ref":"#/definitions/ParameterDefinition"},"type":"array"},"resultType":{"description":"*(optional)* The data type returned by the keyvalue printer. Only required for function types, which are not pre-defined.","type":"string"},"type":{"description":"The type of the function","enum":["keyValuePrinter"]}},"required":["type"],"title":"KeyValuePrinterDefinition","type":"object"},"KeyValuePrinterDefinitionWithImplicitType":{"additionalProperties":false,"description":"Defines a keyvalue printer function, that gets injected into the Kafka Streams topology","properties":{"code":{"anyOf":[{"type":"boolean"},{"type":"number"},{"type":"string"}],"description":"*(optional)* The (multiline) code of the keyvalue printer"},"expression":{"anyOf":[{"type":"boolean"},{"type":"number"},{"type":"string"}],"description":"*(optional)* The expression returned by the keyvalue printer. Only required for functions that return values."},"globalCode":{"anyOf":[{"type":"boolean"},{"type":"number"},{"type":"string"}],"description":"*(optional)* Global (multiline) code that gets loaded into the Python context outside of the keyvalue printer. Can be used for defining eg. global variables."},"name":{"description":"*(optional)* The name of the keyvalue printer. If this field is not defined, then the name is derived from the context.","type":"string"},"parameters":{"description":"*(optional)* A list of parameters to be passed into the keyvalue printer","items":{"$ref":"#/definitions/ParameterDefinition"},"type":"array"},"resultType":{"description":"*(optional)* The data type returned by the keyvalue printer. Only required for function types, which are not pre-defined.","type":"string"}},"title":"KeyValuePrinterDefinitionWithImplicitType","type":"object"},"KeyValueStateStoreDefinition":{"additionalProperties":false,"description":"Definition of a keyValue state store","properties":{"caching":{"description":"*(optional)* \"true\" if changed to the keyValue store need to be buffered and periodically released, \"false\" to emit all changes directly","type":"boolean"},"historyRetention":{"anyOf":[{"type":"number"},{"type":"string"}],"description":"*(optional)* (Versioned only) The duration for which old record versions are available for query (cannot be negative)"},"keyType":{"description":"*(optional)* The key type of the keyValue store","type":"string"},"logging":{"description":"*(optional)* \"true\" if a changelog topic should be set up on Kafka for this keyValue store, \"false\" otherwise","type":"boolean"},"name":{"description":"*(optional)* The name of the keyValue store. If this field is not defined, then the name is derived from the context.","type":"string"},"persistent":{"description":"*(optional)* \"true\" if this keyValue store needs to be stored on disk, \"false\" otherwise","type":"boolean"},"segmentInterval":{"anyOf":[{"type":"number"},{"type":"string"}],"description":"*(optional)* Size of segments for storing old record versions (must be positive). Old record versions for the same key in a single segment are stored (updated and accessed) together. The only impact of this parameter is performance. If segments are large and a workload results in many record versions for the same key being collected in a single segment, performance may degrade as a result. On the other hand, historical reads (which access older segments) and out-of-order writes may slow down if there are too many segments."},"timestamped":{"description":"*(optional)* \"true\" if elements in the store are timestamped, \"false\" otherwise","type":"boolean"},"type":{"description":"The type of the state store","enum":["keyValue"]},"valueType":{"description":"*(optional)* The value type of the keyValue store","type":"string"},"versioned":{"description":"*(optional)* \"true\" if elements in the store are versioned, \"false\" otherwise","type":"boolean"}},"required":["type"],"title":"KeyValueStateStoreDefinition","type":"object"},"KeyValueStateStoreDefinitionWithImplicitType":{"additionalProperties":false,"description":"Definition of a keyValue state store","properties":{"caching":{"description":"*(optional)* \"true\" if changed to the keyValue store need to be buffered and periodically released, \"false\" to emit all changes directly","type":"boolean"},"historyRetention":{"anyOf":[{"type":"number"},{"type":"string"}],"description":"*(optional)* (Versioned only) The duration for which old record versions are available for query (cannot be negative)"},"keyType":{"description":"*(optional)* The key type of the keyValue store","type":"string"},"logging":{"description":"*(optional)* \"true\" if a changelog topic should be set up on Kafka for this keyValue store, \"false\" otherwise","type":"boolean"},"name":{"description":"*(optional)* The name of the keyValue store. If this field is not defined, then the name is derived from the context.","type":"string"},"persistent":{"description":"*(optional)* \"true\" if this keyValue store needs to be stored on disk, \"false\" otherwise","type":"boolean"},"segmentInterval":{"anyOf":[{"type":"number"},{"type":"string"}],"description":"*(optional)* Size of segments for storing old record versions (must be positive). Old record versions for the same key in a single segment are stored (updated and accessed) together. The only impact of this parameter is performance. If segments are large and a workload results in many record versions for the same key being collected in a single segment, performance may degrade as a result. On the other hand, historical reads (which access older segments) and out-of-order writes may slow down if there are too many segments."},"timestamped":{"description":"*(optional)* \"true\" if elements in the store are timestamped, \"false\" otherwise","type":"boolean"},"type":{"description":"The type of the state store","enum":["keyValue"]},"valueType":{"description":"*(optional)* The value type of the keyValue store","type":"string"},"versioned":{"description":"*(optional)* \"true\" if elements in the store are versioned, \"false\" otherwise","type":"boolean"}},"title":"KeyValueStateStoreDefinitionWithImplicitType","type":"object"},"KeyValueToKeyValueListTransformerDefinition":{"additionalProperties":false,"description":"Defines a keyvalue-to-keyvaluelist transformer function, that gets injected into the Kafka Streams topology","properties":{"code":{"anyOf":[{"type":"boolean"},{"type":"number"},{"type":"string"}],"description":"*(optional)* The (multiline) code of the keyvalue-to-keyvaluelist transformer"},"expression":{"anyOf":[{"type":"boolean"},{"type":"number"},{"type":"string"}],"description":"*(optional)* The expression returned by the keyvalue-to-keyvaluelist transformer. Only required for functions that return values."},"globalCode":{"anyOf":[{"type":"boolean"},{"type":"number"},{"type":"string"}],"description":"*(optional)* Global (multiline) code that gets loaded into the Python context outside of the keyvalue-to-keyvaluelist transformer. Can be used for defining eg. global variables."},"name":{"description":"*(optional)* The name of the keyvalue-to-keyvaluelist transformer. If this field is not defined, then the name is derived from the context.","type":"string"},"parameters":{"description":"*(optional)* A list of parameters to be passed into the keyvalue-to-keyvaluelist transformer","items":{"$ref":"#/definitions/ParameterDefinition"},"type":"array"},"resultType":{"description":"*(optional)* The data type returned by the keyvalue-to-keyvaluelist transformer. Only required for function types, which are not pre-defined.","type":"string"},"stores":{"description":"*(optional)* A list of store names that the keyvalue-to-keyvaluelist transformer uses. Only required if the function wants to use a state store.","items":{"type":"string"},"type":"array"},"type":{"description":"The type of the function","enum":["keyValueToKeyValueListTransformer"]}},"required":["type"],"title":"KeyValueToKeyValueListTransformerDefinition","type":"object"},"KeyValueToKeyValueListTransformerDefinitionWithImplicitType":{"additionalProperties":false,"description":"Defines a keyvalue-to-keyvaluelist transformer function, that gets injected into the Kafka Streams topology","properties":{"code":{"anyOf":[{"type":"boolean"},{"type":"number"},{"type":"string"}],"description":"*(optional)* The (multiline) code of the keyvalue-to-keyvaluelist transformer"},"expression":{"anyOf":[{"type":"boolean"},{"type":"number"},{"type":"string"}],"description":"*(optional)* The expression returned by the keyvalue-to-keyvaluelist transformer. Only required for functions that return values."},"globalCode":{"anyOf":[{"type":"boolean"},{"type":"number"},{"type":"string"}],"description":"*(optional)* Global (multiline) code that gets loaded into the Python context outside of the keyvalue-to-keyvaluelist transformer. Can be used for defining eg. global variables."},"name":{"description":"*(optional)* The name of the keyvalue-to-keyvaluelist transformer. If this field is not defined, then the name is derived from the context.","type":"string"},"parameters":{"description":"*(optional)* A list of parameters to be passed into the keyvalue-to-keyvaluelist transformer","items":{"$ref":"#/definitions/ParameterDefinition"},"type":"array"},"resultType":{"description":"*(optional)* The data type returned by the keyvalue-to-keyvaluelist transformer. Only required for function types, which are not pre-defined.","type":"string"},"stores":{"description":"*(optional)* A list of store names that the keyvalue-to-keyvaluelist transformer uses. Only required if the function wants to use a state store.","items":{"type":"string"},"type":"array"}},"title":"KeyValueToKeyValueListTransformerDefinitionWithImplicitType","type":"object"},"KeyValueToValueListTransformerDefinition":{"additionalProperties":false,"description":"Defines a keyvalue-to-valuelist transformer function, that gets injected into the Kafka Streams topology","properties":{"code":{"anyOf":[{"type":"boolean"},{"type":"number"},{"type":"string"}],"description":"*(optional)* The (multiline) code of the keyvalue-to-valuelist transformer"},"expression":{"anyOf":[{"type":"boolean"},{"type":"number"},{"type":"string"}],"description":"*(optional)* The expression returned by the keyvalue-to-valuelist transformer. Only required for functions that return values."},"globalCode":{"anyOf":[{"type":"boolean"},{"type":"number"},{"type":"string"}],"description":"*(optional)* Global (multiline) code that gets loaded into the Python context outside of the keyvalue-to-valuelist transformer. Can be used for defining eg. global variables."},"name":{"description":"*(optional)* The name of the keyvalue-to-valuelist transformer. If this field is not defined, then the name is derived from the context.","type":"string"},"parameters":{"description":"*(optional)* A list of parameters to be passed into the keyvalue-to-valuelist transformer","items":{"$ref":"#/definitions/ParameterDefinition"},"type":"array"},"resultType":{"description":"*(optional)* The data type returned by the keyvalue-to-valuelist transformer. Only required for function types, which are not pre-defined.","type":"string"},"stores":{"description":"*(optional)* A list of store names that the keyvalue-to-valuelist transformer uses. Only required if the function wants to use a state store.","items":{"type":"string"},"type":"array"},"type":{"description":"The type of the function","enum":["keyValueToValueListTransformer"]}},"required":["type"],"title":"KeyValueToValueListTransformerDefinition","type":"object"},"KeyValueToValueListTransformerDefinitionWithImplicitType":{"additionalProperties":false,"description":"Defines a keyvalue-to-valuelist transformer function, that gets injected into the Kafka Streams topology","properties":{"code":{"anyOf":[{"type":"boolean"},{"type":"number"},{"type":"string"}],"description":"*(optional)* The (multiline) code of the keyvalue-to-valuelist transformer"},"expression":{"anyOf":[{"type":"boolean"},{"type":"number"},{"type":"string"}],"description":"*(optional)* The expression returned by the keyvalue-to-valuelist transformer. Only required for functions that return values."},"globalCode":{"anyOf":[{"type":"boolean"},{"type":"number"},{"type":"string"}],"description":"*(optional)* Global (multiline) code that gets loaded into the Python context outside of the keyvalue-to-valuelist transformer. Can be used for defining eg. global variables."},"name":{"description":"*(optional)* The name of the keyvalue-to-valuelist transformer. If this field is not defined, then the name is derived from the context.","type":"string"},"parameters":{"description":"*(optional)* A list of parameters to be passed into the keyvalue-to-valuelist transformer","items":{"$ref":"#/definitions/ParameterDefinition"},"type":"array"},"resultType":{"description":"*(optional)* The data type returned by the keyvalue-to-valuelist transformer. Only required for function types, which are not pre-defined.","type":"string"},"stores":{"description":"*(optional)* A list of store names that the keyvalue-to-valuelist transformer uses. Only required if the function wants to use a state store.","items":{"type":"string"},"type":"array"}},"title":"KeyValueToValueListTransformerDefinitionWithImplicitType","type":"object"},"KeyValueTransformerDefinition":{"additionalProperties":false,"description":"Defines a keyvalue transformer function, that gets injected into the Kafka Streams topology","properties":{"code":{"anyOf":[{"type":"boolean"},{"type":"number"},{"type":"string"}],"description":"*(optional)* The (multiline) code of the keyvalue transformer"},"expression":{"anyOf":[{"type":"boolean"},{"type":"number"},{"type":"string"}],"description":"*(optional)* The expression returned by the keyvalue transformer. Only required for functions that return values."},"globalCode":{"anyOf":[{"type":"boolean"},{"type":"number"},{"type":"string"}],"description":"*(optional)* Global (multiline) code that gets loaded into the Python context outside of the keyvalue transformer. Can be used for defining eg. global variables."},"name":{"description":"*(optional)* The name of the keyvalue transformer. If this field is not defined, then the name is derived from the context.","type":"string"},"parameters":{"description":"*(optional)* A list of parameters to be passed into the keyvalue transformer","items":{"$ref":"#/definitions/ParameterDefinition"},"type":"array"},"resultType":{"description":"*(optional)* The data type returned by the keyvalue transformer. Only required for function types, which are not pre-defined.","type":"string"},"stores":{"description":"*(optional)* A list of store names that the keyvalue transformer uses. Only required if the function wants to use a state store.","items":{"type":"string"},"type":"array"},"type":{"description":"The type of the function","enum":["keyValueTransformer"]}},"required":["type"],"title":"KeyValueTransformerDefinition","type":"object"},"KeyValueTransformerDefinitionWithImplicitType":{"additionalProperties":false,"description":"Defines a keyvalue transformer function, that gets injected into the Kafka Streams topology","properties":{"code":{"anyOf":[{"type":"boolean"},{"type":"number"},{"type":"string"}],"description":"*(optional)* The (multiline) code of the keyvalue transformer"},"expression":{"anyOf":[{"type":"boolean"},{"type":"number"},{"type":"string"}],"description":"*(optional)* The expression returned by the keyvalue transformer. Only required for functions that return values."},"globalCode":{"anyOf":[{"type":"boolean"},{"type":"number"},{"type":"string"}],"description":"*(optional)* Global (multiline) code that gets loaded into the Python context outside of the keyvalue transformer. Can be used for defining eg. global variables."},"name":{"description":"*(optional)* The name of the keyvalue transformer. If this field is not defined, then the name is derived from the context.","type":"string"},"parameters":{"description":"*(optional)* A list of parameters to be passed into the keyvalue transformer","items":{"$ref":"#/definitions/ParameterDefinition"},"type":"array"},"resultType":{"description":"*(optional)* The data type returned by the keyvalue transformer. Only required for function types, which are not pre-defined.","type":"string"},"stores":{"description":"*(optional)* A list of store names that the keyvalue transformer uses. Only required if the function wants to use a state store.","items":{"type":"string"},"type":"array"}},"title":"KeyValueTransformerDefinitionWithImplicitType","type":"object"},"LeftJoinOperationWithGlobalTable":{"additionalProperties":false,"description":"Operation to leftJoin with a globalTable","properties":{"globalTable":{"anyOf":[{"type":"string"},{"$ref":"#/definitions/GlobalTableDefinition"}],"description":"A reference to the globalTable, or an inline definition of the globalTable to join with"},"mapper":{"anyOf":[{"type":"string"},{"$ref":"#/definitions/ValueJoinerDefinitionWithImplicitType"}],"description":"A function that maps the key value from the stream with the primary key of the globalTable"},"name":{"description":"*(optional)* The name of the operation processor","type":"string"},"type":{"description":"The type of the operation","enum":["leftJoin"]},"valueJoiner":{"anyOf":[{"type":"string"},{"$ref":"#/definitions/ValueJoinerDefinitionWithImplicitType"}],"description":"A function that joins two values"}},"required":["globalTable","mapper","valueJoiner","type"],"title":"LeftJoinOperationWithGlobalTable","type":"object"},"LeftJoinOperationWithStream":{"additionalProperties":false,"description":"Operation to leftJoin with a stream","properties":{"grace":{"anyOf":[{"type":"number"},{"type":"string"}],"description":"*(optional)* The window grace period (the time to admit out-of-order events after the end of the window)"},"name":{"description":"*(optional)* The name of the operation processor","type":"string"},"store":{"anyOf":[{"type":"string"},{"$ref":"#/definitions/KeyValueStateStoreDefinition"},{"$ref":"#/definitions/SessionStateStoreDefinition"},{"$ref":"#/definitions/WindowStateStoreDefinition"}],"description":"*(optional)* Materialized view of the joined streams"},"stream":{"anyOf":[{"type":"string"},{"$ref":"#/definitions/StreamDefinition"}],"description":"A reference to the stream, or an inline definition of the stream to leftJoin with"},"timeDifference":{"anyOf":[{"type":"number"},{"type":"string"}],"description":"The maximum time difference for a leftJoin over two streams on the same key"},"type":{"description":"The type of the operation","enum":["leftJoin"]},"valueJoiner":{"anyOf":[{"type":"string"},{"$ref":"#/definitions/ValueJoinerDefinitionWithImplicitType"}],"description":"A function that joins two values"}},"required":["stream","valueJoiner","timeDifference","type"],"title":"LeftJoinOperationWithStream","type":"object"},"LeftJoinOperationWithTable":{"additionalProperties":false,"description":"Operation to leftJoin with a table","properties":{"foreignKeyExtractor":{"anyOf":[{"type":"string"},{"$ref":"#/definitions/ValueJoinerDefinitionWithImplicitType"}],"description":"*(optional)* A function that can translate the join table value to a primary key"},"grace":{"anyOf":[{"type":"number"},{"type":"string"}],"description":"*(optional)* The window grace period (the time to admit out-of-order events after the end of the window)"},"name":{"description":"*(optional)* The name of the operation processor","type":"string"},"otherPartitioner":{"anyOf":[{"type":"string"},{"$ref":"#/definitions/StreamPartitionerDefinitionWithImplicitType"}],"description":"*(optional)* A function that partitions the records on the join table"},"partitioner":{"anyOf":[{"type":"string"},{"$ref":"#/definitions/StreamPartitionerDefinitionWithImplicitType"}],"description":"*(optional)* A function that partitions the records on the primary table"},"store":{"anyOf":[{"type":"string"},{"$ref":"#/definitions/KeyValueStateStoreDefinition"},{"$ref":"#/definitions/SessionStateStoreDefinition"},{"$ref":"#/definitions/WindowStateStoreDefinition"}],"description":"*(optional)* Materialized view of the joined streams"},"table":{"anyOf":[{"type":"string"},{"$ref":"#/definitions/TableDefinition"}],"description":"A reference to the Table, or an inline definition of the Table to join with"},"type":{"description":"The type of the operation","enum":["leftJoin"]},"valueJoiner":{"anyOf":[{"type":"string"},{"$ref":"#/definitions/ValueJoinerDefinitionWithImplicitType"}],"description":"A function that joins two values"}},"required":["table","valueJoiner","type"],"title":"LeftJoinOperationWithTable","type":"object"},"MergeOperation":{"additionalProperties":false,"description":"A merge operation to join two Streams","properties":{"name":{"description":"*(optional)* The name of the operation processor","type":"string"},"stream":{"anyOf":[{"type":"string"},{"$ref":"#/definitions/StreamDefinitionSource"}],"description":"The stream to merge with"},"type":{"description":"The type of the operation","enum":["merge"]}},"required":["stream","type"],"title":"MergeOperation","type":"object"},"MergerDefinition":{"additionalProperties":false,"description":"Defines a merger function, that gets injected into the Kafka Streams topology","properties":{"code":{"anyOf":[{"type":"boolean"},{"type":"number"},{"type":"string"}],"description":"*(optional)* The (multiline) code of the merger"},"expression":{"anyOf":[{"type":"boolean"},{"type":"number"},{"type":"string"}],"description":"*(optional)* The expression returned by the merger. Only required for functions that return values."},"globalCode":{"anyOf":[{"type":"boolean"},{"type":"number"},{"type":"string"}],"description":"*(optional)* Global (multiline) code that gets loaded into the Python context outside of the merger. Can be used for defining eg. global variables."},"name":{"description":"*(optional)* The name of the merger. If this field is not defined, then the name is derived from the context.","type":"string"},"parameters":{"description":"*(optional)* A list of parameters to be passed into the merger","items":{"$ref":"#/definitions/ParameterDefinition"},"type":"array"},"resultType":{"description":"*(optional)* The data type returned by the merger. Only required for function types, which are not pre-defined.","type":"string"},"type":{"description":"The type of the function","enum":["merger"]}},"required":["type"],"title":"MergerDefinition","type":"object"},"MergerDefinitionWithImplicitType":{"additionalProperties":false,"description":"Defines a merger function, that gets injected into the Kafka Streams topology","properties":{"code":{"anyOf":[{"type":"boolean"},{"type":"number"},{"type":"string"}],"description":"*(optional)* The (multiline) code of the merger"},"expression":{"anyOf":[{"type":"boolean"},{"type":"number"},{"type":"string"}],"description":"*(optional)* The expression returned by the merger. Only required for functions that return values."},"globalCode":{"anyOf":[{"type":"boolean"},{"type":"number"},{"type":"string"}],"description":"*(optional)* Global (multiline) code that gets loaded into the Python context outside of the merger. Can be used for defining eg. global variables."},"name":{"description":"*(optional)* The name of the merger. If this field is not defined, then the name is derived from the context.","type":"string"},"parameters":{"description":"*(optional)* A list of parameters to be passed into the merger","items":{"$ref":"#/definitions/ParameterDefinition"},"type":"array"},"resultType":{"description":"*(optional)* The data type returned by the merger. Only required for function types, which are not pre-defined.","type":"string"}},"title":"MergerDefinitionWithImplicitType","type":"object"},"MetadataTransformerDefinition":{"additionalProperties":false,"description":"Defines a metadata transformer function, that gets injected into the Kafka Streams topology","properties":{"code":{"anyOf":[{"type":"boolean"},{"type":"number"},{"type":"string"}],"description":"*(optional)* The (multiline) code of the metadata transformer"},"expression":{"anyOf":[{"type":"boolean"},{"type":"number"},{"type":"string"}],"description":"*(optional)* The expression returned by the metadata transformer. Only required for functions that return values."},"globalCode":{"anyOf":[{"type":"boolean"},{"type":"number"},{"type":"string"}],"description":"*(optional)* Global (multiline) code that gets loaded into the Python context outside of the metadata transformer. Can be used for defining eg. global variables."},"name":{"description":"*(optional)* The name of the metadata transformer. If this field is not defined, then the name is derived from the context.","type":"string"},"parameters":{"description":"*(optional)* A list of parameters to be passed into the metadata transformer","items":{"$ref":"#/definitions/ParameterDefinition"},"type":"array"},"resultType":{"description":"*(optional)* The data type returned by the metadata transformer. Only required for function types, which are not pre-defined.","type":"string"},"stores":{"description":"*(optional)* A list of store names that the metadata transformer uses. Only required if the function wants to use a state store.","items":{"type":"string"},"type":"array"},"type":{"description":"The type of the function","enum":["metadataTransformer"]}},"required":["type"],"title":"MetadataTransformerDefinition","type":"object"},"MetadataTransformerDefinitionWithImplicitType":{"additionalProperties":false,"description":"Defines a metadata transformer function, that gets injected into the Kafka Streams topology","properties":{"code":{"anyOf":[{"type":"boolean"},{"type":"number"},{"type":"string"}],"description":"*(optional)* The (multiline) code of the metadata transformer"},"expression":{"anyOf":[{"type":"boolean"},{"type":"number"},{"type":"string"}],"description":"*(optional)* The expression returned by the metadata transformer. Only required for functions that return values."},"globalCode":{"anyOf":[{"type":"boolean"},{"type":"number"},{"type":"string"}],"description":"*(optional)* Global (multiline) code that gets loaded into the Python context outside of the metadata transformer. Can be used for defining eg. global variables."},"name":{"description":"*(optional)* The name of the metadata transformer. If this field is not defined, then the name is derived from the context.","type":"string"},"parameters":{"description":"*(optional)* A list of parameters to be passed into the metadata transformer","items":{"$ref":"#/definitions/ParameterDefinition"},"type":"array"},"resultType":{"description":"*(optional)* The data type returned by the metadata transformer. Only required for function types, which are not pre-defined.","type":"string"},"stores":{"description":"*(optional)* A list of store names that the metadata transformer uses. Only required if the function wants to use a state store.","items":{"type":"string"},"type":"array"}},"title":"MetadataTransformerDefinitionWithImplicitType","type":"object"},"OuterJoinOperationWithStream":{"additionalProperties":false,"description":"Operation to outerJoin with a stream","properties":{"grace":{"anyOf":[{"type":"number"},{"type":"string"}],"description":"*(optional)* The window grace period (the time to admit out-of-order events after the end of the window)"},"name":{"description":"*(optional)* The name of the operation processor","type":"string"},"store":{"anyOf":[{"type":"string"},{"$ref":"#/definitions/KeyValueStateStoreDefinition"},{"$ref":"#/definitions/SessionStateStoreDefinition"},{"$ref":"#/definitions/WindowStateStoreDefinition"}],"description":"*(optional)* Materialized view of the outerJoined streams"},"stream":{"anyOf":[{"type":"string"},{"$ref":"#/definitions/StreamDefinition"}],"description":"A reference to the stream, or an inline definition of the stream to outerJoin with"},"timeDifference":{"anyOf":[{"type":"number"},{"type":"string"}],"description":"The maximum time difference for an outerJoin over two streams on the same key"},"type":{"description":"The type of the operation","enum":["outerJoin"]},"valueJoiner":{"anyOf":[{"type":"string"},{"$ref":"#/definitions/ValueJoinerDefinitionWithImplicitType"}],"description":"A function that joins two values"}},"required":["stream","valueJoiner","timeDifference","type"],"title":"OuterJoinOperationWithStream","type":"object"},"OuterJoinOperationWithTable":{"additionalProperties":false,"description":"Operation to outerJoin with a table","properties":{"name":{"description":"*(optional)* The name of the operation processor","type":"string"},"store":{"anyOf":[{"type":"string"},{"$ref":"#/definitions/KeyValueStateStoreDefinition"},{"$ref":"#/definitions/SessionStateStoreDefinition"},{"$ref":"#/definitions/WindowStateStoreDefinition"}],"description":"*(optional)* Materialized view of the outerJoined streams"},"table":{"anyOf":[{"type":"string"},{"$ref":"#/definitions/TableDefinition"}],"description":"A reference to the table, or an inline definition of the table to outerJoin with"},"type":{"description":"The type of the operation","enum":["outerJoin"]},"valueJoiner":{"anyOf":[{"type":"string"},{"$ref":"#/definitions/ValueJoinerDefinitionWithImplicitType"}],"description":"A function that joins two values"}},"required":["table","valueJoiner","type"],"title":"OuterJoinOperationWithTable","type":"object"},"ParameterDefinition":{"additionalProperties":false,"description":"Defines a parameter for a user function","properties":{"defaultValue":{"description":"*(optional)* The default value for the parameter","type":"string"},"name":{"description":"The name of the parameter","type":"string"},"type":{"description":"The type of the parameter","type":"string"}},"required":["name","type"],"title":"ParameterDefinition","type":"object"},"PeekOperation":{"additionalProperties":false,"description":"Operation to peek into a stream, without modifying the stream contents","properties":{"forEach":{"anyOf":[{"type":"string"},{"$ref":"#/definitions/ForEachActionDefinitionWithImplicitType"}],"description":"A function that gets called for every message in the stream"},"name":{"description":"*(optional)* The name of the operation processor","type":"string"},"type":{"description":"The type of the operation","enum":["peek"]}},"required":["forEach","type"],"title":"PeekOperation","type":"object"},"PipelineDefinition":{"additionalProperties":false,"description":"Defines a pipeline through a source, a series of operations to perform on it and a sink operation to close the stream with","properties":{"as":{"description":"*(optional)* The name to register the pipeline result under, which can be used as source by follow-up pipelines","type":"string"},"branch":{"description":"*(optional)* Defines a single branch, consisting of a condition and a pipeline to execute for messages that fulfil the predicate","items":{"$ref":"#/definitions/BranchDefinitionWithPipeline"},"type":"array"},"forEach":{"anyOf":[{"type":"string"},{"$ref":"#/definitions/ForEachActionDefinitionWithImplicitType"}],"description":"*(optional)* A function that gets called for every message in the stream"},"from":{"anyOf":[{"type":"string"},{"$ref":"#/definitions/TopicDefinitionSource"}],"description":"Pipeline source"},"name":{"description":"*(optional)* The name of the operation processor","type":"string"},"print":{"$ref":"#/definitions/PrintOperation","description":"*(optional)* The specification of where to print messages to"},"to":{"anyOf":[{"type":"string"},{"$ref":"#/definitions/ToTopicDefinition"}],"description":"*(optional)* Ends the pipeline by sending all messages to a stream, table or globalTable, or to an inline defined output topic and optional partitioner"},"toTopicNameExtractor":{"anyOf":[{"type":"string"},{"$ref":"#/definitions/ToTopicNameExtractorDefinition"}],"description":"*(optional)* Ends the pipeline by sending all messages to a topic provided by a pre-defined topic name extractor function, or to a topic provided by an inline defined topic name extractor and optional partitioner"},"via":{"description":"*(optional)* A series of operations performed on the input stream","items":{"anyOf":[{"$ref":"#/definitions/AggregateOperation"},{"$ref":"#/definitions/CogroupOperation"},{"$ref":"#/definitions/ConvertKeyOperation"},{"$ref":"#/definitions/ConvertKeyValueOperation"},{"$ref":"#/definitions/ConvertValueOperation"},{"$ref":"#/definitions/CountOperation"},{"$ref":"#/definitions/FilterNotOperation"},{"$ref":"#/definitions/FilterOperation"},{"$ref":"#/definitions/GroupByKeyOperation"},{"$ref":"#/definitions/GroupByOperation"},{"$ref":"#/definitions/JoinOperationWithGlobalTable"},{"$ref":"#/definitions/JoinOperationWithStream"},{"$ref":"#/definitions/JoinOperationWithTable"},{"$ref":"#/definitions/LeftJoinOperationWithGlobalTable"},{"$ref":"#/definitions/LeftJoinOperationWithStream"},{"$ref":"#/definitions/LeftJoinOperationWithTable"},{"$ref":"#/definitions/MergeOperation"},{"$ref":"#/definitions/OuterJoinOperationWithStream"},{"$ref":"#/definitions/OuterJoinOperationWithTable"},{"$ref":"#/definitions/PeekOperation"},{"$ref":"#/definitions/ReduceOperationWithAdderAndSubtractor"},{"$ref":"#/definitions/ReduceOperationWithReducer"},{"$ref":"#/definitions/RepartitionOperation"},{"$ref":"#/definitions/SuppressOperationUntilTimeLimit"},{"$ref":"#/definitions/SuppressOperationUntilWindowCloses"},{"$ref":"#/definitions/ToStreamOperation"},{"$ref":"#/definitions/ToTableOperation"},{"$ref":"#/definitions/TransformKeyOperation"},{"$ref":"#/definitions/TransformKeyValueOperation"},{"$ref":"#/definitions/TransformKeyValueToKeyValueListOperation"},{"$ref":"#/definitions/TransformKeyValueToValueListOperation"},{"$ref":"#/definitions/TransformMetadataOperation"},{"$ref":"#/definitions/TransformValueOperation"},{"$ref":"#/definitions/WindowBySessionOperation"},{"$ref":"#/definitions/WindowByTimeOperationWithHoppingWindow"},{"$ref":"#/definitions/WindowByTimeOperationWithSlidingWindow"},{"$ref":"#/definitions/WindowByTimeOperationWithTumblingWindow"}]},"type":"array"}},"required":["from"],"title":"PipelineDefinition","type":"object"},"PredicateDefinition":{"additionalProperties":false,"description":"Defines a predicate function, that gets injected into the Kafka Streams topology","properties":{"code":{"anyOf":[{"type":"boolean"},{"type":"number"},{"type":"string"}],"description":"*(optional)* The (multiline) code of the predicate"},"expression":{"anyOf":[{"type":"boolean"},{"type":"number"},{"type":"string"}],"description":"*(optional)* The expression returned by the predicate. Only required for functions that return values."},"globalCode":{"anyOf":[{"type":"boolean"},{"type":"number"},{"type":"string"}],"description":"*(optional)* Global (multiline) code that gets loaded into the Python context outside of the predicate. Can be used for defining eg. global variables."},"name":{"description":"*(optional)* The name of the predicate. If this field is not defined, then the name is derived from the context.","type":"string"},"parameters":{"description":"*(optional)* A list of parameters to be passed into the predicate","items":{"$ref":"#/definitions/ParameterDefinition"},"type":"array"},"resultType":{"description":"*(optional)* The data type returned by the predicate. Only required for function types, which are not pre-defined.","type":"string"},"stores":{"description":"*(optional)* A list of store names that the predicate uses. Only required if the function wants to use a state store.","items":{"type":"string"},"type":"array"},"type":{"description":"The type of the function","enum":["predicate"]}},"required":["type"],"title":"PredicateDefinition","type":"object"},"PredicateDefinitionWithImplicitType":{"additionalProperties":false,"description":"Defines a predicate function, that gets injected into the Kafka Streams topology","properties":{"code":{"anyOf":[{"type":"boolean"},{"type":"number"},{"type":"string"}],"description":"*(optional)* The (multiline) code of the predicate"},"expression":{"anyOf":[{"type":"boolean"},{"type":"number"},{"type":"string"}],"description":"*(optional)* The expression returned by the predicate. Only required for functions that return values."},"globalCode":{"anyOf":[{"type":"boolean"},{"type":"number"},{"type":"string"}],"description":"*(optional)* Global (multiline) code that gets loaded into the Python context outside of the predicate. Can be used for defining eg. global variables."},"name":{"description":"*(optional)* The name of the predicate. If this field is not defined, then the name is derived from the context.","type":"string"},"parameters":{"description":"*(optional)* A list of parameters to be passed into the predicate","items":{"$ref":"#/definitions/ParameterDefinition"},"type":"array"},"resultType":{"description":"*(optional)* The data type returned by the predicate. Only required for function types, which are not pre-defined.","type":"string"},"stores":{"description":"*(optional)* A list of store names that the predicate uses. Only required if the function wants to use a state store.","items":{"type":"string"},"type":"array"}},"title":"PredicateDefinitionWithImplicitType","type":"object"},"PrintOperation":{"additionalProperties":false,"description":"Operation to print the contents of a pipeline on the screen or to write them to a file","properties":{"filename":{"description":"*(optional)* The filename to output records to. If nothing is specified, then messages will be printed on stdout.","type":"string"},"label":{"description":"*(optional)* A label to attach to the output records","type":"string"},"mapper":{"anyOf":[{"type":"string"},{"$ref":"#/definitions/KeyValuePrinterDefinitionWithImplicitType"}],"description":"*(optional)* A function to convert record into a string for output"},"name":{"description":"*(optional)* The name of the operation processor","type":"string"}},"title":"PrintOperation","type":"object"},"ProducerDefinition":{"additionalProperties":false,"description":"Definition of a Producer that regularly generates messages for a topic","properties":{"condition":{"anyOf":[{"type":"string"},{"$ref":"#/definitions/PredicateDefinitionWithImplicitType"}],"description":"*(optional)* A function that validates the generator's result message. Returns \"true\" when the message may be produced on the topic, \"false\" otherwise."},"count":{"description":"*(optional)* The number of messages to produce."},"generator":{"anyOf":[{"type":"string"},{"$ref":"#/definitions/GeneratorDefinitionWithImplicitType"}],"description":"The function that generates records"},"interval":{"anyOf":[{"type":"number"},{"type":"string"}],"description":"The interval with which the generator is called"},"to":{"anyOf":[{"type":"string"},{"$ref":"#/definitions/TopicDefinition"}],"description":"The topic to produce to"},"until":{"anyOf":[{"type":"string"},{"$ref":"#/definitions/PredicateDefinitionWithImplicitType"}],"description":"*(optional)* A predicate that returns true to indicate producing should stop."}},"required":["generator","interval","to"],"title":"ProducerDefinition","type":"object"},"ReduceOperationWithAdderAndSubtractor":{"additionalProperties":false,"description":"Operation to reduce a series of records into a single aggregate result","properties":{"adder":{"anyOf":[{"type":"string"},{"$ref":"#/definitions/ReducerDefinitionWithImplicitType"}],"description":"A function that adds a record to the aggregate result"},"name":{"description":"*(optional)* The name of the operation processor","type":"string"},"store":{"anyOf":[{"type":"string"},{"$ref":"#/definitions/WindowStateStoreDefinitionWithImplicitType"}],"description":"*(optional)* Materialized view of the aggregation"},"subtractor":{"anyOf":[{"type":"string"},{"$ref":"#/definitions/ReducerDefinitionWithImplicitType"}],"description":"A function that removes a record from the aggregate result"},"type":{"description":"The type of the operation","enum":["reduce"]}},"required":["adder","subtractor","type"],"title":"ReduceOperationWithAdderAndSubtractor","type":"object"},"ReduceOperationWithReducer":{"additionalProperties":false,"description":"Operation to reduce a series of records into a single aggregate result","properties":{"name":{"description":"*(optional)* The name of the operation processor","type":"string"},"reducer":{"anyOf":[{"type":"string"},{"$ref":"#/definitions/ReducerDefinitionWithImplicitType"}],"description":"A function that computes a new aggregate result"},"store":{"anyOf":[{"type":"string"},{"$ref":"#/definitions/WindowStateStoreDefinitionWithImplicitType"}],"description":"*(optional)* Materialized view of the aggregation"},"type":{"description":"The type of the operation","enum":["reduce"]}},"required":["reducer","type"],"title":"ReduceOperationWithReducer","type":"object"},"ReducerDefinition":{"additionalProperties":false,"description":"Defines a reducer function, that gets injected into the Kafka Streams topology","properties":{"code":{"anyOf":[{"type":"boolean"},{"type":"number"},{"type":"string"}],"description":"*(optional)* The (multiline) code of the reducer"},"expression":{"anyOf":[{"type":"boolean"},{"type":"number"},{"type":"string"}],"description":"*(optional)* The expression returned by the reducer. Only required for functions that return values."},"globalCode":{"anyOf":[{"type":"boolean"},{"type":"number"},{"type":"string"}],"description":"*(optional)* Global (multiline) code that gets loaded into the Python context outside of the reducer. Can be used for defining eg. global variables."},"name":{"description":"*(optional)* The name of the reducer. If this field is not defined, then the name is derived from the context.","type":"string"},"parameters":{"description":"*(optional)* A list of parameters to be passed into the reducer","items":{"$ref":"#/definitions/ParameterDefinition"},"type":"array"},"resultType":{"description":"*(optional)* The data type returned by the reducer. Only required for function types, which are not pre-defined.","type":"string"},"type":{"description":"The type of the function","enum":["reducer"]}},"required":["type"],"title":"ReducerDefinition","type":"object"},"ReducerDefinitionWithImplicitType":{"additionalProperties":false,"description":"Defines a reducer function, that gets injected into the Kafka Streams topology","properties":{"code":{"anyOf":[{"type":"boolean"},{"type":"number"},{"type":"string"}],"description":"*(optional)* The (multiline) code of the reducer"},"expression":{"anyOf":[{"type":"boolean"},{"type":"number"},{"type":"string"}],"description":"*(optional)* The expression returned by the reducer. Only required for functions that return values."},"globalCode":{"anyOf":[{"type":"boolean"},{"type":"number"},{"type":"string"}],"description":"*(optional)* Global (multiline) code that gets loaded into the Python context outside of the reducer. Can be used for defining eg. global variables."},"name":{"description":"*(optional)* The name of the reducer. If this field is not defined, then the name is derived from the context.","type":"string"},"parameters":{"description":"*(optional)* A list of parameters to be passed into the reducer","items":{"$ref":"#/definitions/ParameterDefinition"},"type":"array"},"resultType":{"description":"*(optional)* The data type returned by the reducer. Only required for function types, which are not pre-defined.","type":"string"}},"title":"ReducerDefinitionWithImplicitType","type":"object"},"RepartitionOperation":{"additionalProperties":false,"description":"Operation to (re)partition a stream","properties":{"name":{"description":"*(optional)* The name of the operation processor","type":"string"},"numberOfPartitions":{"description":"*(optional)* The target number of partitions"},"partitioner":{"anyOf":[{"type":"string"},{"$ref":"#/definitions/StreamPartitionerDefinitionWithImplicitType"}],"description":"*(optional)* A function that partitions stream records"},"type":{"description":"The type of the operation","enum":["repartition"]}},"required":["type"],"title":"RepartitionOperation","type":"object"},"SessionStateStoreDefinition":{"additionalProperties":false,"description":"Definition of a session state store","properties":{"caching":{"description":"*(optional)* \"true\" if changed to the session store need to be buffered and periodically released, \"false\" to emit all changes directly","type":"boolean"},"keyType":{"description":"*(optional)* The key type of the session store","type":"string"},"logging":{"description":"*(optional)* \"true\" if a changelog topic should be set up on Kafka for this session store, \"false\" otherwise","type":"boolean"},"name":{"description":"*(optional)* The name of the session store. If this field is not defined, then the name is derived from the context.","type":"string"},"persistent":{"description":"*(optional)* \"true\" if this session store needs to be stored on disk, \"false\" otherwise","type":"boolean"},"retention":{"anyOf":[{"type":"number"},{"type":"string"}],"description":"*(optional)* The duration for which elements in the session store are retained"},"timestamped":{"description":"*(optional)* \"true\" if elements in the store are timestamped, \"false\" otherwise","type":"boolean"},"type":{"description":"The type of the state store","enum":["session"]},"valueType":{"description":"*(optional)* The value type of the session store","type":"string"}},"required":["type"],"title":"SessionStateStoreDefinition","type":"object"},"StreamDefinition":{"additionalProperties":false,"description":"Contains a definition of a Stream, which can be referenced by producers and pipelines","properties":{"keyType":{"description":"*(optional)* The key type of the stream","type":"string"},"topic":{"description":"The name of the Kafka topic for this stream","type":"string"},"valueType":{"description":"*(optional)* The value type of the stream","type":"string"}},"required":["topic"],"title":"StreamDefinition","type":"object"},"StreamDefinitionSource":{"additionalProperties":false,"description":"Contains a definition of a Stream, which can be referenced by producers and pipelines","properties":{"keyType":{"description":"The key type of the stream","type":"string"},"offsetResetPolicy":{"description":"*(optional)* Policy that determines what to do when there is no initial offset in Kafka, or if the current offset does not exist any more on the server (e.g. because that data has been deleted)","type":"string"},"timestampExtractor":{"anyOf":[{"type":"string"},{"$ref":"#/definitions/TimestampExtractorDefinitionWithImplicitType"}],"description":"*(optional)* A function extracts the event time from a consumed record"},"topic":{"description":"The name of the Kafka topic for this stream","type":"string"},"valueType":{"description":"The value type of the stream","type":"string"}},"required":["topic","keyType","valueType"],"title":"StreamDefinitionSource","type":"object"},"StreamPartitionerDefinition":{"additionalProperties":false,"description":"Defines a stream partitioner function, that gets injected into the Kafka Streams topology","properties":{"code":{"anyOf":[{"type":"boolean"},{"type":"number"},{"type":"string"}],"description":"*(optional)* The (multiline) code of the stream partitioner"},"expression":{"anyOf":[{"type":"boolean"},{"type":"number"},{"type":"string"}],"description":"*(optional)* The expression returned by the stream partitioner. Only required for functions that return values."},"globalCode":{"anyOf":[{"type":"boolean"},{"type":"number"},{"type":"string"}],"description":"*(optional)* Global (multiline) code that gets loaded into the Python context outside of the stream partitioner. Can be used for defining eg. global variables."},"name":{"description":"*(optional)* The name of the stream partitioner. If this field is not defined, then the name is derived from the context.","type":"string"},"parameters":{"description":"*(optional)* A list of parameters to be passed into the stream partitioner","items":{"$ref":"#/definitions/ParameterDefinition"},"type":"array"},"resultType":{"description":"*(optional)* The data type returned by the stream partitioner. Only required for function types, which are not pre-defined.","type":"string"},"type":{"description":"The type of the function","enum":["streamPartitioner"]}},"required":["type"],"title":"StreamPartitionerDefinition","type":"object"},"StreamPartitionerDefinitionWithImplicitType":{"additionalProperties":false,"description":"Defines a stream partitioner function, that gets injected into the Kafka Streams topology","properties":{"code":{"anyOf":[{"type":"boolean"},{"type":"number"},{"type":"string"}],"description":"*(optional)* The (multiline) code of the stream partitioner"},"expression":{"anyOf":[{"type":"boolean"},{"type":"number"},{"type":"string"}],"description":"*(optional)* The expression returned by the stream partitioner. Only required for functions that return values."},"globalCode":{"anyOf":[{"type":"boolean"},{"type":"number"},{"type":"string"}],"description":"*(optional)* Global (multiline) code that gets loaded into the Python context outside of the stream partitioner. Can be used for defining eg. global variables."},"name":{"description":"*(optional)* The name of the stream partitioner. If this field is not defined, then the name is derived from the context.","type":"string"},"parameters":{"description":"*(optional)* A list of parameters to be passed into the stream partitioner","items":{"$ref":"#/definitions/ParameterDefinition"},"type":"array"},"resultType":{"description":"*(optional)* The data type returned by the stream partitioner. Only required for function types, which are not pre-defined.","type":"string"}},"title":"StreamPartitionerDefinitionWithImplicitType","type":"object"},"StringOrInlinePredicateDefinitionWithImplicitType":{"additionalProperties":false,"description":"Defines the condition under which messages get sent down this branch","properties":{"if":{"anyOf":[{"type":"string"},{"$ref":"#/definitions/PredicateDefinitionWithImplicitType"}],"description":"*(optional)* Defines the condition under which messages get sent down this branch"}},"title":"StringOrInlinePredicateDefinitionWithImplicitType","type":"object"},"SuppressOperationUntilTimeLimit":{"additionalProperties":false,"description":"Operation to suppress messages in the source stream until a time limit is reached","properties":{"bufferFullStrategy":{"description":"*(optional)* What to do when the buffer is full","enum":["emitEarlyWhenFull","shutdownWhenFull"]},"duration":{"anyOf":[{"type":"number"},{"type":"string"}],"description":"The duration for which messages are suppressed"},"maxBytes":{"description":"*(optional)* The maximum number of bytes in the buffer","type":"string"},"maxRecords":{"description":"*(optional)* The maximum number of records in the buffer","type":"string"},"name":{"description":"*(optional)* The name of the operation processor","type":"string"},"type":{"description":"The type of the operation","enum":["suppress"]},"until":{"description":"The until of the Operation to suppress messages in the source stream until a certain limit is reached","enum":["timeLimit"]}},"required":["duration","until","type"],"title":"SuppressOperationUntilTimeLimit","type":"object"},"SuppressOperationUntilWindowCloses":{"additionalProperties":false,"description":"Operation to suppress messages in the source stream until a window limit is reached","properties":{"bufferFullStrategy":{"description":"*(optional)* What to do when the buffer is full","enum":["emitEarlyWhenFull","shutdownWhenFull"]},"maxBytes":{"description":"*(optional)* The maximum number of bytes in the buffer","type":"string"},"maxRecords":{"description":"*(optional)* The maximum number of records in the buffer","type":"string"},"name":{"description":"*(optional)* The name of the operation processor","type":"string"},"type":{"description":"The type of the operation","enum":["suppress"]},"until":{"description":"The until of the Operation to suppress messages in the source stream until a certain limit is reached","enum":["windowCloses"]}},"required":["until","type"],"title":"SuppressOperationUntilWindowCloses","type":"object"},"TableDefinition":{"additionalProperties":false,"description":"Contains a definition of a Table, which can be referenced by producers and pipelines","properties":{"keyType":{"description":"*(optional)* The key type of the table","type":"string"},"store":{"anyOf":[{"type":"string"},{"$ref":"#/definitions/KeyValueStateStoreDefinitionWithImplicitType"}],"description":"*(optional)* KeyValue state store definition"},"topic":{"description":"The name of the Kafka topic for this table","type":"string"},"valueType":{"description":"*(optional)* The value type of the table","type":"string"}},"required":["topic"],"title":"TableDefinition","type":"object"},"TableDefinitionSource":{"additionalProperties":false,"description":"Contains a definition of a Table, which can be referenced by producers and pipelines","properties":{"keyType":{"description":"The key type of the table","type":"string"},"offsetResetPolicy":{"description":"*(optional)* Policy that determines what to do when there is no initial offset in Kafka, or if the current offset does not exist any more on the server (e.g. because that data has been deleted)","type":"string"},"store":{"anyOf":[{"type":"string"},{"$ref":"#/definitions/KeyValueStateStoreDefinitionWithImplicitType"}],"description":"*(optional)* KeyValue state store definition"},"timestampExtractor":{"anyOf":[{"type":"string"},{"$ref":"#/definitions/TimestampExtractorDefinitionWithImplicitType"}],"description":"*(optional)* A function extracts the event time from a consumed record"},"topic":{"description":"The name of the Kafka topic for this table","type":"string"},"valueType":{"description":"The value type of the table","type":"string"}},"required":["topic","keyType","valueType"],"title":"TableDefinitionSource","type":"object"},"TimestampExtractorDefinition":{"additionalProperties":false,"description":"Defines a timestamp extractor function, that gets injected into the Kafka Streams topology","properties":{"code":{"anyOf":[{"type":"boolean"},{"type":"number"},{"type":"string"}],"description":"*(optional)* The (multiline) code of the timestamp extractor"},"expression":{"anyOf":[{"type":"boolean"},{"type":"number"},{"type":"string"}],"description":"*(optional)* The expression returned by the timestamp extractor. Only required for functions that return values."},"globalCode":{"anyOf":[{"type":"boolean"},{"type":"number"},{"type":"string"}],"description":"*(optional)* Global (multiline) code that gets loaded into the Python context outside of the timestamp extractor. Can be used for defining eg. global variables."},"name":{"description":"*(optional)* The name of the timestamp extractor. If this field is not defined, then the name is derived from the context.","type":"string"},"parameters":{"description":"*(optional)* A list of parameters to be passed into the timestamp extractor","items":{"$ref":"#/definitions/ParameterDefinition"},"type":"array"},"resultType":{"description":"*(optional)* The data type returned by the timestamp extractor. Only required for function types, which are not pre-defined.","type":"string"},"type":{"description":"The type of the function","enum":["timestampExtractor"]}},"required":["type"],"title":"TimestampExtractorDefinition","type":"object"},"TimestampExtractorDefinitionWithImplicitType":{"additionalProperties":false,"description":"Defines a timestamp extractor function, that gets injected into the Kafka Streams topology","properties":{"code":{"anyOf":[{"type":"boolean"},{"type":"number"},{"type":"string"}],"description":"*(optional)* The (multiline) code of the timestamp extractor"},"expression":{"anyOf":[{"type":"boolean"},{"type":"number"},{"type":"string"}],"description":"*(optional)* The expression returned by the timestamp extractor. Only required for functions that return values."},"globalCode":{"anyOf":[{"type":"boolean"},{"type":"number"},{"type":"string"}],"description":"*(optional)* Global (multiline) code that gets loaded into the Python context outside of the timestamp extractor. Can be used for defining eg. global variables."},"name":{"description":"*(optional)* The name of the timestamp extractor. If this field is not defined, then the name is derived from the context.","type":"string"},"parameters":{"description":"*(optional)* A list of parameters to be passed into the timestamp extractor","items":{"$ref":"#/definitions/ParameterDefinition"},"type":"array"},"resultType":{"description":"*(optional)* The data type returned by the timestamp extractor. Only required for function types, which are not pre-defined.","type":"string"}},"title":"TimestampExtractorDefinitionWithImplicitType","type":"object"},"ToStreamOperation":{"additionalProperties":false,"description":"Convert a Table into a Stream, optionally through a custom key transformer","properties":{"mapper":{"anyOf":[{"type":"string"},{"$ref":"#/definitions/KeyTransformerDefinitionWithImplicitType"}],"description":"*(optional)* A function that computes the output key for every record"},"name":{"description":"*(optional)* The name of the operation processor","type":"string"},"type":{"description":"The type of the operation","enum":["toStream"]}},"required":["type"],"title":"ToStreamOperation","type":"object"},"ToTableOperation":{"additionalProperties":false,"description":"Convert a Stream into a Table","properties":{"name":{"description":"*(optional)* The name of the operation processor","type":"string"},"store":{"anyOf":[{"type":"string"},{"$ref":"#/definitions/KeyValueStateStoreDefinitionWithImplicitType"}],"description":"*(optional)* Materialized view of the result table"},"type":{"description":"The type of the operation","enum":["toTable"]}},"required":["type"],"title":"ToTableOperation","type":"object"},"ToTopicDefinition":{"additionalProperties":false,"description":"Writes out pipeline messages to a topic","properties":{"keyType":{"description":"*(optional)* The key type of the topic","type":"string"},"partitioner":{"anyOf":[{"type":"string"},{"$ref":"#/definitions/StreamPartitionerDefinitionWithImplicitType"}],"description":"*(optional)* A function that partitions the records in the output topic"},"topic":{"description":"The name of the Kafka topic","type":"string"},"valueType":{"description":"*(optional)* The value type of the topic","type":"string"}},"required":["topic"],"title":"ToTopicDefinition","type":"object"},"ToTopicNameExtractorDefinition":{"additionalProperties":false,"description":"Writes out pipeline messages to a topic as given by a topic name extractor","properties":{"partitioner":{"anyOf":[{"type":"string"},{"$ref":"#/definitions/StreamPartitionerDefinitionWithImplicitType"}],"description":"*(optional)* A function that partitions the records in the output topic"},"topicNameExtractor":{"anyOf":[{"type":"string"},{"$ref":"#/definitions/TopicNameExtractorDefinitionWithImplicitType"}],"description":"Reference to a pre-defined topic name extractor, or an inline definition of a topic name extractor"}},"required":["topicNameExtractor"],"title":"ToTopicNameExtractorDefinition","type":"object"},"TopicDefinition":{"additionalProperties":false,"description":"Contains a definition of a Kafka topic, to be used by producers and pipelines","properties":{"keyType":{"description":"*(optional)* The key type of the topic","type":"string"},"topic":{"description":"The name of the Kafka topic","type":"string"},"valueType":{"description":"*(optional)* The value type of the topic","type":"string"}},"required":["topic"],"title":"TopicDefinition","type":"object"},"TopicDefinitionSource":{"additionalProperties":false,"description":"Contains a definition of a Kafka topic, to be used by producers and pipelines","properties":{"keyType":{"description":"The key type of the topic","type":"string"},"offsetResetPolicy":{"description":"*(optional)* Policy that determines what to do when there is no initial offset in Kafka, or if the current offset does not exist any more on the server (e.g. because that data has been deleted)","type":"string"},"timestampExtractor":{"anyOf":[{"type":"string"},{"$ref":"#/definitions/TimestampExtractorDefinitionWithImplicitType"}],"description":"*(optional)* A function extracts the event time from a consumed record"},"topic":{"description":"The name of the Kafka topic","type":"string"},"valueType":{"description":"The value type of the topic","type":"string"}},"required":["topic","keyType","valueType"],"title":"TopicDefinitionSource","type":"object"},"TopicNameExtractorDefinition":{"additionalProperties":false,"description":"Defines a topic name extractor function, that gets injected into the Kafka Streams topology","properties":{"code":{"anyOf":[{"type":"boolean"},{"type":"number"},{"type":"string"}],"description":"*(optional)* The (multiline) code of the topic name extractor"},"expression":{"anyOf":[{"type":"boolean"},{"type":"number"},{"type":"string"}],"description":"*(optional)* The expression returned by the topic name extractor. Only required for functions that return values."},"globalCode":{"anyOf":[{"type":"boolean"},{"type":"number"},{"type":"string"}],"description":"*(optional)* Global (multiline) code that gets loaded into the Python context outside of the topic name extractor. Can be used for defining eg. global variables."},"name":{"description":"*(optional)* The name of the topic name extractor. If this field is not defined, then the name is derived from the context.","type":"string"},"parameters":{"description":"*(optional)* A list of parameters to be passed into the topic name extractor","items":{"$ref":"#/definitions/ParameterDefinition"},"type":"array"},"resultType":{"description":"*(optional)* The data type returned by the topic name extractor. Only required for function types, which are not pre-defined.","type":"string"},"type":{"description":"The type of the function","enum":["topicNameExtractor"]}},"required":["type"],"title":"TopicNameExtractorDefinition","type":"object"},"TopicNameExtractorDefinitionWithImplicitType":{"additionalProperties":false,"description":"Defines a topic name extractor function, that gets injected into the Kafka Streams topology","properties":{"code":{"anyOf":[{"type":"boolean"},{"type":"number"},{"type":"string"}],"description":"*(optional)* The (multiline) code of the topic name extractor"},"expression":{"anyOf":[{"type":"boolean"},{"type":"number"},{"type":"string"}],"description":"*(optional)* The expression returned by the topic name extractor. Only required for functions that return values."},"globalCode":{"anyOf":[{"type":"boolean"},{"type":"number"},{"type":"string"}],"description":"*(optional)* Global (multiline) code that gets loaded into the Python context outside of the topic name extractor. Can be used for defining eg. global variables."},"name":{"description":"*(optional)* The name of the topic name extractor. If this field is not defined, then the name is derived from the context.","type":"string"},"parameters":{"description":"*(optional)* A list of parameters to be passed into the topic name extractor","items":{"$ref":"#/definitions/ParameterDefinition"},"type":"array"},"resultType":{"description":"*(optional)* The data type returned by the topic name extractor. Only required for function types, which are not pre-defined.","type":"string"}},"title":"TopicNameExtractorDefinitionWithImplicitType","type":"object"},"TransformKeyOperation":{"additionalProperties":false,"description":"Convert the key of every record in the stream to another key","properties":{"mapper":{"anyOf":[{"type":"string"},{"$ref":"#/definitions/KeyTransformerDefinitionWithImplicitType"}],"description":"A function that computes a new key for each record"},"name":{"description":"*(optional)* The name of the operation processor","type":"string"},"type":{"description":"The type of the operation","enum":["transformKey","mapKey","selectKey"]}},"required":["mapper","type"],"title":"TransformKeyOperation","type":"object"},"TransformKeyValueOperation":{"additionalProperties":false,"description":"Convert the key/value of every record in the stream to another key/value","properties":{"mapper":{"anyOf":[{"type":"string"},{"$ref":"#/definitions/KeyValueTransformerDefinitionWithImplicitType"}],"description":"A function that computes a new key/value for each record"},"name":{"description":"*(optional)* The name of the operation processor","type":"string"},"type":{"description":"The type of the operation","enum":["mapKeyValue","map","transformKeyValue"]}},"required":["mapper","type"],"title":"TransformKeyValueOperation","type":"object"},"TransformKeyValueToKeyValueListOperation":{"additionalProperties":false,"description":"Convert a stream by transforming every record into a list of derived records","properties":{"mapper":{"anyOf":[{"type":"string"},{"$ref":"#/definitions/KeyValueToKeyValueListTransformerDefinitionWithImplicitType"}],"description":"A function that converts every record of a stream to a list of output records."},"name":{"description":"*(optional)* The name of the operation processor","type":"string"},"type":{"description":"The type of the operation","enum":["transformKeyValueToKeyValueList","flatMap"]}},"required":["mapper","type"],"title":"TransformKeyValueToKeyValueListOperation","type":"object"},"TransformKeyValueToValueListOperation":{"additionalProperties":false,"description":"Convert every record in the stream to a list of output records with the same key","properties":{"mapper":{"anyOf":[{"type":"string"},{"$ref":"#/definitions/KeyValueToValueListTransformerDefinitionWithImplicitType"}],"description":"A function that converts every key/value into a list of result values, each of which will be combined with the original key to form a new message in the output stream"},"name":{"description":"*(optional)* The name of the operation processor","type":"string"},"type":{"description":"The type of the operation","enum":["transformKeyValueToValueList","flatMapValues"]}},"required":["mapper","type"],"title":"TransformKeyValueToValueListOperation","type":"object"},"TransformMetadataOperation":{"additionalProperties":false,"description":"Convert the metadata of every record in the stream","properties":{"mapper":{"anyOf":[{"type":"string"},{"$ref":"#/definitions/MetadataTransformerDefinitionWithImplicitType"}],"description":"A function that converts the metadata (Kafka headers, timestamp) of every record in the stream"},"name":{"description":"*(optional)* The name of the operation processor","type":"string"},"type":{"description":"The type of the operation","enum":["transformMetadata"]}},"required":["mapper","type"],"title":"TransformMetadataOperation","type":"object"},"TransformValueOperation":{"additionalProperties":false,"description":"Convert the value of every record in the stream to another value","properties":{"mapper":{"anyOf":[{"type":"string"},{"$ref":"#/definitions/ValueTransformerDefinitionWithImplicitType"}],"description":"A function that converts the value of every record into another value"},"name":{"description":"*(optional)* The name of the operation processor","type":"string"},"store":{"anyOf":[{"type":"string"},{"$ref":"#/definitions/KeyValueStateStoreDefinitionWithImplicitType"}],"description":"*(optional)* Materialized view of the transformed table (only applies to tables, ignored for streams)"},"type":{"description":"The type of the operation","enum":["mapValue","transformValue","mapValues"]}},"required":["mapper","type"],"title":"TransformValueOperation","type":"object"},"ValueJoinerDefinition":{"additionalProperties":false,"description":"Defines a value joiner function, that gets injected into the Kafka Streams topology","properties":{"code":{"anyOf":[{"type":"boolean"},{"type":"number"},{"type":"string"}],"description":"*(optional)* The (multiline) code of the value joiner"},"expression":{"anyOf":[{"type":"boolean"},{"type":"number"},{"type":"string"}],"description":"*(optional)* The expression returned by the value joiner. Only required for functions that return values."},"globalCode":{"anyOf":[{"type":"boolean"},{"type":"number"},{"type":"string"}],"description":"*(optional)* Global (multiline) code that gets loaded into the Python context outside of the value joiner. Can be used for defining eg. global variables."},"name":{"description":"*(optional)* The name of the value joiner. If this field is not defined, then the name is derived from the context.","type":"string"},"parameters":{"description":"*(optional)* A list of parameters to be passed into the value joiner","items":{"$ref":"#/definitions/ParameterDefinition"},"type":"array"},"resultType":{"description":"*(optional)* The data type returned by the value joiner. Only required for function types, which are not pre-defined.","type":"string"},"type":{"description":"The type of the function","enum":["valueJoiner"]}},"required":["type"],"title":"ValueJoinerDefinition","type":"object"},"ValueJoinerDefinitionWithImplicitType":{"additionalProperties":false,"description":"Defines a value joiner function, that gets injected into the Kafka Streams topology","properties":{"code":{"anyOf":[{"type":"boolean"},{"type":"number"},{"type":"string"}],"description":"*(optional)* The (multiline) code of the value joiner"},"expression":{"anyOf":[{"type":"boolean"},{"type":"number"},{"type":"string"}],"description":"*(optional)* The expression returned by the value joiner. Only required for functions that return values."},"globalCode":{"anyOf":[{"type":"boolean"},{"type":"number"},{"type":"string"}],"description":"*(optional)* Global (multiline) code that gets loaded into the Python context outside of the value joiner. Can be used for defining eg. global variables."},"name":{"description":"*(optional)* The name of the value joiner. If this field is not defined, then the name is derived from the context.","type":"string"},"parameters":{"description":"*(optional)* A list of parameters to be passed into the value joiner","items":{"$ref":"#/definitions/ParameterDefinition"},"type":"array"},"resultType":{"description":"*(optional)* The data type returned by the value joiner. Only required for function types, which are not pre-defined.","type":"string"}},"title":"ValueJoinerDefinitionWithImplicitType","type":"object"},"ValueTransformerDefinition":{"additionalProperties":false,"description":"Defines a value transformer function, that gets injected into the Kafka Streams topology","properties":{"code":{"anyOf":[{"type":"boolean"},{"type":"number"},{"type":"string"}],"description":"*(optional)* The (multiline) code of the value transformer"},"expression":{"anyOf":[{"type":"boolean"},{"type":"number"},{"type":"string"}],"description":"*(optional)* The expression returned by the value transformer. Only required for functions that return values."},"globalCode":{"anyOf":[{"type":"boolean"},{"type":"number"},{"type":"string"}],"description":"*(optional)* Global (multiline) code that gets loaded into the Python context outside of the value transformer. Can be used for defining eg. global variables."},"name":{"description":"*(optional)* The name of the value transformer. If this field is not defined, then the name is derived from the context.","type":"string"},"parameters":{"description":"*(optional)* A list of parameters to be passed into the value transformer","items":{"$ref":"#/definitions/ParameterDefinition"},"type":"array"},"resultType":{"description":"*(optional)* The data type returned by the value transformer. Only required for function types, which are not pre-defined.","type":"string"},"stores":{"description":"*(optional)* A list of store names that the value transformer uses. Only required if the function wants to use a state store.","items":{"type":"string"},"type":"array"},"type":{"description":"The type of the function","enum":["valueTransformer"]}},"required":["type"],"title":"ValueTransformerDefinition","type":"object"},"ValueTransformerDefinitionWithImplicitType":{"additionalProperties":false,"description":"Defines a value transformer function, that gets injected into the Kafka Streams topology","properties":{"code":{"anyOf":[{"type":"boolean"},{"type":"number"},{"type":"string"}],"description":"*(optional)* The (multiline) code of the value transformer"},"expression":{"anyOf":[{"type":"boolean"},{"type":"number"},{"type":"string"}],"description":"*(optional)* The expression returned by the value transformer. Only required for functions that return values."},"globalCode":{"anyOf":[{"type":"boolean"},{"type":"number"},{"type":"string"}],"description":"*(optional)* Global (multiline) code that gets loaded into the Python context outside of the value transformer. Can be used for defining eg. global variables."},"name":{"description":"*(optional)* The name of the value transformer. If this field is not defined, then the name is derived from the context.","type":"string"},"parameters":{"description":"*(optional)* A list of parameters to be passed into the value transformer","items":{"$ref":"#/definitions/ParameterDefinition"},"type":"array"},"resultType":{"description":"*(optional)* The data type returned by the value transformer. Only required for function types, which are not pre-defined.","type":"string"},"stores":{"description":"*(optional)* A list of store names that the value transformer uses. Only required if the function wants to use a state store.","items":{"type":"string"},"type":"array"}},"title":"ValueTransformerDefinitionWithImplicitType","type":"object"},"WindowBySessionOperation":{"additionalProperties":false,"description":"Operation to window messages by session, configured by an inactivity gap","properties":{"grace":{"anyOf":[{"type":"number"},{"type":"string"}],"description":"*(optional)* (Tumbling, Hopping) The grace period, during which out-of-order records can still be processed"},"inactivityGap":{"anyOf":[{"type":"number"},{"type":"string"}],"description":"The inactivity gap, below which two messages are considered to be of the same session"},"name":{"description":"*(optional)* The name of the operation processor","type":"string"},"type":{"description":"The type of the operation","enum":["windowBySession"]}},"required":["inactivityGap","type"],"title":"WindowBySessionOperation","type":"object"},"WindowByTimeOperationWithHoppingWindow":{"additionalProperties":false,"description":"Operation to window records based on time criteria","properties":{"advanceBy":{"anyOf":[{"type":"number"},{"type":"string"}],"description":"The amount of time to increase time windows by"},"duration":{"anyOf":[{"type":"number"},{"type":"string"}],"description":"The duration of time windows"},"grace":{"anyOf":[{"type":"number"},{"type":"string"}],"description":"*(optional)* The grace period, during which out-of-order records can still be processed"},"name":{"description":"*(optional)* The name of the operation processor","type":"string"},"type":{"description":"The type of the operation","enum":["windowByTime"]},"windowType":{"description":"The windowType of the time window","enum":["hopping"]}},"required":["duration","advanceBy","windowType","type"],"title":"WindowByTimeOperationWithHoppingWindow","type":"object"},"WindowByTimeOperationWithSlidingWindow":{"additionalProperties":false,"description":"Operation to window records based on time criteria","properties":{"grace":{"anyOf":[{"type":"number"},{"type":"string"}],"description":"*(optional)* The grace period, during which out-of-order records can still be processed"},"name":{"description":"*(optional)* The name of the operation processor","type":"string"},"timeDifference":{"anyOf":[{"type":"number"},{"type":"string"}],"description":"The maximum amount of time difference between two records"},"type":{"description":"The type of the operation","enum":["windowByTime"]},"windowType":{"description":"The windowType of the time window","enum":["sliding"]}},"required":["timeDifference","windowType","type"],"title":"WindowByTimeOperationWithSlidingWindow","type":"object"},"WindowByTimeOperationWithTumblingWindow":{"additionalProperties":false,"description":"Operation to window records based on time criteria","properties":{"duration":{"anyOf":[{"type":"number"},{"type":"string"}],"description":"The duration of time windows"},"grace":{"anyOf":[{"type":"number"},{"type":"string"}],"description":"*(optional)* The grace period, during which out-of-order records can still be processed"},"name":{"description":"*(optional)* The name of the operation processor","type":"string"},"type":{"description":"The type of the operation","enum":["windowByTime"]},"windowType":{"description":"The windowType of the time window","enum":["tumbling"]}},"required":["duration","windowType","type"],"title":"WindowByTimeOperationWithTumblingWindow","type":"object"},"WindowStateStoreDefinition":{"additionalProperties":false,"description":"Definition of a window state store","properties":{"caching":{"description":"*(optional)* \"true\" if changed to the window store need to be buffered and periodically released, \"false\" to emit all changes directly","type":"boolean"},"keyType":{"description":"*(optional)* The key type of the window store","type":"string"},"logging":{"description":"*(optional)* \"true\" if a changelog topic should be set up on Kafka for this window store, \"false\" otherwise","type":"boolean"},"name":{"description":"*(optional)* The name of the window store. If this field is not defined, then the name is derived from the context.","type":"string"},"persistent":{"description":"*(optional)* \"true\" if this window store needs to be stored on disk, \"false\" otherwise","type":"boolean"},"retainDuplicates":{"description":"*(optional)* Whether or not to retain duplicates","type":"boolean"},"retention":{"anyOf":[{"type":"number"},{"type":"string"}],"description":"*(optional)* The duration for which elements in the window store are retained"},"timestamped":{"description":"*(optional)* \"true\" if elements in the store are timestamped, \"false\" otherwise","type":"boolean"},"type":{"description":"The type of the state store","enum":["window"]},"valueType":{"description":"*(optional)* The value type of the window store","type":"string"},"windowSize":{"anyOf":[{"type":"number"},{"type":"string"}],"description":"*(optional)* Size of the windows (cannot be negative)"}},"required":["type"],"title":"WindowStateStoreDefinition","type":"object"},"WindowStateStoreDefinitionWithImplicitType":{"additionalProperties":false,"description":"Definition of a window state store","properties":{"caching":{"description":"*(optional)* \"true\" if changed to the window store need to be buffered and periodically released, \"false\" to emit all changes directly","type":"boolean"},"keyType":{"description":"*(optional)* The key type of the window store","type":"string"},"logging":{"description":"*(optional)* \"true\" if a changelog topic should be set up on Kafka for this window store, \"false\" otherwise","type":"boolean"},"name":{"description":"*(optional)* The name of the window store. If this field is not defined, then the name is derived from the context.","type":"string"},"persistent":{"description":"*(optional)* \"true\" if this window store needs to be stored on disk, \"false\" otherwise","type":"boolean"},"retainDuplicates":{"description":"*(optional)* Whether or not to retain duplicates","type":"boolean"},"retention":{"anyOf":[{"type":"number"},{"type":"string"}],"description":"*(optional)* The duration for which elements in the window store are retained"},"timestamped":{"description":"*(optional)* \"true\" if elements in the store are timestamped, \"false\" otherwise","type":"boolean"},"type":{"description":"The type of the state store","enum":["window"]},"valueType":{"description":"*(optional)* The value type of the window store","type":"string"},"windowSize":{"anyOf":[{"type":"number"},{"type":"string"}],"description":"*(optional)* Size of the windows (cannot be negative)"}},"title":"WindowStateStoreDefinitionWithImplicitType","type":"object"}},"description":"KSML definition","properties":{"functions":{"description":"*(optional)* Functions that can be referenced in producers and pipelines","patternProperties":{"^[a-zA-Z0-9_]+$":{"anyOf":[{"$ref":"#/definitions/AggregatorDefinition"},{"$ref":"#/definitions/ForEachActionDefinition"},{"$ref":"#/definitions/ForeignKeyExtractorDefinition"},{"$ref":"#/definitions/GeneratorDefinition"},{"$ref":"#/definitions/GenericFunctionDefinitionWithImplicitType"},{"$ref":"#/definitions/InitializerDefinition"},{"$ref":"#/definitions/KeyTransformerDefinition"},{"$ref":"#/definitions/KeyValueMapperDefinition"},{"$ref":"#/definitions/KeyValuePrinterDefinition"},{"$ref":"#/definitions/KeyValueToKeyValueListTransformerDefinition"},{"$ref":"#/definitions/KeyValueToValueListTransformerDefinition"},{"$ref":"#/definitions/KeyValueTransformerDefinition"},{"$ref":"#/definitions/MergerDefinition"},{"$ref":"#/definitions/MetadataTransformerDefinition"},{"$ref":"#/definitions/PredicateDefinition"},{"$ref":"#/definitions/ReducerDefinition"},{"$ref":"#/definitions/StreamPartitionerDefinition"},{"$ref":"#/definitions/TimestampExtractorDefinition"},{"$ref":"#/definitions/TopicNameExtractorDefinition"},{"$ref":"#/definitions/ValueJoinerDefinition"},{"$ref":"#/definitions/ValueTransformerDefinition"}]}},"type":"object"},"globalTables":{"description":"*(optional)* GlobalTables that can be referenced in producers and pipelines","patternProperties":{"^[a-zA-Z0-9_]+$":{"$ref":"#/definitions/GlobalTableDefinitionSource"}},"type":"object"},"pipelines":{"description":"*(optional)* Collection of named pipelines","patternProperties":{"^[a-zA-Z0-9_]+$":{"$ref":"#/definitions/PipelineDefinition"}},"type":"object"},"producers":{"description":"*(optional)* Collection of named producers","patternProperties":{"^[a-zA-Z0-9_]+$":{"$ref":"#/definitions/ProducerDefinition"}},"type":"object"},"stores":{"description":"*(optional)* State stores that can be referenced in producers and pipelines","patternProperties":{"^[a-zA-Z0-9_]+$":{"anyOf":[{"$ref":"#/definitions/KeyValueStateStoreDefinition"},{"$ref":"#/definitions/SessionStateStoreDefinition"},{"$ref":"#/definitions/WindowStateStoreDefinition"}]}},"type":"object"},"streams":{"description":"*(optional)* Streams that can be referenced in producers and pipelines","patternProperties":{"^[a-zA-Z0-9_]+$":{"$ref":"#/definitions/StreamDefinitionSource"}},"type":"object"},"tables":{"description":"*(optional)* Tables that can be referenced in producers and pipelines","patternProperties":{"^[a-zA-Z0-9_]+$":{"$ref":"#/definitions/TableDefinitionSource"}},"type":"object"}},"title":"TopologyDefinition","type":"object"} diff --git a/docs/ksml-language-spec.md b/docs/ksml-language-spec.md index c5c9d935..9063f60e 100644 --- a/docs/ksml-language-spec.md +++ b/docs/ksml-language-spec.md @@ -93,8 +93,6 @@ - **`parameters`** *(array)*: *(optional)* A list of parameters to be passed into the aggregator. - **Items**: Refer to *[#/definitions/ParameterDefinition](#definitions/ParameterDefinition)*. - **`resultType`** *(string)*: *(optional)* The data type returned by the aggregator. Only required for function types, which are not pre-defined. - - **`stores`** *(array)*: *(optional)* A list of store names that the aggregator uses. Only required if the function wants to use a state store. - - **Items** *(string)* - **`type`**: The type of the function. Must be one of: `["aggregator"]`. - **`AggregatorDefinitionWithImplicitType`** *(object)*: Defines a aggregator function, that gets injected into the Kafka Streams topology. Cannot contain additional properties. - **`code`**: *(optional)* The (multiline) code of the aggregator. @@ -116,8 +114,6 @@ - **`parameters`** *(array)*: *(optional)* A list of parameters to be passed into the aggregator. - **Items**: Refer to *[#/definitions/ParameterDefinition](#definitions/ParameterDefinition)*. - **`resultType`** *(string)*: *(optional)* The data type returned by the aggregator. Only required for function types, which are not pre-defined. - - **`stores`** *(array)*: *(optional)* A list of store names that the aggregator uses. Only required if the function wants to use a state store. - - **Items** *(string)* - **`BranchDefinitionWithPipeline`** *(object)*: Defines a branch with sub-pipeline in a BranchOperation. Cannot contain additional properties. - **`as`** *(string)*: *(optional)* The name to register the pipeline result under, which can be used as source by follow-up pipelines. - **`branch`** *(array)*: *(optional)* Defines a single branch, consisting of a condition and a pipeline to execute for messages that fulfil the predicate. @@ -132,8 +128,6 @@ - : Refer to *[#/definitions/PredicateDefinitionWithImplicitType](#definitions/PredicateDefinitionWithImplicitType)*. - **`name`** *(string)*: *(optional)* The name of the operation processor. - **`print`**: *(optional)* The specification of where to print messages to. Refer to *[#/definitions/PrintOperation](#definitions/PrintOperation)*. - - **`stores`** *(array)*: *(optional)* The names of all state stores used by the function. - - **Items** *(string)* - **`to`**: *(optional)* Ends the pipeline by sending all messages to a stream, table or globalTable, or to an inline defined output topic and optional partitioner. - **Any of** - *string* @@ -165,9 +159,11 @@ - : Refer to *[#/definitions/OuterJoinOperationWithStream](#definitions/OuterJoinOperationWithStream)*. - : Refer to *[#/definitions/OuterJoinOperationWithTable](#definitions/OuterJoinOperationWithTable)*. - : Refer to *[#/definitions/PeekOperation](#definitions/PeekOperation)*. - - : Refer to *[#/definitions/ReduceOperation](#definitions/ReduceOperation)*. + - : Refer to *[#/definitions/ReduceOperationWithAdderAndSubtractor](#definitions/ReduceOperationWithAdderAndSubtractor)*. + - : Refer to *[#/definitions/ReduceOperationWithReducer](#definitions/ReduceOperationWithReducer)*. - : Refer to *[#/definitions/RepartitionOperation](#definitions/RepartitionOperation)*. - - : Refer to *[#/definitions/SuppressOperation](#definitions/SuppressOperation)*. + - : Refer to *[#/definitions/SuppressOperationUntilTimeLimit](#definitions/SuppressOperationUntilTimeLimit)*. + - : Refer to *[#/definitions/SuppressOperationUntilWindowCloses](#definitions/SuppressOperationUntilWindowCloses)*. - : Refer to *[#/definitions/ToStreamOperation](#definitions/ToStreamOperation)*. - : Refer to *[#/definitions/ToTableOperation](#definitions/ToTableOperation)*. - : Refer to *[#/definitions/TransformKeyOperation](#definitions/TransformKeyOperation)*. @@ -177,7 +173,9 @@ - : Refer to *[#/definitions/TransformMetadataOperation](#definitions/TransformMetadataOperation)*. - : Refer to *[#/definitions/TransformValueOperation](#definitions/TransformValueOperation)*. - : Refer to *[#/definitions/WindowBySessionOperation](#definitions/WindowBySessionOperation)*. - - : Refer to *[#/definitions/WindowByTimeOperation](#definitions/WindowByTimeOperation)*. + - : Refer to *[#/definitions/WindowByTimeOperationWithHoppingWindow](#definitions/WindowByTimeOperationWithHoppingWindow)*. + - : Refer to *[#/definitions/WindowByTimeOperationWithSlidingWindow](#definitions/WindowByTimeOperationWithSlidingWindow)*. + - : Refer to *[#/definitions/WindowByTimeOperationWithTumblingWindow](#definitions/WindowByTimeOperationWithTumblingWindow)*. - **`CogroupOperation`** *(object)*: A cogroup operation. Cannot contain additional properties. - **`aggregator`**: (GroupedStream, SessionWindowedStream, TimeWindowedStream) The aggregator function, which combines a value with the previous aggregation result and outputs a new aggregation result. - **Any of** @@ -220,8 +218,6 @@ - **Any of** - *string* - : Refer to *[#/definitions/KeyValueStateStoreDefinitionWithImplicitType](#definitions/KeyValueStateStoreDefinitionWithImplicitType)*. - - **`stores`** *(array)*: *(optional)* The names of all state stores used by the function. - - **Items** *(string)* - **`type`**: The type of the operation. Must be one of: `["filterNot"]`. - **`FilterOperation`** *(object)*: Filter records based on a predicate function. Cannot contain additional properties. - **`if`**: A function that returns "true" when records are accepted, "false" otherwise. @@ -233,8 +229,6 @@ - **Any of** - *string* - : Refer to *[#/definitions/KeyValueStateStoreDefinitionWithImplicitType](#definitions/KeyValueStateStoreDefinitionWithImplicitType)*. - - **`stores`** *(array)*: *(optional)* The names of all state stores used by the function. - - **Items** *(string)* - **`type`**: The type of the operation. Must be one of: `["filter"]`. - **`ForEachActionDefinition`** *(object)*: Defines a foreach action function, that gets injected into the Kafka Streams topology. Cannot contain additional properties. - **`code`**: *(optional)* The (multiline) code of the foreach action. @@ -301,8 +295,6 @@ - **`parameters`** *(array)*: *(optional)* A list of parameters to be passed into the foreign key extractor. - **Items**: Refer to *[#/definitions/ParameterDefinition](#definitions/ParameterDefinition)*. - **`resultType`** *(string)*: *(optional)* The data type returned by the foreign key extractor. Only required for function types, which are not pre-defined. - - **`stores`** *(array)*: *(optional)* A list of store names that the foreign key extractor uses. Only required if the function wants to use a state store. - - **Items** *(string)* - **`type`**: The type of the function. Must be one of: `["foreignKeyExtractor"]`. - **`GeneratorDefinition`** *(object)*: Defines a message generator function, that gets injected into the Kafka Streams topology. Cannot contain additional properties. - **`code`**: *(optional)* The (multiline) code of the message generator. @@ -324,8 +316,6 @@ - **`parameters`** *(array)*: *(optional)* A list of parameters to be passed into the message generator. - **Items**: Refer to *[#/definitions/ParameterDefinition](#definitions/ParameterDefinition)*. - **`resultType`** *(string)*: *(optional)* The data type returned by the message generator. Only required for function types, which are not pre-defined. - - **`stores`** *(array)*: *(optional)* A list of store names that the message generator uses. Only required if the function wants to use a state store. - - **Items** *(string)* - **`type`**: The type of the function. Must be one of: `["generator"]`. - **`GeneratorDefinitionWithImplicitType`** *(object)*: Defines a message generator function, that gets injected into the Kafka Streams topology. Cannot contain additional properties. - **`code`**: *(optional)* The (multiline) code of the message generator. @@ -347,8 +337,6 @@ - **`parameters`** *(array)*: *(optional)* A list of parameters to be passed into the message generator. - **Items**: Refer to *[#/definitions/ParameterDefinition](#definitions/ParameterDefinition)*. - **`resultType`** *(string)*: *(optional)* The data type returned by the message generator. Only required for function types, which are not pre-defined. - - **`stores`** *(array)*: *(optional)* A list of store names that the message generator uses. Only required if the function wants to use a state store. - - **Items** *(string)* - **`GenericFunctionDefinitionWithImplicitType`** *(object)*: Defines a generic function function, that gets injected into the Kafka Streams topology. Cannot contain additional properties. - **`code`**: *(optional)* The (multiline) code of the generic function. - **Any of** @@ -369,8 +357,6 @@ - **`parameters`** *(array)*: *(optional)* A list of parameters to be passed into the generic function. - **Items**: Refer to *[#/definitions/ParameterDefinition](#definitions/ParameterDefinition)*. - **`resultType`** *(string)*: *(optional)* The data type returned by the generic function. Only required for function types, which are not pre-defined. - - **`stores`** *(array)*: *(optional)* A list of store names that the generic function uses. Only required if the function wants to use a state store. - - **Items** *(string)* - **`type`**: The type of the function. Must be one of: `["generic"]`. - **`GlobalTableDefinition`** *(object)*: Contains a definition of a GlobalTable, which can be referenced by producers and pipelines. Cannot contain additional properties. - **`keyType`** *(string)*: *(optional)* The key type of the global table. @@ -431,8 +417,6 @@ - **`parameters`** *(array)*: *(optional)* A list of parameters to be passed into the initializer. - **Items**: Refer to *[#/definitions/ParameterDefinition](#definitions/ParameterDefinition)*. - **`resultType`** *(string)*: *(optional)* The data type returned by the initializer. Only required for function types, which are not pre-defined. - - **`stores`** *(array)*: *(optional)* A list of store names that the initializer uses. Only required if the function wants to use a state store. - - **Items** *(string)* - **`type`**: The type of the function. Must be one of: `["initializer"]`. - **`InitializerDefinitionWithImplicitType`** *(object)*: Defines a initializer function, that gets injected into the Kafka Streams topology. Cannot contain additional properties. - **`code`**: *(optional)* The (multiline) code of the initializer. @@ -454,24 +438,16 @@ - **`parameters`** *(array)*: *(optional)* A list of parameters to be passed into the initializer. - **Items**: Refer to *[#/definitions/ParameterDefinition](#definitions/ParameterDefinition)*. - **`resultType`** *(string)*: *(optional)* The data type returned by the initializer. Only required for function types, which are not pre-defined. - - **`stores`** *(array)*: *(optional)* A list of store names that the initializer uses. Only required if the function wants to use a state store. - - **Items** *(string)* - **`JoinOperationWithGlobalTable`** *(object)*: Operation to join with a table. Cannot contain additional properties. - **`globalTable`**: A reference to the globalTable, or an inline definition of the globalTable to join with. - **Any of** - *string* - : Refer to *[#/definitions/GlobalTableDefinition](#definitions/GlobalTableDefinition)*. - - **`mapper`**: A function that maps the key value from the stream with the primary key of the globalTable. + - **`mapper`**: A function that maps the key value from the stream to the primary key type of the globalTable. - **Any of** - *string* - : Refer to *[#/definitions/ValueJoinerDefinitionWithImplicitType](#definitions/ValueJoinerDefinitionWithImplicitType)*. - **`name`** *(string)*: *(optional)* The name of the operation processor. - - **`store`**: *(optional)* Materialized view of the joined streams. - - **Any of** - - *string* - - : Refer to *[#/definitions/KeyValueStateStoreDefinition](#definitions/KeyValueStateStoreDefinition)*. - - : Refer to *[#/definitions/SessionStateStoreDefinition](#definitions/SessionStateStoreDefinition)*. - - : Refer to *[#/definitions/WindowStateStoreDefinition](#definitions/WindowStateStoreDefinition)*. - **`type`**: The type of the operation. Must be one of: `["join"]`. - **`valueJoiner`**: A function that joins two values. - **Any of** @@ -600,8 +576,6 @@ - **`parameters`** *(array)*: *(optional)* A list of parameters to be passed into the keyvalue mapper. - **Items**: Refer to *[#/definitions/ParameterDefinition](#definitions/ParameterDefinition)*. - **`resultType`** *(string)*: *(optional)* The data type returned by the keyvalue mapper. Only required for function types, which are not pre-defined. - - **`stores`** *(array)*: *(optional)* A list of store names that the keyvalue mapper uses. Only required if the function wants to use a state store. - - **Items** *(string)* - **`type`**: The type of the function. Must be one of: `["keyValueMapper"]`. - **`KeyValueMapperDefinitionWithImplicitType`** *(object)*: Defines a keyvalue mapper function, that gets injected into the Kafka Streams topology. Cannot contain additional properties. - **`code`**: *(optional)* The (multiline) code of the keyvalue mapper. @@ -623,8 +597,6 @@ - **`parameters`** *(array)*: *(optional)* A list of parameters to be passed into the keyvalue mapper. - **Items**: Refer to *[#/definitions/ParameterDefinition](#definitions/ParameterDefinition)*. - **`resultType`** *(string)*: *(optional)* The data type returned by the keyvalue mapper. Only required for function types, which are not pre-defined. - - **`stores`** *(array)*: *(optional)* A list of store names that the keyvalue mapper uses. Only required if the function wants to use a state store. - - **Items** *(string)* - **`KeyValuePrinterDefinition`** *(object)*: Defines a keyvalue printer function, that gets injected into the Kafka Streams topology. Cannot contain additional properties. - **`code`**: *(optional)* The (multiline) code of the keyvalue printer. - **Any of** @@ -645,8 +617,6 @@ - **`parameters`** *(array)*: *(optional)* A list of parameters to be passed into the keyvalue printer. - **Items**: Refer to *[#/definitions/ParameterDefinition](#definitions/ParameterDefinition)*. - **`resultType`** *(string)*: *(optional)* The data type returned by the keyvalue printer. Only required for function types, which are not pre-defined. - - **`stores`** *(array)*: *(optional)* A list of store names that the keyvalue printer uses. Only required if the function wants to use a state store. - - **Items** *(string)* - **`type`**: The type of the function. Must be one of: `["keyValuePrinter"]`. - **`KeyValuePrinterDefinitionWithImplicitType`** *(object)*: Defines a keyvalue printer function, that gets injected into the Kafka Streams topology. Cannot contain additional properties. - **`code`**: *(optional)* The (multiline) code of the keyvalue printer. @@ -668,8 +638,6 @@ - **`parameters`** *(array)*: *(optional)* A list of parameters to be passed into the keyvalue printer. - **Items**: Refer to *[#/definitions/ParameterDefinition](#definitions/ParameterDefinition)*. - **`resultType`** *(string)*: *(optional)* The data type returned by the keyvalue printer. Only required for function types, which are not pre-defined. - - **`stores`** *(array)*: *(optional)* A list of store names that the keyvalue printer uses. Only required if the function wants to use a state store. - - **Items** *(string)* - **`KeyValueStateStoreDefinition`** *(object)*: Definition of a keyValue state store. Cannot contain additional properties. - **`caching`** *(boolean)*: *(optional)* "true" if changed to the keyValue store need to be buffered and periodically released, "false" to emit all changes directly. - **`historyRetention`**: *(optional)* (Versioned only) The duration for which old record versions are available for query (cannot be negative). @@ -851,12 +819,6 @@ - *string* - : Refer to *[#/definitions/ValueJoinerDefinitionWithImplicitType](#definitions/ValueJoinerDefinitionWithImplicitType)*. - **`name`** *(string)*: *(optional)* The name of the operation processor. - - **`store`**: *(optional)* Materialized view of the leftJoined streams. - - **Any of** - - *string* - - : Refer to *[#/definitions/KeyValueStateStoreDefinition](#definitions/KeyValueStateStoreDefinition)*. - - : Refer to *[#/definitions/SessionStateStoreDefinition](#definitions/SessionStateStoreDefinition)*. - - : Refer to *[#/definitions/WindowStateStoreDefinition](#definitions/WindowStateStoreDefinition)*. - **`type`**: The type of the operation. Must be one of: `["leftJoin"]`. - **`valueJoiner`**: A function that joins two values. - **Any of** @@ -947,8 +909,6 @@ - **`parameters`** *(array)*: *(optional)* A list of parameters to be passed into the merger. - **Items**: Refer to *[#/definitions/ParameterDefinition](#definitions/ParameterDefinition)*. - **`resultType`** *(string)*: *(optional)* The data type returned by the merger. Only required for function types, which are not pre-defined. - - **`stores`** *(array)*: *(optional)* A list of store names that the merger uses. Only required if the function wants to use a state store. - - **Items** *(string)* - **`type`**: The type of the function. Must be one of: `["merger"]`. - **`MergerDefinitionWithImplicitType`** *(object)*: Defines a merger function, that gets injected into the Kafka Streams topology. Cannot contain additional properties. - **`code`**: *(optional)* The (multiline) code of the merger. @@ -970,8 +930,6 @@ - **`parameters`** *(array)*: *(optional)* A list of parameters to be passed into the merger. - **Items**: Refer to *[#/definitions/ParameterDefinition](#definitions/ParameterDefinition)*. - **`resultType`** *(string)*: *(optional)* The data type returned by the merger. Only required for function types, which are not pre-defined. - - **`stores`** *(array)*: *(optional)* A list of store names that the merger uses. Only required if the function wants to use a state store. - - **Items** *(string)* - **`MetadataTransformerDefinition`** *(object)*: Defines a metadata transformer function, that gets injected into the Kafka Streams topology. Cannot contain additional properties. - **`code`**: *(optional)* The (multiline) code of the metadata transformer. - **Any of** @@ -1069,8 +1027,6 @@ - *string* - : Refer to *[#/definitions/ForEachActionDefinitionWithImplicitType](#definitions/ForEachActionDefinitionWithImplicitType)*. - **`name`** *(string)*: *(optional)* The name of the operation processor. - - **`stores`** *(array)*: *(optional)* The names of all state stores used by the function. - - **Items** *(string)* - **`type`**: The type of the operation. Must be one of: `["peek"]`. - **`PipelineDefinition`** *(object)*: Defines a pipeline through a source, a series of operations to perform on it and a sink operation to close the stream with. Cannot contain additional properties. - **`as`** *(string)*: *(optional)* The name to register the pipeline result under, which can be used as source by follow-up pipelines. @@ -1086,8 +1042,6 @@ - : Refer to *[#/definitions/TopicDefinitionSource](#definitions/TopicDefinitionSource)*. - **`name`** *(string)*: *(optional)* The name of the operation processor. - **`print`**: *(optional)* The specification of where to print messages to. Refer to *[#/definitions/PrintOperation](#definitions/PrintOperation)*. - - **`stores`** *(array)*: *(optional)* The names of all state stores used by the function. - - **Items** *(string)* - **`to`**: *(optional)* Ends the pipeline by sending all messages to a stream, table or globalTable, or to an inline defined output topic and optional partitioner. - **Any of** - *string* @@ -1119,9 +1073,11 @@ - : Refer to *[#/definitions/OuterJoinOperationWithStream](#definitions/OuterJoinOperationWithStream)*. - : Refer to *[#/definitions/OuterJoinOperationWithTable](#definitions/OuterJoinOperationWithTable)*. - : Refer to *[#/definitions/PeekOperation](#definitions/PeekOperation)*. - - : Refer to *[#/definitions/ReduceOperation](#definitions/ReduceOperation)*. + - : Refer to *[#/definitions/ReduceOperationWithAdderAndSubtractor](#definitions/ReduceOperationWithAdderAndSubtractor)*. + - : Refer to *[#/definitions/ReduceOperationWithReducer](#definitions/ReduceOperationWithReducer)*. - : Refer to *[#/definitions/RepartitionOperation](#definitions/RepartitionOperation)*. - - : Refer to *[#/definitions/SuppressOperation](#definitions/SuppressOperation)*. + - : Refer to *[#/definitions/SuppressOperationUntilTimeLimit](#definitions/SuppressOperationUntilTimeLimit)*. + - : Refer to *[#/definitions/SuppressOperationUntilWindowCloses](#definitions/SuppressOperationUntilWindowCloses)*. - : Refer to *[#/definitions/ToStreamOperation](#definitions/ToStreamOperation)*. - : Refer to *[#/definitions/ToTableOperation](#definitions/ToTableOperation)*. - : Refer to *[#/definitions/TransformKeyOperation](#definitions/TransformKeyOperation)*. @@ -1131,51 +1087,53 @@ - : Refer to *[#/definitions/TransformMetadataOperation](#definitions/TransformMetadataOperation)*. - : Refer to *[#/definitions/TransformValueOperation](#definitions/TransformValueOperation)*. - : Refer to *[#/definitions/WindowBySessionOperation](#definitions/WindowBySessionOperation)*. - - : Refer to *[#/definitions/WindowByTimeOperation](#definitions/WindowByTimeOperation)*. -- **`PredicateDefinition`** *(object)*: Defines a Function that returns true or false based on key/value input function, that gets injected into the Kafka Streams topology. Cannot contain additional properties. - - **`code`**: *(optional)* The (multiline) code of the Function that returns true or false based on key/value input. + - : Refer to *[#/definitions/WindowByTimeOperationWithHoppingWindow](#definitions/WindowByTimeOperationWithHoppingWindow)*. + - : Refer to *[#/definitions/WindowByTimeOperationWithSlidingWindow](#definitions/WindowByTimeOperationWithSlidingWindow)*. + - : Refer to *[#/definitions/WindowByTimeOperationWithTumblingWindow](#definitions/WindowByTimeOperationWithTumblingWindow)*. +- **`PredicateDefinition`** *(object)*: Defines a predicate function, that gets injected into the Kafka Streams topology. Cannot contain additional properties. + - **`code`**: *(optional)* The (multiline) code of the predicate. - **Any of** - *boolean* - *number* - *string* - - **`expression`**: *(optional)* The expression returned by the Function that returns true or false based on key/value input. Only required for functions that return values. + - **`expression`**: *(optional)* The expression returned by the predicate. Only required for functions that return values. - **Any of** - *boolean* - *number* - *string* - - **`globalCode`**: *(optional)* Global (multiline) code that gets loaded into the Python context outside of the Function that returns true or false based on key/value input. Can be used for defining eg. global variables. + - **`globalCode`**: *(optional)* Global (multiline) code that gets loaded into the Python context outside of the predicate. Can be used for defining eg. global variables. - **Any of** - *boolean* - *number* - *string* - - **`name`** *(string)*: *(optional)* The name of the Function that returns true or false based on key/value input. If this field is not defined, then the name is derived from the context. - - **`parameters`** *(array)*: *(optional)* A list of parameters to be passed into the Function that returns true or false based on key/value input. + - **`name`** *(string)*: *(optional)* The name of the predicate. If this field is not defined, then the name is derived from the context. + - **`parameters`** *(array)*: *(optional)* A list of parameters to be passed into the predicate. - **Items**: Refer to *[#/definitions/ParameterDefinition](#definitions/ParameterDefinition)*. - - **`resultType`** *(string)*: *(optional)* The data type returned by the Function that returns true or false based on key/value input. Only required for function types, which are not pre-defined. - - **`stores`** *(array)*: *(optional)* A list of store names that the Function that returns true or false based on key/value input uses. Only required if the function wants to use a state store. + - **`resultType`** *(string)*: *(optional)* The data type returned by the predicate. Only required for function types, which are not pre-defined. + - **`stores`** *(array)*: *(optional)* A list of store names that the predicate uses. Only required if the function wants to use a state store. - **Items** *(string)* - **`type`**: The type of the function. Must be one of: `["predicate"]`. -- **`PredicateDefinitionWithImplicitType`** *(object)*: Defines a Function that returns true or false based on key/value input function, that gets injected into the Kafka Streams topology. Cannot contain additional properties. - - **`code`**: *(optional)* The (multiline) code of the Function that returns true or false based on key/value input. +- **`PredicateDefinitionWithImplicitType`** *(object)*: Defines a predicate function, that gets injected into the Kafka Streams topology. Cannot contain additional properties. + - **`code`**: *(optional)* The (multiline) code of the predicate. - **Any of** - *boolean* - *number* - *string* - - **`expression`**: *(optional)* The expression returned by the Function that returns true or false based on key/value input. Only required for functions that return values. + - **`expression`**: *(optional)* The expression returned by the predicate. Only required for functions that return values. - **Any of** - *boolean* - *number* - *string* - - **`globalCode`**: *(optional)* Global (multiline) code that gets loaded into the Python context outside of the Function that returns true or false based on key/value input. Can be used for defining eg. global variables. + - **`globalCode`**: *(optional)* Global (multiline) code that gets loaded into the Python context outside of the predicate. Can be used for defining eg. global variables. - **Any of** - *boolean* - *number* - *string* - - **`name`** *(string)*: *(optional)* The name of the Function that returns true or false based on key/value input. If this field is not defined, then the name is derived from the context. - - **`parameters`** *(array)*: *(optional)* A list of parameters to be passed into the Function that returns true or false based on key/value input. + - **`name`** *(string)*: *(optional)* The name of the predicate. If this field is not defined, then the name is derived from the context. + - **`parameters`** *(array)*: *(optional)* A list of parameters to be passed into the predicate. - **Items**: Refer to *[#/definitions/ParameterDefinition](#definitions/ParameterDefinition)*. - - **`resultType`** *(string)*: *(optional)* The data type returned by the Function that returns true or false based on key/value input. Only required for function types, which are not pre-defined. - - **`stores`** *(array)*: *(optional)* A list of store names that the Function that returns true or false based on key/value input uses. Only required if the function wants to use a state store. + - **`resultType`** *(string)*: *(optional)* The data type returned by the predicate. Only required for function types, which are not pre-defined. + - **`stores`** *(array)*: *(optional)* A list of store names that the predicate uses. Only required if the function wants to use a state store. - **Items** *(string)* - **`PrintOperation`** *(object)*: Operation to print the contents of a pipeline on the screen or to write them to a file. Cannot contain additional properties. - **`filename`** *(string)*: *(optional)* The filename to output records to. If nothing is specified, then messages will be printed on stdout. @@ -1207,16 +1165,12 @@ - **Any of** - *string* - : Refer to *[#/definitions/PredicateDefinitionWithImplicitType](#definitions/PredicateDefinitionWithImplicitType)*. -- **`ReduceOperation`** *(object)*: Operation to reduce a series of records into a single aggregate result. Cannot contain additional properties. +- **`ReduceOperationWithAdderAndSubtractor`** *(object)*: Operation to reduce a series of records into a single aggregate result. Cannot contain additional properties. - **`adder`**: A function that adds a record to the aggregate result. - **Any of** - *string* - : Refer to *[#/definitions/ReducerDefinitionWithImplicitType](#definitions/ReducerDefinitionWithImplicitType)*. - **`name`** *(string)*: *(optional)* The name of the operation processor. - - **`reducer`**: A function that computes a new aggregate result. - - **Any of** - - *string* - - : Refer to *[#/definitions/ReducerDefinitionWithImplicitType](#definitions/ReducerDefinitionWithImplicitType)*. - **`store`**: *(optional)* Materialized view of the aggregation. - **Any of** - *string* @@ -1226,6 +1180,17 @@ - *string* - : Refer to *[#/definitions/ReducerDefinitionWithImplicitType](#definitions/ReducerDefinitionWithImplicitType)*. - **`type`**: The type of the operation. Must be one of: `["reduce"]`. +- **`ReduceOperationWithReducer`** *(object)*: Operation to reduce a series of records into a single aggregate result. Cannot contain additional properties. + - **`name`** *(string)*: *(optional)* The name of the operation processor. + - **`reducer`**: A function that computes a new aggregate result. + - **Any of** + - *string* + - : Refer to *[#/definitions/ReducerDefinitionWithImplicitType](#definitions/ReducerDefinitionWithImplicitType)*. + - **`store`**: *(optional)* Materialized view of the aggregation. + - **Any of** + - *string* + - : Refer to *[#/definitions/WindowStateStoreDefinitionWithImplicitType](#definitions/WindowStateStoreDefinitionWithImplicitType)*. + - **`type`**: The type of the operation. Must be one of: `["reduce"]`. - **`ReducerDefinition`** *(object)*: Defines a reducer function, that gets injected into the Kafka Streams topology. Cannot contain additional properties. - **`code`**: *(optional)* The (multiline) code of the reducer. - **Any of** @@ -1246,8 +1211,6 @@ - **`parameters`** *(array)*: *(optional)* A list of parameters to be passed into the reducer. - **Items**: Refer to *[#/definitions/ParameterDefinition](#definitions/ParameterDefinition)*. - **`resultType`** *(string)*: *(optional)* The data type returned by the reducer. Only required for function types, which are not pre-defined. - - **`stores`** *(array)*: *(optional)* A list of store names that the reducer uses. Only required if the function wants to use a state store. - - **Items** *(string)* - **`type`**: The type of the function. Must be one of: `["reducer"]`. - **`ReducerDefinitionWithImplicitType`** *(object)*: Defines a reducer function, that gets injected into the Kafka Streams topology. Cannot contain additional properties. - **`code`**: *(optional)* The (multiline) code of the reducer. @@ -1269,11 +1232,10 @@ - **`parameters`** *(array)*: *(optional)* A list of parameters to be passed into the reducer. - **Items**: Refer to *[#/definitions/ParameterDefinition](#definitions/ParameterDefinition)*. - **`resultType`** *(string)*: *(optional)* The data type returned by the reducer. Only required for function types, which are not pre-defined. - - **`stores`** *(array)*: *(optional)* A list of store names that the reducer uses. Only required if the function wants to use a state store. - - **Items** *(string)* - **`RepartitionOperation`** *(object)*: Operation to (re)partition a stream. Cannot contain additional properties. - **`name`** *(string)*: *(optional)* The name of the operation processor. - - **`partitioner`**: A function that partitions stream records. + - **`numberOfPartitions`**: *(optional)* The target number of partitions. + - **`partitioner`**: *(optional)* A function that partitions stream records. - **Any of** - *string* - : Refer to *[#/definitions/StreamPartitionerDefinitionWithImplicitType](#definitions/StreamPartitionerDefinitionWithImplicitType)*. @@ -1324,8 +1286,6 @@ - **`parameters`** *(array)*: *(optional)* A list of parameters to be passed into the stream partitioner. - **Items**: Refer to *[#/definitions/ParameterDefinition](#definitions/ParameterDefinition)*. - **`resultType`** *(string)*: *(optional)* The data type returned by the stream partitioner. Only required for function types, which are not pre-defined. - - **`stores`** *(array)*: *(optional)* A list of store names that the stream partitioner uses. Only required if the function wants to use a state store. - - **Items** *(string)* - **`type`**: The type of the function. Must be one of: `["streamPartitioner"]`. - **`StreamPartitionerDefinitionWithImplicitType`** *(object)*: Defines a stream partitioner function, that gets injected into the Kafka Streams topology. Cannot contain additional properties. - **`code`**: *(optional)* The (multiline) code of the stream partitioner. @@ -1347,15 +1307,13 @@ - **`parameters`** *(array)*: *(optional)* A list of parameters to be passed into the stream partitioner. - **Items**: Refer to *[#/definitions/ParameterDefinition](#definitions/ParameterDefinition)*. - **`resultType`** *(string)*: *(optional)* The data type returned by the stream partitioner. Only required for function types, which are not pre-defined. - - **`stores`** *(array)*: *(optional)* A list of store names that the stream partitioner uses. Only required if the function wants to use a state store. - - **Items** *(string)* - **`StringOrInlinePredicateDefinitionWithImplicitType`** *(object)*: Defines the condition under which messages get sent down this branch. Cannot contain additional properties. - **`if`**: *(optional)* Defines the condition under which messages get sent down this branch. - **Any of** - *string* - : Refer to *[#/definitions/PredicateDefinitionWithImplicitType](#definitions/PredicateDefinitionWithImplicitType)*. -- **`SuppressOperation`** *(object)*: Operation to suppress messages in the source stream until a certain limit is reached. Cannot contain additional properties. - - **`bufferFullStrategy`** *(string)*: *(optional)* What to do when the buffer is full, either "emitEarlyWhenFull", or "shutdownWhenFull". +- **`SuppressOperationUntilTimeLimit`** *(object)*: Operation to suppress messages in the source stream until a time limit is reached. Cannot contain additional properties. + - **`bufferFullStrategy`**: *(optional)* What to do when the buffer is full. Must be one of: `["emitEarlyWhenFull", "shutdownWhenFull"]`. - **`duration`**: The duration for which messages are suppressed. - **Any of** - *number* @@ -1364,7 +1322,14 @@ - **`maxRecords`** *(string)*: *(optional)* The maximum number of records in the buffer. - **`name`** *(string)*: *(optional)* The name of the operation processor. - **`type`**: The type of the operation. Must be one of: `["suppress"]`. - - **`until`** *(string, required)*: The method by which messages are held, either "timeLimit", or "windowCloses". + - **`until`**: The until of the Operation to suppress messages in the source stream until a certain limit is reached. Must be one of: `["timeLimit"]`. +- **`SuppressOperationUntilWindowCloses`** *(object)*: Operation to suppress messages in the source stream until a window limit is reached. Cannot contain additional properties. + - **`bufferFullStrategy`**: *(optional)* What to do when the buffer is full. Must be one of: `["emitEarlyWhenFull", "shutdownWhenFull"]`. + - **`maxBytes`** *(string)*: *(optional)* The maximum number of bytes in the buffer. + - **`maxRecords`** *(string)*: *(optional)* The maximum number of records in the buffer. + - **`name`** *(string)*: *(optional)* The name of the operation processor. + - **`type`**: The type of the operation. Must be one of: `["suppress"]`. + - **`until`**: The until of the Operation to suppress messages in the source stream until a certain limit is reached. Must be one of: `["windowCloses"]`. - **`TableDefinition`** *(object)*: Contains a definition of a Table, which can be referenced by producers and pipelines. Cannot contain additional properties. - **`keyType`** *(string)*: *(optional)* The key type of the table. - **`store`**: *(optional)* KeyValue state store definition. @@ -1406,8 +1371,6 @@ - **`parameters`** *(array)*: *(optional)* A list of parameters to be passed into the timestamp extractor. - **Items**: Refer to *[#/definitions/ParameterDefinition](#definitions/ParameterDefinition)*. - **`resultType`** *(string)*: *(optional)* The data type returned by the timestamp extractor. Only required for function types, which are not pre-defined. - - **`stores`** *(array)*: *(optional)* A list of store names that the timestamp extractor uses. Only required if the function wants to use a state store. - - **Items** *(string)* - **`type`**: The type of the function. Must be one of: `["timestampExtractor"]`. - **`TimestampExtractorDefinitionWithImplicitType`** *(object)*: Defines a timestamp extractor function, that gets injected into the Kafka Streams topology. Cannot contain additional properties. - **`code`**: *(optional)* The (multiline) code of the timestamp extractor. @@ -1429,8 +1392,6 @@ - **`parameters`** *(array)*: *(optional)* A list of parameters to be passed into the timestamp extractor. - **Items**: Refer to *[#/definitions/ParameterDefinition](#definitions/ParameterDefinition)*. - **`resultType`** *(string)*: *(optional)* The data type returned by the timestamp extractor. Only required for function types, which are not pre-defined. - - **`stores`** *(array)*: *(optional)* A list of store names that the timestamp extractor uses. Only required if the function wants to use a state store. - - **Items** *(string)* - **`ToStreamOperation`** *(object)*: Convert a Table into a Stream, optionally through a custom key transformer. Cannot contain additional properties. - **`mapper`**: *(optional)* A function that computes the output key for every record. - **Any of** @@ -1495,8 +1456,6 @@ - **`parameters`** *(array)*: *(optional)* A list of parameters to be passed into the topic name extractor. - **Items**: Refer to *[#/definitions/ParameterDefinition](#definitions/ParameterDefinition)*. - **`resultType`** *(string)*: *(optional)* The data type returned by the topic name extractor. Only required for function types, which are not pre-defined. - - **`stores`** *(array)*: *(optional)* A list of store names that the topic name extractor uses. Only required if the function wants to use a state store. - - **Items** *(string)* - **`type`**: The type of the function. Must be one of: `["topicNameExtractor"]`. - **`TopicNameExtractorDefinitionWithImplicitType`** *(object)*: Defines a topic name extractor function, that gets injected into the Kafka Streams topology. Cannot contain additional properties. - **`code`**: *(optional)* The (multiline) code of the topic name extractor. @@ -1518,16 +1477,12 @@ - **`parameters`** *(array)*: *(optional)* A list of parameters to be passed into the topic name extractor. - **Items**: Refer to *[#/definitions/ParameterDefinition](#definitions/ParameterDefinition)*. - **`resultType`** *(string)*: *(optional)* The data type returned by the topic name extractor. Only required for function types, which are not pre-defined. - - **`stores`** *(array)*: *(optional)* A list of store names that the topic name extractor uses. Only required if the function wants to use a state store. - - **Items** *(string)* - **`TransformKeyOperation`** *(object)*: Convert the key of every record in the stream to another key. Cannot contain additional properties. - **`mapper`**: A function that computes a new key for each record. - **Any of** - *string* - : Refer to *[#/definitions/KeyTransformerDefinitionWithImplicitType](#definitions/KeyTransformerDefinitionWithImplicitType)*. - **`name`** *(string)*: *(optional)* The name of the operation processor. - - **`stores`** *(array)*: *(optional)* The names of all state stores used by the function. - - **Items** *(string)* - **`type`**: The type of the operation. Must be one of: `["transformKey", "mapKey", "selectKey"]`. - **`TransformKeyValueOperation`** *(object)*: Convert the key/value of every record in the stream to another key/value. Cannot contain additional properties. - **`mapper`**: A function that computes a new key/value for each record. @@ -1535,8 +1490,6 @@ - *string* - : Refer to *[#/definitions/KeyValueTransformerDefinitionWithImplicitType](#definitions/KeyValueTransformerDefinitionWithImplicitType)*. - **`name`** *(string)*: *(optional)* The name of the operation processor. - - **`stores`** *(array)*: *(optional)* The names of all state stores used by the function. - - **Items** *(string)* - **`type`**: The type of the operation. Must be one of: `["mapKeyValue", "map", "transformKeyValue"]`. - **`TransformKeyValueToKeyValueListOperation`** *(object)*: Convert a stream by transforming every record into a list of derived records. Cannot contain additional properties. - **`mapper`**: A function that converts every record of a stream to a list of output records. @@ -1544,8 +1497,6 @@ - *string* - : Refer to *[#/definitions/KeyValueToKeyValueListTransformerDefinitionWithImplicitType](#definitions/KeyValueToKeyValueListTransformerDefinitionWithImplicitType)*. - **`name`** *(string)*: *(optional)* The name of the operation processor. - - **`stores`** *(array)*: *(optional)* The names of all state stores used by the function. - - **Items** *(string)* - **`type`**: The type of the operation. Must be one of: `["transformKeyValueToKeyValueList", "flatMap"]`. - **`TransformKeyValueToValueListOperation`** *(object)*: Convert every record in the stream to a list of output records with the same key. Cannot contain additional properties. - **`mapper`**: A function that converts every key/value into a list of result values, each of which will be combined with the original key to form a new message in the output stream. @@ -1553,8 +1504,6 @@ - *string* - : Refer to *[#/definitions/KeyValueToValueListTransformerDefinitionWithImplicitType](#definitions/KeyValueToValueListTransformerDefinitionWithImplicitType)*. - **`name`** *(string)*: *(optional)* The name of the operation processor. - - **`stores`** *(array)*: *(optional)* The names of all state stores used by the function. - - **Items** *(string)* - **`type`**: The type of the operation. Must be one of: `["transformKeyValueToValueList", "flatMapValues"]`. - **`TransformMetadataOperation`** *(object)*: Convert the metadata of every record in the stream. Cannot contain additional properties. - **`mapper`**: A function that converts the metadata (Kafka headers, timestamp) of every record in the stream. @@ -1562,8 +1511,6 @@ - *string* - : Refer to *[#/definitions/MetadataTransformerDefinitionWithImplicitType](#definitions/MetadataTransformerDefinitionWithImplicitType)*. - **`name`** *(string)*: *(optional)* The name of the operation processor. - - **`stores`** *(array)*: *(optional)* The names of all state stores used by the function. - - **Items** *(string)* - **`type`**: The type of the operation. Must be one of: `["transformMetadata"]`. - **`TransformValueOperation`** *(object)*: Convert the value of every record in the stream to another value. Cannot contain additional properties. - **`mapper`**: A function that converts the value of every record into another value. @@ -1575,8 +1522,6 @@ - **Any of** - *string* - : Refer to *[#/definitions/KeyValueStateStoreDefinitionWithImplicitType](#definitions/KeyValueStateStoreDefinitionWithImplicitType)*. - - **`stores`** *(array)*: *(optional)* The names of all state stores used by the function. - - **Items** *(string)* - **`type`**: The type of the operation. Must be one of: `["mapValue", "transformValue", "mapValues"]`. - **`ValueJoinerDefinition`** *(object)*: Defines a value joiner function, that gets injected into the Kafka Streams topology. Cannot contain additional properties. - **`code`**: *(optional)* The (multiline) code of the value joiner. @@ -1598,8 +1543,6 @@ - **`parameters`** *(array)*: *(optional)* A list of parameters to be passed into the value joiner. - **Items**: Refer to *[#/definitions/ParameterDefinition](#definitions/ParameterDefinition)*. - **`resultType`** *(string)*: *(optional)* The data type returned by the value joiner. Only required for function types, which are not pre-defined. - - **`stores`** *(array)*: *(optional)* A list of store names that the value joiner uses. Only required if the function wants to use a state store. - - **Items** *(string)* - **`type`**: The type of the function. Must be one of: `["valueJoiner"]`. - **`ValueJoinerDefinitionWithImplicitType`** *(object)*: Defines a value joiner function, that gets injected into the Kafka Streams topology. Cannot contain additional properties. - **`code`**: *(optional)* The (multiline) code of the value joiner. @@ -1621,8 +1564,6 @@ - **`parameters`** *(array)*: *(optional)* A list of parameters to be passed into the value joiner. - **Items**: Refer to *[#/definitions/ParameterDefinition](#definitions/ParameterDefinition)*. - **`resultType`** *(string)*: *(optional)* The data type returned by the value joiner. Only required for function types, which are not pre-defined. - - **`stores`** *(array)*: *(optional)* A list of store names that the value joiner uses. Only required if the function wants to use a state store. - - **Items** *(string)* - **`ValueTransformerDefinition`** *(object)*: Defines a value transformer function, that gets injected into the Kafka Streams topology. Cannot contain additional properties. - **`code`**: *(optional)* The (multiline) code of the value transformer. - **Any of** @@ -1679,26 +1620,46 @@ - *string* - **`name`** *(string)*: *(optional)* The name of the operation processor. - **`type`**: The type of the operation. Must be one of: `["windowBySession"]`. -- **`WindowByTimeOperation`** *(object)*: Operation to reduce a series of records into a single aggregate result. Cannot contain additional properties. - - **`advanceBy`**: *(optional)* (Hopping) The amount of time to increase time windows by. +- **`WindowByTimeOperationWithHoppingWindow`** *(object)*: Operation to window records based on time criteria. Cannot contain additional properties. + - **`advanceBy`**: The amount of time to increase time windows by. - **Any of** - *number* - *string* - - **`duration`**: *(optional)* (Tumbling) The duration of time windows. + - **`duration`**: The duration of time windows. - **Any of** - *number* - *string* - - **`grace`**: *(optional)* (Tumbling, Hopping, Sliding) The grace period, during which out-of-order records can still be processed. + - **`grace`**: *(optional)* The grace period, during which out-of-order records can still be processed. - **Any of** - *number* - *string* - **`name`** *(string)*: *(optional)* The name of the operation processor. - - **`timeDifference`**: *(optional)* (Sliding) The maximum amount of time difference between two records. + - **`type`**: The type of the operation. Must be one of: `["windowByTime"]`. + - **`windowType`**: The windowType of the time window. Must be one of: `["hopping"]`. +- **`WindowByTimeOperationWithSlidingWindow`** *(object)*: Operation to window records based on time criteria. Cannot contain additional properties. + - **`grace`**: *(optional)* The grace period, during which out-of-order records can still be processed. - **Any of** - *number* - *string* + - **`name`** *(string)*: *(optional)* The name of the operation processor. + - **`timeDifference`**: The maximum amount of time difference between two records. + - **Any of** + - *number* + - *string* + - **`type`**: The type of the operation. Must be one of: `["windowByTime"]`. + - **`windowType`**: The windowType of the time window. Must be one of: `["sliding"]`. +- **`WindowByTimeOperationWithTumblingWindow`** *(object)*: Operation to window records based on time criteria. Cannot contain additional properties. + - **`duration`**: The duration of time windows. + - **Any of** + - *number* + - *string* + - **`grace`**: *(optional)* The grace period, during which out-of-order records can still be processed. + - **Any of** + - *number* + - *string* + - **`name`** *(string)*: *(optional)* The name of the operation processor. - **`type`**: The type of the operation. Must be one of: `["windowByTime"]`. - - **`windowType`** *(string, required)*: The type of the operation, either "tumbling", or "hopping", or "sliding". + - **`windowType`**: The windowType of the time window. Must be one of: `["tumbling"]`. - **`WindowStateStoreDefinition`** *(object)*: Definition of a window state store. Cannot contain additional properties. - **`caching`** *(boolean)*: *(optional)* "true" if changed to the window store need to be buffered and periodically released, "false" to emit all changes directly. - **`keyType`** *(string)*: *(optional)* The key type of the window store. diff --git a/docs/notations.md b/docs/notations.md index 392f3537..7f4f65be 100644 --- a/docs/notations.md +++ b/docs/notations.md @@ -1,53 +1,62 @@ # Notations ### Table of Contents + 1. [Introduction](#introduction) -1. [Avro](#avro) -1. [CSV](#csv) -1. [JSON](#json) -1. [SOAP](#soap) -1. [XML](#xml) +2. [Avro](#avro) +3. [CSV](#csv) +4. [JSON](#json) +5. [SOAP](#soap) +6. [XML](#xml) ## Introduction -KSML is able to express its internal data types in a number of external representations. Internally these are called _notations_. -The different notations are described below. +KSML is able to express its internal data types in a number of external representations. Internally these are called +_notations_. The different notations are described below. ## AVRO -Avro types are supported through the "avro" prefix in types. The notation is ```avro:schema```, where schema is the schema fqdn, or just the schema name itself. +Avro types are supported through the "avro" prefix in types. The notation is ```avro:schema```, where schema is the +schema fqdn, or just the schema name itself. On Kafka topics, Avro types are serialized in binary format. Internally they are represented as structs. -Examples +Examples: + ``` avro:SensorData avro:io.axual.ksml.example.SensorData ``` -Note: when referencing an AVRO schema, you have to ensure that the respective schema file can be found in the KSML working directory and has the .avsc file extension. +Note: when referencing an AVRO schema, you have to ensure that the respective schema file can be found in the KSML +working directory and has the .avsc file extension. ## CSV -Comma-separated values are supported through the "csv" prefix in types. The notation is ```csv:schema```, where schema is the schema fqdn, or just the schema name itself. +Comma-separated values are supported through the "csv" prefix in types. The notation is ```csv:schema```, where schema +is the schema fqdn, or just the schema name itself. On Kafka topics, CSV types are serialized as `string`. Internally they are represented as structs. -Examples +Examples: + ``` csv:SensorData csv:io.axual.ksml.example.SensorData ``` -Note: when referencing an CSV schema, you have to ensure that the respective schema file can be found in the KSML working directory and has the .csv file extension. +Note: when referencing an CSV schema, you have to ensure that the respective schema file can be found in the KSML +working directory and has the .csv file extension. ## JSON -JSON types are supported through the "json" prefix in types. The notation is ```json:schema```, where `schema` is the schema fqdn, or just the schema name itself. +JSON types are supported through the "json" prefix in types. The notation is ```json:schema```, where `schema` is the +schema fqdn, or just the schema name itself. On Kafka topics, JSON types are serialized as `string`. Internally they are represented as structs or lists. -Examples +Examples: + ``` json:SensorData json:io.axual.ksml.example.SensorData @@ -58,20 +67,27 @@ If you want to use JSON without a schema, you can leave out the colon and schema ``` json ``` -Note: when referencing an JSON schema, you have to ensure that the respective schema file can be found in the KSML working directory and has the .json file extension. + +Note: when referencing an JSON schema, you have to ensure that the respective schema file can be found in the KSML +working directory and has the .json file extension. ## SOAP -SOAP is supported through built-in serializers and deserializers. The representation on Kafka will always be ```string```. Internally SOAP objects are structs with their own schema. Field names are derived from the SOAP standards. +SOAP is supported through built-in serializers and deserializers. The representation on Kafka will always +be ```string```. Internally SOAP objects are structs with their own schema. Field names are derived from the SOAP +standards. ## XML -XML is supported through built-in serializers and deserializers. The representation on Kafka will always be ```string```. Internally XML objects are structs. +XML is supported through built-in serializers and deserializers. The representation on Kafka will always +be ```string```. Internally XML objects are structs. + +Examples: -Examples ``` xml:SensorData xml:io.axual.ksml.example.SensorData ``` -Note: when referencing an XML schema, you have to ensure that the respective schema file can be found in the KSML working directory and has the .xsd file extension. +Note: when referencing an XML schema, you have to ensure that the respective schema file can be found in the KSML +working directory and has the .xsd file extension. diff --git a/docs/operations.md b/docs/operations.md index 3eb46a60..b2366a8d 100644 --- a/docs/operations.md +++ b/docs/operations.md @@ -3,8 +3,9 @@ ### Table of Contents 1. [Introduction](#introduction) -1. [Operations](#transform-operations) +2. [Operations](#transform-operations) * [aggregate](#aggregate) + * [cogroup](#cogroup) * [convertKey](#convertkey) * [convertKeyValue](#convertkeyvalue) * [convertValue](#convertvalue) @@ -33,79 +34,113 @@ * [transformValue](#transformvalue) * [windowBySession](#windowbysession) * [windowByTime](#windowbytime) -1. [Sink Operations](#sink-operations) +3. [Sink Operations](#sink-operations) + * [as](#as) * [branch](#branch) * [forEach](#foreach) + * [print](#print) * [to](#to) - * [toExtractor](#toextractor) + * [toTopicNameExtractor](#totopicnameextractor) [Duration]: types.md#duration + [Store]: stores.md + [KStream]: https://kafka.apache.org/37/javadoc/org/apache/kafka/streams/kstream/KStream.html + [KTable]: https://kafka.apache.org/37/javadoc/org/apache/kafka/streams/kstream/KTable.html + [GlobalKTable]: https://kafka.apache.org/37/javadoc/org/apache/kafka/streams/kstream/GlobalKTable.html + [KGroupedStream]: https://kafka.apache.org/37/javadoc/org/apache/kafka/streams/kstream/KGroupedStream.html + [KGroupedTable]: https://kafka.apache.org/37/javadoc/org/apache/kafka/streams/kstream/KGroupedTable.html + [SessionWindowedKStream]: https://kafka.apache.org/37/javadoc/org/apache/kafka/streams/kstream/SessionWindowedKStream.html + [SessionWindowedCogroupedKStream]: https://kafka.apache.org/37/javadoc/org/apache/kafka/streams/kstream/SessionWindowedCogroupedKStream.html + [TimeWindowedKStream]: https://kafka.apache.org/37/javadoc/org/apache/kafka/streams/kstream/TimeWindowedKStream.html + [TimeWindowedCogroupedKStream]: https://kafka.apache.org/37/javadoc/org/apache/kafka/streams/kstream/TimeWindowedCogroupedKStream.html [Aggregator]: functions.md#function-types + [Initializer]: functions.md#function-types + [KeyTransformer]: functions.md#function-types + [KeyValueMapper]: functions.md#function-types + [KeyValueToKeyValueListTransformer]: functions.md#function-types + [KeyValueToValueListTransformer]: functions.md#function-types + [KeyValueTransformer]: functions.md#function-types + [Merger]: functions.md#function-types + [Predicate]: functions.md#function-types + [Reducer]: functions.md#function-types + [StreamPartitioner]: functions.md#function-types + [ValueTransformer]: functions.md#function-types + [Windowed]: https://kafka.apache.org/37/javadoc/org/apache/kafka/streams/kstream/Windowed.html ## Introduction -Pipelines in KSML have a beginning, a middle and (optionally) an end. Operations form the middle part of pipelines. They are modeled as separate YAML entities, where each operation takes input from the previous operation and applies its own logic. The returned stream then serves as input for the next operation. +Pipelines in KSML have a beginning, a middle and (optionally) an end. Operations form the middle part of pipelines. They +are modeled as separate YAML entities, where each operation takes input from the previous operation and applies its own +logic. The returned stream then serves as input for the next operation. ## Transform Operations -Transformations are operations that take an input stream and convert it to an output stream. This section lists all supported transformations. Each one states the type of stream it returns. +Transformations are operations that take an input stream and convert it to an output stream. This section lists all +supported transformations. Each one states the type of stream it returns. -| Parameter | Value Type | Description | -|:----------|:-----------|:------------------------------------------| -| `name` | string | The name of the transformation operation. | +| Parameter | Value Type | Description | +|:----------|:-----------|:---------------------------| +| `name` | string | The name of the operation. | -Note that not all combinations of output/input streams are supported by Kafka Streams. The user that writes the KSML definition needs to make sure that streams that result from one operations can actually serve as input to the next. KSML does type checking and will exit with an error when operations that can not be chained together are listed after another in the KSML definition. +Note that not all combinations of output/input streams are supported by Kafka Streams. The user that writes the KSML +definition needs to make sure that streams that result from one operation can actually serve as input to the next. KSML +does type checking and will exit with an error when operations that can not be chained together are listed after another +in the KSML definition. ### aggregate -[KGroupedStream::aggregate]: https://kafka.apache.org/37/javadoc/org/apache/kafka/streams/kstream/KGroupedStream.html#aggregate-org.apache.kafka.streams.kstream.Initializer-org.apache.kafka.streams.kstream.Aggregator-org.apache.kafka.streams.kstream.Materialized- -[KGroupedTable::aggregate]: https://kafka.apache.org/37/javadoc/org/apache/kafka/streams/kstream/KGroupedTable.html#aggregate-org.apache.kafka.streams.kstream.Initializer-org.apache.kafka.streams.kstream.Aggregator-org.apache.kafka.streams.kstream.Aggregator-org.apache.kafka.streams.kstream.Materialized- -[SessionWindowedKStream::aggregate]: https://kafka.apache.org/37/javadoc/org/apache/kafka/streams/kstream/SessionWindowedKStream.html#aggregate-org.apache.kafka.streams.kstream.Initializer-org.apache.kafka.streams.kstream.Aggregator-org.apache.kafka.streams.kstream.Merger-org.apache.kafka.streams.kstream.Named-org.apache.kafka.streams.kstream.Materialized- -[TimeWindowedKStreamObject:aggregate]: https://kafka.apache.org/37/javadoc/org/apache/kafka/streams/kstream/TimeWindowedKStream.html#aggregate-org.apache.kafka.streams.kstream.Initializer-org.apache.kafka.streams.kstream.Aggregator-org.apache.kafka.streams.kstream.Named-org.apache.kafka.streams.kstream.Materialized- - -This operations aggregates multiple values into a single one by repeatedly calling an aggregator function. It can operate on a range of stream types. - -| Stream Type | Returns | Parameter | Value Type | Required | Description | -|:------------------------------------------------------------------------|:---------------------------|:--------------|:--------------------|:---------|:--------------------------------------------------| -| [KGroupedStream][KGroupedStream::aggregate]`` | [KTable]`` | `store` | Store configuration | No | The [Store] configuration. | -| | | `initializer` | Inline or reference | Yes | The [Initializer] function. | -| | | `aggregator` | Inline or reference | Yes | The [Aggregator] function. | -| [KGroupedTable][KGroupedTable::aggregate]`` | [KTable]`` | `store` | Store configuration | No | The [Store] configuration. | -| | | `initializer` | Inline or reference | Yes | The [Initializer] function. | -| | | `adder` | Inline or reference | Yes | The [Reducer] function that adds two values. | -| | | `subtractor` | Inline or reference | Yes | The [Reducer] function that subtracts two values. | -| [SessionWindowedKStream][SessionWindowedKStream::aggregate]`` | [KTable]`,VR>` | `store` | Store configuration | No | The [Store] configuration. | -| | | `initializer` | Inline or reference | Yes | The [Initializer] function. | -| | | `aggregator` | Inline or reference | Yes | The [Aggregator] function. | -| | | `merger` | Inline or reference | Yes | The [Merger] function. | -| [TimeWindowedKStreamObject][TimeWindowedKStreamObject:aggregate]`` | [KTable]`,VR>` | `store` | Store configuration | No | The [Store] configuration. | -| | | `initializer` | Inline or reference | Yes | The [Initializer] function. | -| | | `aggregator` | Inline or reference | Yes | The [Aggregator] function. | +This operation aggregates multiple values into a single one by repeatedly calling an aggregator function. It can +operate on a range of stream types. + +| Stream Type | Returns | Parameter | Value Type | Required | Description | +|:-----------------------------------------|:---------------------------|:--------------|:--------------------|:---------|:---------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------| +| [KGroupedStream]`` | [KTable]`` | `store` | Store configuration | No | An optional [Store] configuration, should be of type `keyValue`. | +| | | `initializer` | Inline or reference | Yes | An [Initializer] function, which takes no arguments and returns a value of type `VR`. | +| | | `aggregator` | Inline or reference | Yes | An [Aggregator] function, which takes a `key` of type `K`, a `value` of type `V` and `aggregatedValue` of type `VR`. It should add the key/value to the previously calculated `aggregateValue` and return a new aggregate value of type `VR`. | +| [KGroupedTable]`` | [KTable]`` | `store` | Store configuration | No | An optional [Store] configuration, should be of type `keyValue`. | +| | | `initializer` | Inline or reference | Yes | An [Initializer] function, which takes no arguments and returns a value of type `VR`. | +| | | `adder` | Inline or reference | Yes | An [Aggregator] function, which takes a `key` of type `K`, a `value` of type `V` and `aggregatedValue` of type `VR`. It should add the key/value to the previously calculated `aggregateValue` and return a new aggregate value of type `VR`. | +| | | `subtractor` | Inline or reference | Yes | An [Aggregator] function, which takes a `key` of type `K`, a `value` of type `V` and `aggregatedValue` of type `VR`. It should remove the key/value from the previously calculated `aggregateValue` and return a new aggregate value of type `VR`. | +| [SessionWindowedKStream]`` | [KTable]`,VR>` | `store` | Store configuration | No | An optional [Store] configuration, should be of type `session`. | +| | | `initializer` | Inline or reference | Yes | An [Initializer] function, which takes no arguments and returns a value of type `VR`. | +| | | `aggregator` | Inline or reference | Yes | An [Aggregator] function, which takes a `key` of type `K`, a `value` of type `V` and `aggregatedValue` of type `VR`. It should add the key/value to the previously calculated `aggregateValue` and return a new aggregate value of type `VR`. | +| | | `merger` | Inline or reference | Yes | A [Merger] function, which takes a `key` of type `K`, and two values `value1` and `value2` of type `V`. The return value is the merged result, also of type `V`. | +| [TimeWindowedKStreamObject]`` | [KTable]`,VR>` | `store` | Store configuration | No | An optional [Store] configuration, should be of type `window`. | +| | | `initializer` | Inline or reference | Yes | An [Initializer] function, which takes no arguments and returns a value of type `VR`. | +| | | `aggregator` | Inline or reference | Yes | An [Aggregator] function, which takes a `key` of type `K`, a `value` of type `V` and `aggregatedValue` of type `VR`. It should add the key/value to the previously calculated `aggregateValue` and return a new aggregate value of type `VR`. | +| [CogroupedKStream]`` | [KTable]`` | `store` | Store configuration | No | An optional [Store] configuration, should be of type `keyValue`. | +| | | `initializer` | Inline or reference | Yes | An [Initializer] function, which takes no arguments and returns a value of type `VR`. | +| [SessionWindowedCogroupedKStream]`` | [KTable]`,VR>` | `store` | Store configuration | No | An optional [Store] configuration, should be of type `session`. | +| | | `initializer` | Inline or reference | Yes | An [Initializer] function, which takes no arguments and returns a value of type `VR`. | +| | | `merger` | Inline or reference | Yes | A [Merger] function, which takes a `key` of type `K`, and two values `value1` and `value2` of type `V`. The return value is the merged result, also of type `V`. | +| [TimeWindowedCogroupedKStream]`` | [KTable]`,VR>` | `store` | Store configuration | No | An optional [Store] configuration, should be of type `window`. | +| | | `initializer` | Inline or reference | Yes | An [Initializer] function, which takes no arguments and returns a value of type `VR`. | Example: + ```yaml from: input_stream via: @@ -115,25 +150,56 @@ via: initializer: expression: 0 aggregator: - expression: value1+value2 + expression: aggregatedValue + value + - type: toStream +to: output_stream +``` + +### cogroup + +This operation cogroups multiple values into a single one by repeatedly calling an aggregator function. It can +operate on a range of stream types. + +| Stream Type | Returns | Parameter | Value Type | Required | Description | +|:--------------------------|:---------------------------|:-------------|:--------------------|:---------|:----------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------| +| [KGroupedStream]`` | [CogroupedKStream]`` | `aggregator` | Inline or reference | Yes | An [Aggregator] function, which takes a `key` of type `K`, a `value` of type `V` and `aggregatedValue` of type `VR`. It should add the key/value to the previously calculated `aggregateValue` and return a new aggregate value of type `VR`. | +| [CogroupedKStream]`` | n/a | n/a | n/a | n/a | This method is currently not supported in KSML. | + +Example: + +```yaml +from: input_stream +via: + - type: groupBy + mapper: my_mapper_function + - type: cogroup + aggregator: + expression: aggregatedValue + value - type: toStream to: output_stream ``` +_Note: this operation was added to KSML for completion purposes, but is not considered ready or fully functional. Feel +free to experiment, but don't rely on this in production. Syntax changes may occur in future KSML releases._ + ### convertKey This built-in operation takes a message and converts the key into a given type. -| Stream Type | Returns | Parameter | Value Type | Description | -|:---------------|:----------------|:----------|:-----------|:-----------------------| -| KStream`` | KStream`` | `into` | string | The type to convert to | +| Stream Type | Returns | Parameter | Value Type | Description | +|:-----------------|:------------------|:----------|:-----------|:----------------------------------------------------------------------| +| [KStream]`` | [KStream]`` | `into` | string | The type to convert the key into. Conversion to `KR` is done by KSML. | Example: + ```yaml -from: input_stream +from: + topic: input_stream + keyType: string + valueType: string via: - type: convertKey - into: string + into: json to: output_stream ``` @@ -141,16 +207,20 @@ to: output_stream This built-in operation takes a message and converts the key and value into a given type. -| Stream Type | Returns | Parameter | Value Type | Description | -|:---------------|:-----------------|:----------|:-----------|:-----------------------| -| KStream`` | KStream`` | `into` | string | The type to convert to | +| Stream Type | Returns | Parameter | Value Type | Description | +|:-----------------|:-------------------|:----------|:-----------|:-------------------------------------------------------------------------------------------------------------| +| [KStream]`` | [KStream]`` | `into` | string | The type to convert the key and value into. Conversion of key into `KR` and value into `VR` is done by KSML. | Example: + ```yaml -from: input_stream +from: + topic: input_stream + keyType: string + valueType: string via: - type: convertKeyValue - into: (string,avro:SensorData) + into: (json,xml) to: output_stream ``` @@ -158,36 +228,37 @@ to: output_stream This built-in operation takes a message and converts the value into a given type. -| Stream Type | Returns | Parameter | Value Type | Description | -|:---------------|:----------------|:----------|:-----------|:-----------------------| -| KStream`` | KStream`` | `into` | string | The type to convert to | +| Stream Type | Returns | Parameter | Value Type | Description | +|:-----------------|:------------------|:----------|:-----------|:-----------------------------------------------------------------------------------| +| [KStream]`` | [KStream]`` | `into` | string | The type to convert the value into. Conversion of value into `VR` is done by KSML. | Example: + ```yaml -from: input_stream +from: + topic: input_stream + keyType: string + valueType: string via: - type: convertValue - into: avro:SensorData + into: xml to: output_stream ``` ### count -[KGroupedStream::count]: https://kafka.apache.org/37/javadoc/org/apache/kafka/streams/kstream/KGroupedStream.html#count-org.apache.kafka.streams.kstream.Named-org.apache.kafka.streams.kstream.Materialized- -[KGroupedTable::count]: https://kafka.apache.org/37/javadoc/org/apache/kafka/streams/kstream/KGroupedTable.html#count-org.apache.kafka.streams.kstream.Named-org.apache.kafka.streams.kstream.Materialized- -[SessionWindowedKStream::count]: https://kafka.apache.org/37/javadoc/org/apache/kafka/streams/kstream/SessionWindowedKStream.html#count-org.apache.kafka.streams.kstream.Named-org.apache.kafka.streams.kstream.Materialized- -[TimeWindowedKStreamObject:count]: https://kafka.apache.org/37/javadoc/org/apache/kafka/streams/kstream/TimeWindowedKStream.html#count-org.apache.kafka.streams.kstream.Named-org.apache.kafka.streams.kstream.Materialized- +This operation counts the number of messages and returns a table multiple values into a single one by repeatedly +calling an aggregator function. It can operate on a range of stream types. -This operations counts the number of messages and returns a table multiple values into a single one by repeatedly calling an aggregator function. It can operate on a range of stream types. - -| Stream Type | Returns | Parameter | Value Type | Required | Description | -|:--------------------------------------------------------------------|:-----------------------------|:----------|:--------------------|:---------|:---------------------------| -| [KGroupedStream][KGroupedStream::count]`` | [KTable]`` | `store` | Store configuration | No | The [Store] configuration. | -| [KGroupedTable][KGroupedTable::count]`` | [KTable]`` | `store` | Store configuration | No | The [Store] configuration. | -| [SessionWindowedKStream][SessionWindowedKStream::count]`` | [KTable]`,Long>` | `store` | Store configuration | No | The [Store] configuration. | -| [TimeWindowedKStreamObject][TimeWindowedKStreamObject:count]`` | [KTable]`,Long>` | `store` | Store configuration | No | The [Store] configuration. | +| Stream Type | Returns | Parameter | Value Type | Required | Description | +|:-----------------------------------|:-----------------------------|:----------|:--------------------|:---------|:-----------------------------------------------------------------| +| [KGroupedStream]`` | [KTable]`` | `store` | Store configuration | No | An optional [Store] configuration, should be of type `keyValue`. | +| [KGroupedTable]`` | [KTable]`` | `store` | Store configuration | No | An optional [Store] configuration, should be of type `keyValue`. | +| [SessionWindowedKStream]`` | [KTable]`,Long>` | `store` | Store configuration | No | An optional [Store] configuration, should be of type `session`. | +| [TimeWindowedKStreamObject]`` | [KTable]`,Long>` | `store` | Store configuration | No | An optional [Store] configuration, should be of type `window`. | Example: + ```yaml from: input_stream via: @@ -200,47 +271,46 @@ to: output_stream ### filter -[KStream::filter]: https://kafka.apache.org/37/javadoc/org/apache/kafka/streams/kstream/KStream.html#filter-org.apache.kafka.streams.kstream.Predicate-org.apache.kafka.streams.kstream.Named- -[KTable::filter]: https://kafka.apache.org/37/javadoc/org/apache/kafka/streams/kstream/KTable.html#filter-org.apache.kafka.streams.kstream.Predicate-org.apache.kafka.streams.kstream.Named-org.apache.kafka.streams.kstream.Materialized- +Filter all incoming messages according to some predicate. The predicate function is called for every message. Only when +the predicate returns `true`, then the message will be sent to the output stream. -Filter all incoming messages according to some predicate. The predicate function is called for every message. Only when the predicate returns `true`, then the message will be sent to the output stream. - -| Stream Type | Returns | Parameter | Value Type | Required | Description | -|:----------------------------------|:-----------------|:------------|:-----------|:--------------------|:--------------------------| -| [KStream][KStream::filter]`` | [KStream]`` | `predicate` | Yes | Inline or reference | The [Predicate] function. | -| [KTable][KTable::filter]`` | [KTable]`` | `predicate` | Yes | Inline or reference | The [Predicate] function. | +| Stream Type | Returns | Parameter | Value Type | Required | Description | +|:-----------------|:-----------------|:----------|:-----------|:--------------------|:----------------------------------------------------------------------------------------------------| +| [KStream]`` | [KStream]`` | `if` | Yes | Inline or reference | A [Predicate] function, which returns `True` if the message can pass the filter, `False` otherwise. | +| | | | | | | +| [KTable]`` | [KTable]`` | `if` | Yes | Inline or reference | A [Predicate] function, which returns `True` if the message can pass the filter, `False` otherwise. | Example: + ```yaml from: input_stream via: - type: filter - predicate: my_filter_function - - type: filter - predicate: + if: expression: key.startswith('a') to: output_stream ``` ### filterNot -This transformation works exactly like [filter](#filter), but negates all predicates before applying them. See [filter](#filter) for details on how to implement. +This operation works exactly like [filter](#filter), but negates all predicates before applying them. That means +messages for which the predicate returns `False` are accepted, while those that the predicate returns `True` for are +filtered out. +See [filter](#filter) for details on how to implement. ### groupBy -[KStream::groupBy]: https://kafka.apache.org/37/javadoc/org/apache/kafka/streams/kstream/KStream.html#groupBy-org.apache.kafka.streams.kstream.KeyValueMapper-org.apache.kafka.streams.kstream.Grouped- -[KTable::groupBy]: https://kafka.apache.org/37/javadoc/org/apache/kafka/streams/kstream/KTable.html#groupBy-org.apache.kafka.streams.kstream.KeyValueMapper-org.apache.kafka.streams.kstream.Grouped- +Group the records of a stream by value resulting from a KeyValueMapper. -Group the records of a stream on a new key that is selected using the provided KeyValueMapper. - -| Stream Type | Returns | Parameter | Value Type | Required | Description | -|:-----------------------------------|:------------------------|:----------|:--------------------|:---------|:-------------------------------| -| [KStream][KStream::groupBy]`` | [KGroupedStream]`` | `store` | Store configuration | No | The [Store] configuration. | -| | | `mapper` | Inline or reference | Yes | The [KeyValueMapper] function. | -| [KTable][KTable::groupBy]`` | [KGroupedTable]`` | `store` | Store configuration | No | The [Store] configuration. | -| | | `mapper` | Inline or reference | Yes | The [KeyValueMapper] function. | +| Stream Type | Returns | Parameter | Value Type | Required | Description | +|:-----------------|:-------------------------|:----------|:--------------------|:---------|:------------------------------------------------------------------------------------------------------------------------------------------------| +| [KStream]`` | [KGroupedStream]`` | `store` | Store configuration | No | An optional [Store] configuration, should be of type `keyValue`. | +| | | `mapper` | Inline or reference | Yes | A [KeyValueMapper] function, which takes a `key` of type `K` and a `value` of type `V` and returns a value of type `KR` to group the stream by. | +| [KTable]`` | [KGroupedTable]`` | `store` | Store configuration | No | An optional [Store] configuration, should be of type `keyValue`. | +| | | `mapper` | Inline or reference | Yes | A [KeyValueMapper] function, which takes a `key` of type `K` and a `value` of type `V` and returns a value of type `KR` to group the stream by. | Example: + ```yaml from: input_stream via: @@ -257,16 +327,14 @@ to: output_stream ### groupByKey -[KStream::groupByKey]: https://kafka.apache.org/37/javadoc/org/apache/kafka/streams/kstream/KStream.html#groupByKey-- +Group the records of a stream by the stream's key. -Group the records of a stream on the stream's key. - -| Stream Type | Returns | Parameter | Value Type | Required | Description | -|:--------------------------------------|:------------------------|:----------|:--------------------|:---------|:-------------------------------| -| [KStream][KStream::groupByKey]`` | [KGroupedStream]`` | `store` | Store configuration | No | The [Store] configuration. | -| | | `mapper` | Inline or reference | Yes | The [KeyValueMapper] function. | +| Stream Type | Returns | Parameter | Value Type | Required | Description | +|:-----------------|:------------------------|:----------|:--------------------|:---------|:-----------------------------------------------------------------| +| [KStream]`` | [KGroupedStream]`` | `store` | Store configuration | No | An optional [Store] configuration, should be of type `keyValue`. | Example: + ```yaml from: input_stream via: @@ -282,79 +350,79 @@ to: output_stream ### join -[KStream::joinStream]: https://kafka.apache.org/37/javadoc/org/apache/kafka/streams/kstream/KStream.html#join-org.apache.kafka.streams.kstream.KStream-org.apache.kafka.streams.kstream.ValueJoiner-org.apache.kafka.streams.kstream.JoinWindows-org.apache.kafka.streams.kstream.StreamJoined- -[KStream::joinTable]: https://kafka.apache.org/37/javadoc/org/apache/kafka/streams/kstream/KStream.html#join-org.apache.kafka.streams.kstream.KTable-org.apache.kafka.streams.kstream.ValueJoiner-org.apache.kafka.streams.kstream.Joined- -[KStream::joinGlobalTable]: https://kafka.apache.org/37/javadoc/org/apache/kafka/streams/kstream/KStream.html#join-org.apache.kafka.streams.kstream.GlobalKTable-org.apache.kafka.streams.kstream.KeyValueMapper-org.apache.kafka.streams.kstream.ValueJoiner-org.apache.kafka.streams.kstream.Named- -[KTable::joinTable]: https://kafka.apache.org/37/javadoc/org/apache/kafka/streams/kstream/KTable.html#join-org.apache.kafka.streams.kstream.KTable-org.apache.kafka.streams.kstream.ValueJoiner-org.apache.kafka.streams.kstream.Named-org.apache.kafka.streams.kstream.Materialized- - -Join records of this stream with another stream's records using windowed inner equi join. The join is computed on the records' key with join predicate `thisKStream.key == otherKStream.key`. Furthermore, two records are only joined if their timestamps are close to each other as defined by the given JoinWindows, i.e., the window defines an additional join predicate on the record timestamps. - -| Stream Type | Returns | Parameter | Value Type | Required | Description | -|:-------------------------------------------|:-----------------|:--------------|:--------------------|:---------|:-------------------------------------------| -| [KStream][KStream::joinStream]`` | [KStream]`` | `store` | Store configuration | No | The [Store] configuration. | -| | | `stream` | `string` | Yes | The name of the stream to join with. | -| | | `valueJoiner` | Inline or reference | Yes | The [KeyValueMapper] function. | -| | | `duration` | `string` | Yes | The [Duration] of the windows to join. | -| [KStream][KStream::joinTable]`` | [KStream]`` | `store` | Store configuration | No | The [Store] configuration. | -| | | `table` | `string` | Yes | The name of the table to join with. | -| | | `valueJoiner` | Inline or reference | Yes | The [KeyValueMapper] function. | -| | | `duration` | `string` | Yes | The [Duration] of the windows to join. | -| [KStream][KStream::joinGlobalTable]`` | [KStream]`` | `store` | Store configuration | No | The [Store] configuration. | -| | | `globalTable` | `string` | Yes | The name of the global table to join with. | -| | | `valueJoiner` | Inline or reference | Yes | The [KeyValueMapper] function. | -| | | `duration` | `string` | Yes | The [Duration] of the windows to join. | -| [KTable][KTable::joinTable]`` | [KTable]`` | `store` | Store configuration | No | The [Store] configuration. | -| | | `table` | `string` | Yes | The name of the table to join with. | -| | | `valueJoiner` | Inline or reference | Yes | The [KeyValueMapper] function. | +Join records of this stream with another stream's records using inner join. The join is computed on the +records' key with join predicate `thisStream.key == otherStream.key`. If both streams are not tables, then +their timestamps need to be close enough as defined by timeDifference. + +| Stream Type | Returns | Parameter | Value Type | Required | Description | +|:-----------------|:------------------|:----------------------|:--------------------|:---------|:---------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------| +| [KStream]`` | [KStream]`` | `store` | Store configuration | No | An optional [Store] configuration, should be of type `window`. | +| | | `stream` | `string` | Yes | The name of the stream to join with. The stream should be of key type `K` and value type `VR`. | +| | | `valueJoiner` | Inline or reference | Yes | A [ValueJoiner] function, which takes a `key` of type `K`, and two values `value1` and `value2` of type `V`. The return value is the joined value of type `VR`. | +| | | `timeDifference` | `duration` | Yes | The maximum allowed between two joined records. | +| | | `grace` | `duration` | No | A grace period during with out-of-order to-be-joined records may still arrive. | +| [KStream]`` | [KStream]`` | `store` | Store configuration | No | An optional [Store] configuration, should be of type `keyValue`. | +| | | `table` | `string` | Yes | The name of the table to join with. The table should be of key type `K` and value type `VO`. | | +| | | `valueJoiner` | Inline or reference | Yes | A [ValueJoiner] function, which takes a `value1` of type `V` from the source table and a `value2` of type `VO` from the join table. The return value is the joined value of type `VR`. | +| | | `grace` | `duration` | No | A grace period during with out-of-order to-be-joined records may still arrive. | +| [KStream]`` | [KStream]`` | `globalTable` | `string` | Yes | The name of the global table to join with. The global table should be of key type `GK` and value type `GV`. | +| | | `mapper` | Inline or reference | Yes | A [KeyValueMapper] function, which takes a `key` of type `K` and a `value` of type `V`. The return value is the key of type `GK` of the records from the GlobalTable to join with. | +| | | `valueJoiner` | Inline or reference | Yes | A [ValueJoiner] function, which takes a `key` of type `K`, and two values `value1` and `value2` of type `V`. The return value is the joined value of type `VR`. | +| [KTable]`` | [KTable]`` | `store` | Store configuration | No | The [Store] configuration. | +| | | `table` | `string` | Yes | The name of the table to join with. The table should be of key type `K` and value type `VO`. | | +| | | `foreignKeyExtractor` | Inline or reference | No | A [ForeignKeyExtractor] function, which takes a `value` of type `V`, which needs to be converted into the key type `KO` of the table to join with. | +| | | `valueJoiner` | Inline or reference | Yes | A [ValueJoiner] function, which takes a `value1` of type `V` from the source table and a `value2` of type `VO` from the join table. The return value is the joined value of type `VR`. | +| | | `partitioner` | Inline or reference | No | A [Partitioner] function, which partitions the records on the primary stream. | | +| | | `otherPartitioner` | Inline or reference | No | A [Partitioner] function, which partitions the records on the join table. | Example: + ```yaml from: input_stream via: - type: join stream: second_stream valueJoiner: my_key_value_mapper - duration: 1m + timeDifference: 1m to: output_stream ``` ### leftJoin -[KStream::leftJoinStream]: https://kafka.apache.org/37/javadoc/org/apache/kafka/streams/kstream/KStream.html#leftJoin-org.apache.kafka.streams.kstream.KStream-org.apache.kafka.streams.kstream.ValueJoiner-org.apache.kafka.streams.kstream.JoinWindows-org.apache.kafka.streams.kstream.StreamJoined- -[KStream::leftJoinTable]: https://kafka.apache.org/37/javadoc/org/apache/kafka/streams/kstream/KStream.html#leftJoin-org.apache.kafka.streams.kstream.KTable-org.apache.kafka.streams.kstream.ValueJoiner-org.apache.kafka.streams.kstream.Joined- -[KStream::leftJoinGlobalTable]: https://kafka.apache.org/37/javadoc/org/apache/kafka/streams/kstream/KStream.html#join-org.apache.kafka.streams.kstream.GlobalKTable-org.apache.kafka.streams.kstream.KeyValueMapper-org.apache.kafka.streams.kstream.ValueJoiner-org.apache.kafka.streams.kstream.Named- -[KTable::leftJoinTable]: https://kafka.apache.org/37/javadoc/org/apache/kafka/streams/kstream/KTable.html#leftJoin-org.apache.kafka.streams.kstream.KTable-org.apache.kafka.streams.kstream.ValueJoiner-org.apache.kafka.streams.kstream.Named-org.apache.kafka.streams.kstream.Materialized- - -Join records of this stream with another stream's records using windowed left equi join. In contrast to inner-join, all records from this stream will produce at least one output record. The join is computed on the records' key with join attribute thisKStream.key == otherKStream.key. Furthermore, two records are only joined if their timestamps are close to each other as defined by the given JoinWindows, i.e., the window defines an additional join predicate on the record timestamps. - -| Stream Type | Returns | Parameter | Value Type | Required | Description | -|:-----------------------------------------------|:-----------------|:--------------|:--------------------|:---------|:-------------------------------------------| -| [KStream][KStream::leftJoinStream]`` | [KStream]`` | `store` | Store configuration | No | The [Store] configuration. | -| | | `stream` | `string` | Yes | The name of the stream to join with. | -| | | `valueJoiner` | Inline or reference | Yes | The [KeyValueMapper] function. | -| | | `duration` | `string` | Yes | The [Duration] of the windows to join. | -| [KStream][KStream::leftJoinTable]`` | [KStream]`` | `store` | Store configuration | No | The [Store] configuration. | -| | | `table` | `string` | Yes | The name of the table to join with. | -| | | `valueJoiner` | Inline or reference | Yes | The [KeyValueMapper] function. | -| | | `duration` | `string` | Yes | The [Duration] of the windows to join. | -| [KStream][KStream::leftJoinGlobalTable]`` | [KStream]`` | `store` | Store configuration | No | The [Store] configuration. | -| | | `globalTable` | `string` | Yes | The name of the global table to join with. | -| | | `valueJoiner` | Inline or reference | Yes | The [KeyValueMapper] function. | -| | | `duration` | `string` | Yes | The [Duration] of the windows to join. | -| [KTable][KTable::leftJoinTable]`` | [KTable]`` | `store` | Store configuration | No | The [Store] configuration. | -| | | `table` | `string` | Yes | The name of the table to join with. | -| | | `valueJoiner` | Inline or reference | Yes | The [KeyValueMapper] function. | +Join records of this stream with another stream's records using left join. The join is computed on the +records' key with join predicate `thisStream.key == otherStream.key`. If both streams are not tables, then +their timestamps need to be close enough as defined by timeDifference. + +| Stream Type | Returns | Parameter | Value Type | Required | Description | +|:-----------------|:------------------|:----------------------|:--------------------|:---------|:---------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------| +| [KStream]`` | [KStream]`` | `store` | Store configuration | No | An optional [Store] configuration, should be of type `window`. | +| | | `stream` | `string` | Yes | The name of the stream to join with. The stream should be of key type `K` and value type `VR`. | +| | | `valueJoiner` | Inline or reference | Yes | A [ValueJoiner] function, which takes a `key` of type `K`, and two values `value1` and `value2` of type `V`. The return value is the joined value of type `VR`. | +| | | `timeDifference` | `duration` | Yes | The maximum allowed between two joined records. | +| | | `grace` | `duration` | No | A grace period during with out-of-order to-be-joined records may still arrive. | +| [KStream]`` | [KStream]`` | `store` | Store configuration | No | An optional [Store] configuration, should be of type `keyValue`. | +| | | `table` | `string` | Yes | The name of the table to join with. The table should be of key type `K` and value type `VO`. | | +| | | `valueJoiner` | Inline or reference | Yes | A [ValueJoiner] function, which takes a `value1` of type `V` from the source table and a `value2` of type `VO` from the join table. The return value is the joined value of type `VR`. | +| | | `grace` | `duration` | No | A grace period during with out-of-order to-be-joined records may still arrive. | +| [KStream]`` | [KStream]`` | `globalTable` | `string` | Yes | The name of the global table to join with. The global table should be of key type `GK` and value type `GV`. | +| | | `mapper` | Inline or reference | Yes | A [KeyValueMapper] function, which takes a `key` of type `K` and a `value` of type `V`. The return value is the key of type `GK` of the records from the GlobalTable to join with. | +| | | `valueJoiner` | Inline or reference | Yes | A [ValueJoiner] function, which takes a `key` of type `K`, and two values `value1` and `value2` of type `V`. The return value is the joined value of type `VR`. | +| [KTable]`` | [KTable]`` | `store` | Store configuration | No | The [Store] configuration. | +| | | `table` | `string` | Yes | The name of the table to join with. The table should be of key type `K` and value type `VO`. | | +| | | `foreignKeyExtractor` | Inline or reference | No | A [ForeignKeyExtractor] function, which takes a `value` of type `V`, which needs to be converted into the key type `KO` of the table to join with. | +| | | `valueJoiner` | Inline or reference | Yes | A [ValueJoiner] function, which takes a `value1` of type `V` from the source table and a `value2` of type `VO` from the join table. The return value is the joined value of type `VR`. | +| | | `partitioner` | Inline or reference | No | A [Partitioner] function, which partitions the records on the primary stream. | | +| | | `otherPartitioner` | Inline or reference | No | A [Partitioner] function, which partitions the records on the join table. | Example: + ```yaml -[yaml] ----- from: input_stream via: - type: leftJoin stream: second_stream - valueJoiner: my_join_function - duration: 1m + valueJoiner: my_key_value_mapper + timeDifference: 1m to: output_stream ``` @@ -372,15 +440,16 @@ This is an alias for [transformValue](#transformvalue). ### merge -[KStream::merge]: https://kafka.apache.org/37/javadoc/org/apache/kafka/streams/kstream/KStream.html#merge-org.apache.kafka.streams.kstream.KStream-org.apache.kafka.streams.kstream.Named- - -Merge this stream and the given stream into one larger stream. There is no ordering guarantee between records from this stream and records from the provided stream in the merged stream. Relative order is preserved within each input stream though (ie, records within one input stream are processed in order). +Merge this stream and the given stream into one larger stream. There is no ordering guarantee between records from this +stream and records from the provided stream in the merged stream. Relative order is preserved within each input stream +though (ie, records within one input stream are processed in order). -| Stream Type | Returns | Parameter | Value Type | Description | -|:---------------------------------|:-----------------|:----------|:-----------|:--------------------------------------| -| [KStream][KStream::merge]`` | [KStream]`` | `stream` | `string` | The name of the stream to merge with. | +| Stream Type | Returns | Parameter | Value Type | Description | +|:-----------------|:-----------------|:----------|:-----------|:--------------------------------------| +| [KStream]`` | [KStream]`` | `stream` | `string` | The name of the stream to merge with. | Example: + ```yaml from: input_stream via: @@ -391,45 +460,44 @@ to: output_stream ### outerJoin -[KStream::outerJoinStream]: https://kafka.apache.org/37/javadoc/org/apache/kafka/streams/kstream/KStream.html#outerJoin-org.apache.kafka.streams.kstream.KStream-org.apache.kafka.streams.kstream.ValueJoiner-org.apache.kafka.streams.kstream.JoinWindows-org.apache.kafka.streams.kstream.StreamJoined- -[KTable::outerJoinTable]: https://kafka.apache.org/37/javadoc/org/apache/kafka/streams/kstream/KTable.html#outerJoin-org.apache.kafka.streams.kstream.KTable-org.apache.kafka.streams.kstream.ValueJoiner-org.apache.kafka.streams.kstream.Named-org.apache.kafka.streams.kstream.Materialized- - -Join records of this stream with another stream's records using windowed outer equi join. In contrast to inner-join or left-join, all records from both streams will produce at least one output record. The join is computed on the records' key with join attribute thisKStream.key == otherKStream.key. Furthermore, two records are only joined if their timestamps are close to each other as defined by the given JoinWindows, i.e., the window defines an additional join predicate on the record timestamps. - -| Stream Type | Returns | Parameter | Value Type | Required | Description | -|:-------------------------------------------|:-----------------|:--------------|:--------------------|:---------|:---------------------------------------| -| [KStream][KStream::outerJoinStream]`` | [KStream]`` | `store` | Store configuration | No | The [Store] configuration. | -| | | `stream` | `string` | Yes | The name of the stream to join with. | -| | | `valueJoiner` | Inline or reference | Yes | The [KeyValueMapper] function. | -| | | `duration` | `string` | Yes | The [Duration] of the windows to join. | -| [KTable][KTable::outerJoinTable]`` | [KTable]`` | `store` | Store configuration | No | The [Store] configuration. | -| | | `table` | `string` | Yes | The name of the table to join with. | -| | | `valueJoiner` | Inline or reference | Yes | The [KeyValueMapper] function. | +Join records of this stream with another stream's records using outer join. The join is computed on the +records' key with join predicate `thisStream.key == otherStream.key`. If both streams are not tables, then +their timestamps need to be close enough as defined by timeDifference. + +| Stream Type | Returns | Parameter | Value Type | Required | Description | +|:-----------------|:------------------|:-----------------|:--------------------|:---------|:---------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------| +| [KStream]`` | [KStream]`` | `store` | Store configuration | No | An optional [Store] configuration, should be of type `window`. | +| | | `stream` | `string` | Yes | The name of the stream to join with. The stream should be of key type `K` and value type `VR`. | +| | | `valueJoiner` | Inline or reference | Yes | A [ValueJoiner] function, which takes a `key` of type `K`, and two values `value1` and `value2` of type `V`. The return value is the joined value of type `VR`. | +| | | `timeDifference` | `duration` | Yes | The maximum allowed between two joined records. | +| | | `grace` | `duration` | No | A grace period during with out-of-order to-be-joined records may still arrive. | +| [KTable]`` | [KStream]`` | `store` | Store configuration | No | An optional [Store] configuration, should be of type `keyValue`. | +| | | `table` | `string` | Yes | The name of the table to join with. The table should be of key type `K` and value type `VO`. | | +| | | `valueJoiner` | Inline or reference | Yes | A [ValueJoiner] function, which takes a `value1` of type `V` from the source table and a `value2` of type `VO` from the join table. The return value is the joined value of type `VR`. | Example: + ```yaml -[yaml] ----- from: input_stream via: - type: outerJoin stream: second_stream - valueJoiner: my_join_function - duration: 1m + valueJoiner: my_key_value_mapper + timeDifference: 1m to: output_stream ``` ### peek -[KStream::peek]: https://kafka.apache.org/37/javadoc/org/apache/kafka/streams/kstream/KStream.html#peek-org.apache.kafka.streams.kstream.ForeachAction-org.apache.kafka.streams.kstream.Named- - -Perform an action on each record of a stream. This is a stateless record-by-record operation. Peek is a non-terminal operation that triggers a side effect (such as logging or statistics collection) and returns an unchanged stream. +Perform an action on each record of a stream. This is a stateless record-by-record operation. Peek is a non-terminal +operation that triggers a side effect (such as logging or statistics collection) and returns an unchanged stream. -| Stream Type | Returns | Parameter | Value Type | Description | -|:--------------------------------|:-----------------|:----------|:--------------------|:--------------------------------------------------------------| -| [KStream][KStream::peek]`` | [KStream]`` | `forEach` | Inline or reference | The [ForEach] function that will be called for every message. | +| Stream Type | Returns | Parameter | Value Type | Description | +|:-----------------|:-----------------|:----------|:--------------------|:-----------------------------------------------------------------------------------------------------------------------------| +| [KStream]`` | [KStream]`` | `forEach` | Inline or reference | The [ForEach] function that will be called for every message, receiving arguments `key` of type `K` and `value` of type `V`. | Example: + ```yaml from: input_stream via: @@ -440,29 +508,43 @@ to: output_stream ### reduce -[KGroupedStream::reduce]: https://kafka.apache.org/37/javadoc/org/apache/kafka/streams/kstream/KGroupedStream.html#reduce-org.apache.kafka.streams.kstream.Reducer-org.apache.kafka.streams.kstream.Named-org.apache.kafka.streams.kstream.Materialized- -[KGroupedTable::reduce]: https://kafka.apache.org/37/javadoc/org/apache/kafka/streams/kstream/KGroupedTable.html#reduce-org.apache.kafka.streams.kstream.Reducer-org.apache.kafka.streams.kstream.Reducer-org.apache.kafka.streams.kstream.Named-org.apache.kafka.streams.kstream.Materialized- -[SessionWindowedKStream::reduce]: https://kafka.apache.org/37/javadoc/org/apache/kafka/streams/kstream/SessionWindowedKStream.html#reduce-org.apache.kafka.streams.kstream.Reducer-org.apache.kafka.streams.kstream.Named-org.apache.kafka.streams.kstream.Materialized- -[TimeWindowedKStreamObject:reduce]: https://kafka.apache.org/37/javadoc/org/apache/kafka/streams/kstream/TimeWindowedKStream.html#reduce-org.apache.kafka.streams.kstream.Reducer-org.apache.kafka.streams.kstream.Named-org.apache.kafka.streams.kstream.Materialized- - -Combine the values of records in this stream by the grouped key. Records with null key or value are ignored. Combining implies that the type of the aggregate result is the same as the type of the input value, similar to [aggregate(Initializer, Aggregator)](#aggregate). - -| Stream Type | Returns | Parameter | Value Type | Required | Description | -|:---------------------------------------------------------------------|:---------------------------|:--------------|:--------------------|:---------|:--------------------------------------------------| -| [KGroupedStream][KGroupedStream::reduce]`` | [KTable]`` | `store` | Store configuration | No | The [Store] configuration. | -| | | `reducer` | Inline or reference | Yes | The [Reducer] function. | -| [KGroupedTable][KGroupedTable::reduce]`` | [KTable]`` | `store` | Store configuration | No | The [Store] configuration. | -| | | `adder` | Inline or reference | Yes | The [Reducer] function that adds two values. | -| | | `subtractor` | Inline or reference | Yes | The [Reducer] function that subtracts two values. | -| [SessionWindowedKStream][SessionWindowedKStream::reduce]`` | [KTable]`,VR>` | `store` | Store configuration | No | The [Store] configuration. | -| | | `reducer` | Inline or reference | Yes | The [Reducer] function. | -| [TimeWindowedKStreamObject][TimeWindowedKStreamObject:reduce]`` | [KTable]`,VR>` | `store` | Store configuration | No | The [Store] configuration. | -| | | `initializer` | Inline or reference | Yes | The [Reducer] function. | +Combine the values of records in this stream by the grouped key. Records with null key or value are ignored. Combining +implies that the type of the aggregate result is the same as the type of the input value, similar +to [aggregate(Initializer, Aggregator)](#aggregate). + +| Stream Type | Returns | Parameter | Value Type | Required | Description | +|:-----------------------------------|:--------------------------|:-------------|:--------------------|:---------|:---------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------| +| [KGroupedStream]`` | [KTable]`` | `store` | Store configuration | No | An optional [Store] configuration, should be of type `keyValue`. | +| | | `reducer` | Inline or reference | Yes | A [Reducer] function, which takes a `key` of type `K`, a `value` of type `V` and `aggregatedValue` of type `V`. It should add the key/value to the previously calculated `aggregateValue` and return a new aggregate value of type `V`. | +| [KGroupedTable]`` | [KTable]`` | `store` | Store configuration | No | An optional [Store] configuration, should be of type `keyValue`. | +| | | `adder` | Inline or reference | Yes | A [Reducer] function, which takes a `key` of type `K`, a `value` of type `V` and `aggregatedValue` of type `V`. It should add the key/value to the previously calculated `aggregateValue` and return a new aggregate value of type `V`. | +| | | `subtractor` | Inline or reference | Yes | A [Reducer] function, which takes a `key` of type `K`, a `value` of type `V` and `aggregatedValue` of type `V`. It should remove the key/value from the previously calculated `aggregateValue` and return a new aggregate value of type `V`. | +| [SessionWindowedKStream]`` | [KTable]`,V>` | `store` | Store configuration | No | An optional [Store] configuration, should be of type `session`. | +| | | `reducer` | Inline or reference | Yes | A [Reducer] function, which takes a `key` of type `K`, a `value` of type `V` and `aggregatedValue` of type `V`. It should add the key/value to the previously calculated `aggregateValue` and return a new aggregate value of type `V`. | +| [TimeWindowedKStreamObject]`` | [KTable]`,V>` | `store` | Store configuration | No | An optional [Store] configuration, should be of type `window`. | +| | | `reducer` | Inline or reference | Yes | A [Reducer] function, which takes a `key` of type `K`, a `value` of type `V` and `aggregatedValue` of type `V`. It should add the key/value to the previously calculated `aggregateValue` and return a new aggregate value of type `V`. | Example: + ```yaml -[yaml] ----- +from: input_stream +via: + - type: groupBy + mapper: my_mapper_function + - type: aggregate + initializer: + expression: 0 + aggregator: + expression: aggregatedValue + value + - type: toStream +to: output_stream +``` + +Example: + +```yaml +[ yaml ] + ---- from: input_stream via: - type: groupBy @@ -476,23 +558,23 @@ to: output_stream ### repartition -[KStream::repartition]: https://kafka.apache.org/37/javadoc/org/apache/kafka/streams/kstream/KStream.html#repartition-org.apache.kafka.streams.kstream.Repartitioned- - -Materialize this stream to an auto-generated repartition topic and create a new KStream from the auto-generated topic using key serde, value serde, StreamPartitioner, number of partitions, and topic name part. -The created topic is considered as an internal topic and is meant to be used only by the current Kafka Streams instance. Similar to auto-repartitioning, the topic will be created with infinite retention time and data will be automatically purged by Kafka Streams. The topic will be named as "${applicationId}--repartition", where "applicationId" is user-specified in StreamsConfig via parameter APPLICATION_ID_CONFIG, "" is either provided via Repartitioned.as(String) or an internally generated name, and "-repartition" is a fixed suffix. +Materialize this stream to an auto-generated repartition topic with a given number of partitions, using a custom +partitioner. Similar to auto-repartitioning, the topic will be created with infinite retention time and data will be +automatically purged. The topic will be named as "${applicationId}--repartition". -| Stream Type | Returns | Parameter | Value Type | Required | Description | -|:---------------------------------------|:-----------------|:--------------|:--------------------|:---------|:---------------------------------------------------------------| -| [KStream][KStream::repartition]`` | [KStream]`` | `store` | Store configuration | No | The [Store] configuration. | -| | | `name` | `string` | Yes | The name used as part of repartition topic and processor name. | -| | | `partitioner` | Inline or reference | Yes | The [StreamPartitioner] function. | +| Stream Type | Returns | Parameter | Value Type | Required | Description | +|:-----------------|:-----------------|:---------------------|:--------------------|:---------|:------------------------------------------------------| +| [KStream]`` | [KStream]`` | `numberOfPartitions` | integer | No | The number of partitions of the repartitioned topic. | +| | | `partitioner` | Inline or reference | No | A custom [Partitioner] function to partition records. | Example: + ```yaml from: input_stream via: - type: repartition - name: my_repartitioner + name: my_partitioner + numberOfPartitions: 3 partitioner: my_own_partitioner - type: peek forEach: print_key_and_value @@ -506,20 +588,20 @@ This is an alias for [transformKey](#transformkey). ### suppress -[KTable::suppress]: https://kafka.apache.org/37/javadoc/org/apache/kafka/streams/kstream/KTable.html#suppress-org.apache.kafka.streams.kstream.Suppressed- - Suppress some updates from this changelog stream, determined by the supplied Suppressed configuration. When -_windowCloses_ is selected and no further restrictions are provided, then this is interpreted as _Suppressed.untilWindowCloses(unbounded())_. +_windowCloses_ is selected and no further restrictions are provided, then this is interpreted as +_Suppressed.untilWindowCloses(unbounded())_. -| Stream Type | Returns | Parameter | Value Type | Description | -|:----------------------------------|:----------------|:---------------------|:-----------|:---------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------| -| [KTable][KTable::suppress]`` | [KTable]`` | `until` | `string` | This value can either be `timeLimit` or `windowCloses`. Note that _timeLimit_ suppression works on any stream, while _windowCloses_ suppression works only on _Windowed_ streams. For the latter, see [windowedBy](#windowedby). | -| | | `duration` | `string` | The [Duration] to suppress updates (only when `until`==`timeLimit`) | -| | | `maxBytes` | `int` | (Optional) The maximum number of bytes to suppress updates | -| | | `maxRecords` | `int` | (Optional) The maximum number of records to suppress updates | -| | | `bufferFullStrategy` | `string` | (Optional) Can be one of `emitEarlyWhenFull`, `shutdownWhenFull` | +| Stream Type | Returns | Parameter | Value Type | Required | Description | +|:----------------|:----------------|:---------------------|:-----------|:---------|:-------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------| +| [KTable]`` | [KTable]`` | `until` | `string` | Yes | This value can either be `timeLimit` or `windowCloses`. Note that _timeLimit_ suppression works on any stream, while _windowCloses_ suppression works only on _Windowed_ streams. For the latter, see [windowByTime] or [windowBySession]. | +| | | `duration` | `string` | No | The [Duration] to suppress updates (only when `until`==`timeLimit`) | +| | | `maxBytes` | `int` | No | The maximum number of bytes to suppress updates | +| | | `maxRecords` | `int` | No | The maximum number of records to suppress updates | +| | | `bufferFullStrategy` | `string` | No | Can be one of `emitEarlyWhenFull`, `shutdownWhenFull` | Example: + ```yaml from: input_table via: @@ -537,15 +619,14 @@ to: output_stream ### toStream -[KTable::toStream]: https://kafka.apache.org/37/javadoc/org/apache/kafka/streams/kstream/KTable.html#toStream-- - Convert a KTable into a KStream object. -| Stream Type | Returns | Parameter | Value Type | Description | -|:----------------------------------|:-----------------|:----------|:--------------------|:--------------------------------------------------------------------------------------| -| [KTable][KTable::toStream]`` | [KStream]`` | `mapper` | Inline or reference | (Optional)The [KeyValueMapper] function. If no mapper is specified, `K` will be used. | +| Stream Type | Returns | Parameter | Value Type | Required | Description | +|:----------------|:------------------|:----------|:--------------------|:---------|:----------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------| +| [KTable]`` | [KStream]`` | `mapper` | Inline or reference | No | A [KeyValueMapper] function, which takes a `key` of type `K` and a `value` of type `V`. The return value is the key of resulting stream, which is of type `KR`. If no mapper is provided, then keys remain unchanged. | Example: + ```yaml from: input_table via: @@ -555,174 +636,179 @@ to: output_stream ### transformKey -[KStream::transformKey]: https://kafka.apache.org/37/javadoc/org/apache/kafka/streams/kstream/KStream.html#selectKey-org.apache.kafka.streams.kstream.KeyValueMapper-org.apache.kafka.streams.kstream.Named- - -This operation takes a message and transforms the key into a new key, which can be potentially of different type. +This operation takes a message and transforms the key into a new key, which may have a different type. -| Stream Type | Returns | Parameter | Value Type | Description | -|:----------------------------------------|:------------------|:----------|:--------------------|:-------------------------------------------| -| [KStream][KStream::transformKey]`` | [KStream]`` | `mapper` | Inline or reference | The [KeyValueMapper] function. | -| | | `name` | `string` | (Optional) The name of the processor node. | +| Stream Type | Returns | Parameter | Value Type | Required | Description | +|:-----------------|:------------------|:----------|:--------------------|:---------|:----------------------------------------------------------------------------------------------------------------------------------------------------------------| +| [KStream]`` | [KStream]`` | `mapper` | Inline or reference | Yes | A [KeyValueMapper] function, which takes a `key` of type `K` and a `value` of type `V`. The return value is the key of resulting stream, which is of type `KR`. | Example: + ```yaml from: input_stream via: - type: transformKey mapper: - expression: key=str(key) # convert key from Integer to String + expression: str(key) # convert key from source type to string to: output_stream ``` ### transformKeyValue -[KStream::transformKeyValue]: https://kafka.apache.org/37/javadoc/org/apache/kafka/streams/kstream/KStream.html#map-org.apache.kafka.streams.kstream.KeyValueMapper-org.apache.kafka.streams.kstream.Named- - -This operation takes a message and transforms the key and value into a new key and value, which can each be potentially of different type. +This operation takes a message and transforms the key and value into a new key and value, which can each have a +different type than the source message key and value. -| Stream Type | Returns | Parameter | Value Type | Description | -|:---------------------------------------------|:-------------------|:----------|:--------------------|:-------------------------------------------| -| [KStream][KStream::transformKeyValue]`` | [KStream]`` | `mapper` | Inline or reference | The [KeyValueMapper] function. | -| | | `name` | `string` | (Optional) The name of the processor node. | +| Stream Type | Returns | Parameter | Value Type | Required | Description | +|:-----------------|:-------------------|:----------|:--------------------|:---------|:------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------| +| [KStream]`` | [KStream]`` | `mapper` | Inline or reference | Yes | A [KeyValueMapper] function, which takes a `key` of type `K` and a `value` of type `V`. The return type should be a tuple of type `(KR,VR)` containing the transformed `key` and `value`. | Example: + ```yaml from: input_stream via: - type: transformKeyValue mapper: - expression: (str(key), str(value)) # convert key and value from Integer to String + expression: (str(key), str(value)) # convert key and value from source type to string to: output_stream ``` ### transformKeyValueToKeyValueList -[KStream::transformKeyValueToKeyValueList]: https://kafka.apache.org/37/javadoc/org/apache/kafka/streams/kstream/KStream.html#flatMap-org.apache.kafka.streams.kstream.KeyValueMapper-org.apache.kafka.streams.kstream.Named- - -This operation takes a message and transforms it into zero, one or more new messages, which can be potentially of different type. +This operation takes a message and transforms it into zero, one or more new messages, which may have different key and +value types than the source. -| Stream Type | Returns | Parameter | Value Type | Description | -|:-----------------------------------------------------------|:-------------------|:----------|:--------------------|:--------------------------------------------------| -| [KStream][KStream::transformKeyValueToKeyValueList]`` | [KStream]`` | `mapper` | Inline or reference | The [KeyValueToKeyValueListTransformer] function. | -| | | `name` | `string` | (Optional) The name of the processor node. | +| Stream Type | Returns | Parameter | Value Type | Required | Description | +|:-----------------|:-------------------|:----------|:--------------------|:---------|:-------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------| +| [KStream]`` | [KStream]`` | `mapper` | Inline or reference | Yes | A [KeyValueMapper] function, which takes a `key` of type `K` and a `value` of type `V`. The return type should be a list of type `[(KR,VR)]` containing a list of transformed `key` and `value` pairs. | Example: + ```yaml from: input_stream via: - type: transformKeyValueToKeyValueList mapper: - expression: [(key,value),(key,value)] # duplicate all incoming messages + expression: [ (key,value), (key,value) ] # duplicate all incoming messages to: output_stream ``` ### transformKeyValueToValueList -[KStream::transformKeyValueToValueList]: https://kafka.apache.org/37/javadoc/org/apache/kafka/streams/kstream/KStream.html#flatMapValues-org.apache.kafka.streams.kstream.ValueMapperWithKey-org.apache.kafka.streams.kstream.Named- - -This operation takes a message and generates a new list of values for the key, which can be potentially of different type. +This operation takes a message and transforms it into zero, one or more new values, which may have different value types +than the source. Every entry in the result list is combined with the source key and produced on the output stream. -| Stream Type | Returns | Parameter | Value Type | Description | -|:--------------------------------------------------------|:------------------|:----------|:--------------------|:-----------------------------------------------| -| [KStream][KStream::transformKeyValueToValueList]`` | [KStream]`` | `mapper` | Inline or reference | The [KeyValueToValueListTransformer] function. | -| | | `name` | `string` | (Optional) The name of the processor node. | +| Stream Type | Returns | Parameter | Value Type | Description | +|:-----------------|:------------------|:----------|:--------------------|:-----------------------------------------------------------------------------------------------------------------------------------------------------------------------------------| +| [KStream]`` | [KStream]`` | `mapper` | Inline or reference | A [KeyValueMapper] function, which takes a `key` of type `K` and a `value` of type `V`. The return type should be a list of type `[VR]` containing a list of transformed `value`s. | Example: + ```yaml from: input_stream via: - type: transformKeyValueToValueList mapper: - expression: [value+1,value+2,value+3] # creates 3 new messages [key,VR] for every input message + expression: [ value+1, value+2, value+3 ] # creates 3 new messages [key,VR] for every input message to: output_stream ``` -### transformValue +### transformMetadata -[KStream::transformValue]: https://kafka.apache.org/37/javadoc/org/apache/kafka/streams/kstream/KStream.html#flatMapValues-org.apache.kafka.streams.kstream.ValueMapperWithKey-org.apache.kafka.streams.kstream.Named- +This operation takes a message and transforms its value to a new value, which may have different value type +than the source. -This operation takes a message and transforms the value into a new value, which can be potentially of different type. +| Stream Type | Returns | Parameter | Value Type | Required | Description | +|:-----------------|:------------------|:----------|:--------------------|:---------|:--------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------| +| [KStream]`` | [KStream]`` | `mapper` | Inline or reference | Yes | A [MetadataTransformer] function that converts the metadata (Kafka headers, timestamp) of every record in the stream. It gets a metadata object as input and should return the same type, but potentially with modified fields. | -| Stream Type | Returns | Parameter | Value Type | Required | Description | -|:------------------------------------------|:------------------|:----------|:--------------------|:---------|:--------------------------------------------------| -| [KStream][KStream::transformValue]`` | [KStream]`` | `store` | Store configuration | No | The [Store] configuration. | -| | | `mapper` | Inline or reference | Yes | The [KeyValueToKeyValueListTransformer] function. | -| | | `name` | `string` | No | The name of the processor node. | +Example: + +```yaml +from: input_stream +via: + - type: transformValue + mapper: + expression: str(value) # convert value from source type to String +to: output_stream +``` + +### transformValue + +This operation takes a message and transforms its value to a new value, which may have different value type +than the source. + +| Stream Type | Returns | Parameter | Value Type | Required | Description | +|:-----------------|:------------------|:----------|:--------------------|:---------|:----------------------------------------------------------------------------------------------------------------------------------------| +| [KStream]`` | [KStream]`` | `mapper` | Inline or reference | Yes | A [KeyValueMapper] function, which takes a `key` of type `K` and a `value` of type `V`. The return type should be a value of type `VR`. | Example: + ```yaml from: input_stream via: - type: transformValue mapper: - expression: value=str(key) # convert value from Integer to String + expression: str(value) # convert value from source type to String to: output_stream ``` ### windowBySession -[KGroupedStream::windowedBy]: https://kafka.apache.org/37/javadoc/org/apache/kafka/streams/kstream/KGroupedStream.html#windowedBy-org.apache.kafka.streams.kstream.SessionWindows- -[CogroupedKStream::windowedBySession]: https://kafka.apache.org/37/javadoc/org/apache/kafka/streams/kstream/CogroupedKStream.html#windowedBy-org.apache.kafka.streams.kstream.SessionWindows- -[SessionWindows]: https://kafka.apache.org/37/javadoc/org/apache/kafka/streams/kstream/SessionWindows.html -[WindowTypes]: https://kafka.apache.org/37/documentation/streams/developer-guide/dsl-api.html#windowing - -Create a new windowed KStream instance that can be used to perform windowed aggregations. For more details on the different types of windows, please refer to [WindowTypes]|[this page]. +Create a new windowed KStream instance that can be used to perform windowed aggregations. For more details on the +different types of windows, please refer to [WindowTypes]|[this page]. -| Stream Type | Returns | Parameter | Value Type | Description | -|:------------------------------------------------------|:-----------------------------------------|:---------------|:-----------|:----------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------| -| [KGroupedStream][KGroupedStream::windowedBySession] | [SessionWindowedKStream]`` | inactivityGap | [Duration] | The inactivity gap parameter for the [SessionWindows] object. | -| | | grace | [Duration] | (Optional) The grace parameter for the [SessionWindows] object. | -| [CogroupedKStream][CogroupedKStream::windowedBySession] | [SessionWindowedCogroupedKStream]`` | inactivityGap | [Duration] | The inactivity gap parameter for the [SessionWindows] object. | -| | | grace | [Duration] | (Optional) The grace parameter for the [SessionWindows] object. | +| Stream Type | Returns | Parameter | Value Type | Required | Description | +|:--------------------------|:-----------------------------------------|:--------------|:-----------|:---------|:-----------------------------------------------------------------------------------------------------| +| [KGroupedStream]`` | [SessionWindowedKStream]`` | inactivityGap | [Duration] | Yes | The maximum inactivity gap with which keys are grouped. | +| | | grace | [Duration] | No | The grace duration allowing for out-of-order messages to still be associated with the right session. | +| [CogroupedKStream]`` | [SessionWindowedCogroupedKStream]`` | inactivityGap | [Duration] | Yes | The maximum inactivity gap with which keys are grouped. | +| | | grace | [Duration] | No | The grace duration allowing for out-of-order messages to still be associated with the right session. | Example: + ```yaml from: input_stream via: - type: groupBy mapper: my_mapper_function - - type: windowedBy - windowType: time - duration: 1h - advanceBy: 15m + - type: windowedBySession + inactivityGap: 1h grace: 5m - type: reduce reducer: my_reducer_function - type: toStream to: output_stream ``` + ### windowByTime -[KGroupedStream::windowedBySliding]: https://kafka.apache.org/37/javadoc/org/apache/kafka/streams/kstream/KGroupedStream.html#windowedBy-org.apache.kafka.streams.kstream.SlidingWindows- -[KGroupedStream::windowedByDuration]: https://kafka.apache.org/37/javadoc/org/apache/kafka/streams/kstream/KGroupedStream.html#windowedBy-org.apache.kafka.streams.kstream.Windows- -[SlidingWindows]: https://kafka.apache.org/37/javadoc/org/apache/kafka/streams/kstream/SlidingWindows.html -[TimeWindows]: https://kafka.apache.org/37/javadoc/org/apache/kafka/streams/kstream/TimeWindows.html -[WindowTypes]: https://kafka.apache.org/37/documentation/streams/developer-guide/dsl-api.html#windowing - -Create a new windowed KStream instance that can be used to perform windowed aggregations. For more details on the different types of windows, please refer to [WindowTypes]|[this page]. - -| Stream Type | Returns | Parameter | Value Type | Description | -|:---------------------------------------------------------|:--------------------------------------|:---------------|:-----------|:-----------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------| -| [KGroupedStream][KGroupedStream::windowedBySliding] | [TimeWindowedKStream]`` | `windowType` | `string` | Fixed value `sliding`. | -| | | timeDifference | [Duration] | The time difference parameter for the [SlidingWindows] object. | -| | | grace | [Duration] | (Optional) The grace parameter for the [SlidingWindows] object. | -| [KGroupedStream][KGroupedStream::windowedByDuration] | [TimeWindowedKStream]`` | `windowType` | `string` | Fixed value `hopping`. | -| | | advanceBy | [Duration] | The amount by which each window is advanced. If this value is not specified, then it will be equal to _duration_, which gives tumbling windows. If you make this value smaller than _duration_ you will get hopping windows. | -| | | grace | [Duration] | (Optional) The grace parameter for the [TimeWindows] object. | -| [KGroupedStream][KGroupedStream::windowedByDuration] | [TimeWindowedKStream]`` | `windowType` | `string` | Fixed value `tumbling`. | -| | | duration | [Duration] | The duration parameter for the [TimeWindows] object. | -| | | grace | [Duration] | (Optional) The grace parameter for the [TimeWindows] object. | -| [CogroupedKStream][CogroupedKStream::windowedBySliding] | [TimeWindowedCogroupedKStream]`` | `windowType` | `string` | Fixed value `sliding`. | -| | | timeDifference | [Duration] | The time difference parameter for the [SlidingWindows] object. | -| | | grace | [Duration] | (Optional) The grace parameter for the [SlidingWindows] object. | -| [CogroupedKStream][CogroupedKStream::windowedByDuration] | [TimeWindowedCogroupedKStream]`` | `windowType` | `string` | Fixed value `hopping`. | -| | | advanceBy | [Duration] | The amount by which each window is advanced. If this value is not specified, then it will be equal to _duration_, which gives tumbling windows. If you make this value smaller than _duration_ you will get hopping windows. | -| | | grace | [Duration] | (Optional) The grace parameter for the [TimeWindows] object. | -| [CogroupedKStream][CogroupedKStream::windowedByDuration] | [TimeWindowedCogroupedKStream]`` | `windowType` | `string` | Fixed value `tumbling`. | -| | | duration | [Duration] | The duration parameter for the [TimeWindows] object. | -| | | grace | [Duration] | (Optional) The grace parameter for the [TimeWindows] object. | +Create a new windowed KStream instance that can be used to perform windowed aggregations. For more details on the +different types of windows, please refer to [WindowTypes]|[this page]. + +| Stream Type | Returns | Parameter | Value Type | Description | +|:--------------------------|:--------------------------------------|:---------------|:-----------|:-----------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------| +| [KGroupedStream]`` | [TimeWindowedKStream]`` | `windowType` | `string` | Fixed value `sliding`. | +| | | timeDifference | [Duration] | The time difference parameter for the [SlidingWindows] object. | +| | | grace | [Duration] | (Optional) The grace parameter for the [SlidingWindows] object. | +| [KGroupedStream]`` | [TimeWindowedKStream]`` | `windowType` | `string` | Fixed value `hopping`. | +| | | advanceBy | [Duration] | The amount by which each window is advanced. If this value is not specified, then it will be equal to _duration_, which gives tumbling windows. If you make this value smaller than _duration_ you will get hopping windows. | +| | | grace | [Duration] | (Optional) The grace parameter for the [TimeWindows] object. | +| [KGroupedStream]`` | [TimeWindowedKStream]`` | `windowType` | `string` | Fixed value `tumbling`. | +| | | duration | [Duration] | The duration parameter for the [TimeWindows] object. | +| | | grace | [Duration] | (Optional) The grace parameter for the [TimeWindows] object. | +| [CogroupedKStream]`` | [TimeWindowedCogroupedKStream]`` | `windowType` | `string` | Fixed value `sliding`. | +| | | timeDifference | [Duration] | The time difference parameter for the [SlidingWindows] object. | +| | | grace | [Duration] | (Optional) The grace parameter for the [SlidingWindows] object. | +| [CogroupedKStream]`` | [TimeWindowedCogroupedKStream]`` | `windowType` | `string` | Fixed value `hopping`. | +| | | advanceBy | [Duration] | The amount by which each window is advanced. If this value is not specified, then it will be equal to _duration_, which gives tumbling windows. If you make this value smaller than _duration_ you will get hopping windows. | +| | | grace | [Duration] | (Optional) The grace parameter for the [TimeWindows] object. | +| [CogroupedKStream]`` | [TimeWindowedCogroupedKStream]`` | `windowType` | `string` | Fixed value `tumbling`. | +| | | duration | [Duration] | The duration parameter for the [TimeWindows] object. | +| | | grace | [Duration] | (Optional) The grace parameter for the [TimeWindows] object. | Example: + ```yaml from: input_stream via: @@ -739,27 +825,57 @@ via: to: output_stream ``` - ## Sink Operations -### branch +### as -[KStream::branch]: https://kafka.apache.org/37/javadoc/org/apache/kafka/streams/kstream/KStream.html#branch-org.apache.kafka.streams.kstream.Named-org.apache.kafka.streams.kstream.Predicate...- +Pipelines closed of with `as` can be referred by other pipelines as their starting reference. This allows for a common +part of processing logic to be placed in its own pipeline in KSML, serving as an intermediate result. -Branches out messages from the input stream into several branches based on predicates. Each branch is defined as a list item below the branch operation. Branch predicates are defined using the `if` keyword. Messages are only processed by one of the branches, namely the first one for which the predicate returns `true`. +| Applies to | Value Type | Required | Description | +|:--------------------|:-----------|:---------|:-------------------------------------------------------------------------------| +| Any pipeline`` | string | Yes | The name under which the pipeline result can be referenced by other pipelines. | -| Applies to | Value Type | Description | -|:----------------------------------|:---------------------------|:-------------------------------------------------| -| [KStream][KStream::branch]`` | List of branch definitions | See for description of branch definitions below. | +Example: -Branches in KSML are nested pipelines, which are parsed without the requirement of a source attribute. Each branch accepts the following parameters: +```yaml +pipelines: + first: + from: some_source_topic + via: + - type: ... + as: first_pipeline + + second: + from: first_pipeline + via: + - type: ... + to: ... +``` -| Branch element | Value Type | Description | -|:---------------|:----------------------------------------|:------------------------------------------------------------------------------------------------------------------------------| -| `if` | Inline [Predicate] or reference | The [Predicate] function that determines if the message is sent down this branch, or is passed on to the next branch in line. | -| _Inline_ | All pipeline parameters, see [Pipeline] | The inlined pipeline describes the topology of the specific branch. | +Here, the first pipeline ends by sending its output to a stream internally called `first_pipeline`. This stream is used +as input for the `second` pipeline. + +### branch + +Branches out messages from the input stream into several branches based on predicates. Each branch is defined as a list +item below the branch operation. Branch predicates are defined using the `if` keyword. Messages are only processed by +one of the branches, namely the first one for which the predicate returns `true`. + +| Applies to | Value Type | Required | Description | +|:-----------------|:---------------------------|:---------|:-------------------------------------------------| +| [KStream]`` | List of branch definitions | Yes | See for description of branch definitions below. | + +Branches in KSML are nested pipelines, which are parsed without the requirement of a source attribute. Each branch +accepts the following parameters: + +| Branch element | Value Type | Required | Description | +|:---------------|:----------------------------------------|:---------|:------------------------------------------------------------------------------------------------------------------------------| +| `if` | Inline [Predicate] or reference | No | The [Predicate] function that determines if the message is sent down this branch, or is passed on to the next branch in line. | +| _Inline_ | All pipeline parameters, see [Pipeline] | Yes | The inlined pipeline describes the topology of the specific branch. | Example: + ```yaml from: some_source_topic branch: @@ -771,25 +887,27 @@ branch: to: ksml_sensordata_red - forEach: code: | - print('Unknown color sensor: '+str(value)) + print('Unknown color sensor: '+value["color"]) ``` -In this example, the first two branches are entered if the respective predicate matches (the color attribute of value matches a certain color). -If the predicate returns `false`, then the next predicate/branch is tried. Only the last branch in the list can be a sink operation. +In this example, the first two branches are entered if the respective predicate matches (the color attribute of value +matches a certain color). +If the predicate returns `false`, then the next predicate/branch is tried. Only the last branch in the list can be a +sink operation. ### forEach -[KStream::forEach]: https://kafka.apache.org/37/javadoc/org/apache/kafka/streams/kstream/KStream.html#foreach-org.apache.kafka.streams.kstream.ForeachAction-org.apache.kafka.streams.kstream.Named- +This sends each message to a custom defined function. This function is expected to handle each message as its final +step. The function does not (need to) return anything. -This sends each message to a custom defined function. This function is expected to handle each message as its final step. The function does not (need to) return anything. - -| Applies to | Value Type | Description | -|:-----------------------------------|:--------------------|:-----------------------------------------------------------------------------| -| [KStream][KStream::forEach]`` | Inline or reference | The [ForEach] function that is called for every record on the source stream. | +| Applies to | Value Type | Description | +|:-----------------|:--------------------|:------------------------------------------------------------------------------------------------------------------------------------------| +| [KStream]`` | Inline or reference | The [ForEach] function that is called for every record on the source stream. Its arguments are `key` of type `K` and `value` of type `V`. | Examples: + ```yaml -forEach: my_print_function +forEach: my_foreach_function ``` ```yaml @@ -797,38 +915,74 @@ forEach: code: print(value) ``` -### to +### print + +This sends each message to a custom defined print function. This function is expected to handle each message as the +final in the pipeline. The function does not (need to) return anything. + +As target, you can specify a filename. If none is specified, then all messages are printed to stdout. + +| Applies to | Parameter | Value Type | Required | Description | +|:-----------------|:----------|:--------------------|:---------|----------------------------------------------------------------------------------------------------------------------------------------------------------------------------------| +| [KStream]`` | filename | string | No | The filename to output records to. If nothing is specified, then messages will be printed on stdout. | +| | label | string | No | A label to attach to every output record. | +| | `mapper` | Inline or reference | Yes | A [KeyValueMapper] function, which takes a `key` of type `K` and a `value` of type `V`. The return value should be of type `string` and is sent to the specified file or stdout. | + +Examples: + +```yaml +from: source +via: + - type: ... +print: + filename: file.txt + mapper: + expression: "record value: " + str(value) +``` -[KStream::to]: https://kafka.apache.org/37/javadoc/org/apache/kafka/streams/kstream/KStream.html#to-java.lang.String-org.apache.kafka.streams.kstream.Produced- +### to Messages are sent directly to a named `Stream`. -| Applies to | Value Type | Description | -|:------------------------------|:-----------|:--------------------------------------------| -| [KStream][KStream::to]`` | `string` | The name of a defined [Stream](streams.md). | +| Applies to | Value Type | Required | Description | +|:-----------------|:---------------------------------------------------------------|:---------|:--------------------------------------------| +| [KStream]`` | Inline [Topic] or reference to a stream, table or global table | Yes | The name of a defined [stream](streams.md). | + +Examples: -Example: ```yaml to: my_target_topic ``` -### toExtractor +```yaml +from: source +via: + - type: ... +to: + topic: my_target_topic + keyType: someType + valueType: someOtherType + partitioner: + expression: hash_of(key) +``` -[KStream::toExtractor]: https://kafka.apache.org/37/javadoc/org/apache/kafka/streams/kstream/KStream.html#to-org.apache.kafka.streams.processor.TopicNameExtractor-org.apache.kafka.streams.kstream.Produced- +### toTopicNameExtractor -Messages are passed onto a user function, which returns the name of the topic that message needs to be sent to. This operation acts as a Sink and is always the last operation in a [pipeline](pipelines.md). +Messages are passed onto a user function, which returns the name of the topic that message needs to be sent to. This +operation acts as a Sink and is always the last operation in a [pipeline](pipelines.md). -| Applies to | Value Type | Description | -|:---------------------------------------|:--------------------|:-------------------------------------------------------------------------------------------------------------------------------------| -| [KStream][KStream::toExtractor]`` | Inline or reference | The [TopicNameExtractor] function that is called for every message and returns the topic name to which the message shall be written. | +| Applies to | Value Type | Required | Description | +|:-----------------|:--------------------|:---------|:-------------------------------------------------------------------------------------------------------------------------------------| +| [KStream]`` | Inline or reference | Yes | The [TopicNameExtractor] function that is called for every message and returns the topic name to which the message shall be written. | Examples: + ```yaml -toExtractor: my_extractor_function +toTopicNameExtractor: my_extractor_function ``` ```yaml -toExtractor: +toTopicNameExtractor: code: | if key == 'sensor1': return 'ksml_sensordata_sensor1' diff --git a/docs/pipelines.md b/docs/pipelines.md index d5639a23..eb15db42 100644 --- a/docs/pipelines.md +++ b/docs/pipelines.md @@ -67,14 +67,14 @@ four sink types in KSML: | Sink type | Description | |-----------|----------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------| -| `as` | Allows the pipeline result to be saved under an internal referenceable name. Pipelines defined after this point may refer to this name in their `from` statement. | +| `as` | Allows the pipeline result to be saved under an internal name, which can later be referenced. Pipelines defined after this point may refer to this name in their `from` statement. | | `branch` | This statement allows the pipeline to be split up in several branches. Each branch filters messages with an `if` statement. Messages will be processed only by the first branch of which the `if` statement is true. | | `forEach` | Sends every message to a function, without expecting any return type. Because there is no return type, the pipeline always stops after this statement. | | `print` | Prints out every message according to a given output specification. | | `to` | Sends all output messages to a specific target. This target can be a pre-defined `stream`, `table` or `globalTable`, an inline-defined topic, or a special function called a `topicNameExtractor`. | -For more information, see the respective documentation -on [pipeline definitions](specifications.md#definitions/PipelineDefinition). +For more information, see the respective documentation on pipeline definitions in +the [definitions section of the KSML language spec](ksml-language-spec.html#definitions/PipelineDefinition). ## Duration diff --git a/docs/getting-started.md b/docs/quick-start.md similarity index 91% rename from docs/getting-started.md rename to docs/quick-start.md index a7e45271..a6776b9a 100644 --- a/docs/getting-started.md +++ b/docs/quick-start.md @@ -1,22 +1,27 @@ -# Getting started +# Quick start ### Table of Contents + 1. [Introduction](#introduction) 2. [Starting a demo setup](#starting-a-demo-setup) 3. [Starting a KSML runner](#starting-a-ksml-runner) 4. [Next steps](#next-steps) ## Introduction -KSML comes with an example setup, where a producer outputs SensorData messages to Kafka, which are then processed by KSML pipelines. + +KSML comes with example definitions, which contain a producer that outputs SensorData messages to Kafka, +and several pipelines, which each independently consume and process the produced messages. ## Starting a demo setup + After checking out the repository, go to the KSML directory and execute the following: ``` docker compose up -d ``` -This will start Zookeeper, Kafka and a Schema Registry in the background. It will also start the demo producer, which outputs two random messages per second on a `ksml_sensordata_avro` topic. +This will start Zookeeper, Kafka and a Schema Registry in the background. It will also start the demo producer, which +outputs two random messages per second on a `ksml_sensordata_avro` topic. You can check the valid starting of these containers using the following command: @@ -25,6 +30,7 @@ docker compose logs -f ``` Press CTRL-C when you verified data is produced. This typically looks like this: + ``` example-producer-1 | 2024-03-06T20:24:49,480Z INFO i.a.k.r.backend.KafkaProducerRunner Calling generate_sensordata_message example-producer-1 | 2024-03-06T20:24:49,480Z INFO i.a.k.r.backend.ExecutableProducer Message: key=sensor2, value=SensorData: {"city":"Utrecht", "color":"white", "name":"sensor2", "owner":"Alice", "timestamp":1709756689480, "type":"HUMIDITY", "unit":"%", "value":"66"} @@ -41,7 +47,6 @@ example-producer-1 | 2024-03-06T20:24:50,035Z INFO i.a.k.r.backend.ExecutableP ``` - ## Starting a KSML runner To start a container which executes the example KSML definitions, type @@ -69,6 +74,7 @@ This will start the KSML docker container. You should see the following typical ## Next steps -Check out the examples in the [Examples]({{ site.github.repository_url }}/tree/main/examples/) directory. By modifying the file `examples/ksml-runner.yaml` you can select the example(s) to run. +Check out the examples in the `examples` directory of the project. By modifying the file `examples/ksml-runner.yaml` you +can select the example(s) to run. For a more elaborate introduction, you can start [here](introduction.md) or refer to the [documentation](index.md). diff --git a/docs/release-notes.md b/docs/release-notes.md index 4d499ac8..206064f1 100644 --- a/docs/release-notes.md +++ b/docs/release-notes.md @@ -2,45 +2,70 @@ ## Releases - * [Release Notes](#release-notes) - * [Releases](#releases) - * [0.8.0 (2024-03-08)](#080-2024-03-08) - * [0.2.2 (2024-01-30)](#022-2024-01-30) - * [0.2.1 (2023-12-20)](#021-2023-12-20) - * [0.2.0 (2023-12-07)](#020-2023-12-07) - * [0.1.0 (2023-03-15)](#010-2023-03-15) - * [0.0.4 (2022-12-02)](#004-2022-12-02) - * [0.0.3 (2021-07-30)](#003-2021-07-30) - * [0.0.2 (2021-06-28)](#002-2021-06-28) - * [0.0.1 (2021-04-30)](#001-2021-04-30) - + * [Releases](#releases) + * [1.0.0 (2024-06-28)](#100-2024-06-28) + * [0.8.0 (2024-03-08)](#080-2024-03-08) + * [0.9.1 (2024-06-21)](#091-2024-06-21) + * [0.9.0 (2024-06-05)](#090-2024-06-05) + * [0.2.2 (2024-01-30)](#022-2024-01-30) + * [0.2.1 (2023-12-20)](#021-2023-12-20) + * [0.2.0 (2023-12-07)](#020-2023-12-07) + * [0.1.0 (2023-03-15)](#010-2023-03-15) + * [0.0.4 (2022-12-02)](#004-2022-12-02) + * [0.0.3 (2021-07-30)](#003-2021-07-30) + * [0.0.2 (2021-06-28)](#002-2021-06-28) + * [0.0.1 (2021-04-30)](#001-2021-04-30) + +### 1.0.0 (2024-06-28) + +* Reworked parsing logic, allowing alternatives for operations and other definitions to co-exist in the KSML language + specification. This allows for better syntax checking in IDEs. +* Lots of small fixes and completion modifications. + +### 0.9.1 (2024-06-21) + +* Fix failing test in GitHub Actions during release +* Unified build workflows + +### 0.9.0 (2024-06-05) + +* Collectable metrics +* New topology test suite +* Python context hardening +* Improved handling of Kafka tombstones +* Added flexibility to producers (single shot, n-shot, or user condition-based) +* JSON Logging support +* Bumped GraalVM to 23.1.2 +* Bumped several dependency versions +* Several fixes and security updates ### 0.8.0 (2024-03-08) + * Reworked all parsing logic, to allow for exporting the JSON schema of the KSML specification: - * docs/specification.md is now derived from internal parser logic, guaranteeing consistency and completeness. - * examples/ksml.json contains the JSON schema, which can be loaded into IDEs for syntax validation and completion. + * docs/specification.md is now derived from internal parser logic, guaranteeing consistency and completeness. + * examples/ksml.json contains the JSON schema, which can be loaded into IDEs for syntax validation and completion. * Improved schema handling: - * Better compatibility checking between schema fields. + * Better compatibility checking between schema fields. * Improved support for state stores: - * Update to state store typing and handling. - * Manual state stores can be defined and referenced in pipelines. - * Manual state stores are also available in Python functions. - * State stores can be used 'side-effect-free' (eg. no AVRO schema registration) + * Update to state store typing and handling. + * Manual state stores can be defined and referenced in pipelines. + * Manual state stores are also available in Python functions. + * State stores can be used 'side-effect-free' (e.g. no AVRO schema registration) * Python function improvements: - * Automatic variable assignment for state stores. - * Every Python function can use a Java Logger, integrating Python output with KSML log output. - * Type inference in situations where parameters or result types can be derived from the context. + * Automatic variable assignment for state stores. + * Every Python function can use a Java Logger, integrating Python output with KSML log output. + * Type inference in situations where parameters or result types can be derived from the context. * Lots of small language updates: - * Improve readability for store types, filter operations and windowing operations - * Introduction of the "as" operation, which allows for pipeline referencing and chaining. + * Improve readability for store types, filter operations and windowing operations + * Introduction of the "as" operation, which allows for pipeline referencing and chaining. * Better data type handling: - * Separation of data types and KSML core, allowing for easier addition of new data types in the future. - * Automatic conversion of data types, removing common pipeline failure scenarios. - * New implementation for CSV handling. + * Separation of data types and KSML core, allowing for easier addition of new data types in the future. + * Automatic conversion of data types, removing common pipeline failure scenarios. + * New implementation for CSV handling. * Merged the different runners into a single runner. - * KSML definitions can now include both producers (data generators) and pipelines (Kafka Streams topologies). - * Removal of Kafka and Axual backend distinctions. + * KSML definitions can now include both producers (data generators) and pipelines (Kafka Streams topologies). + * Removal of Kafka and Axual backend distinctions. * Configuration file updates, allowing for running multiple definitions in a single runner (each in its own namespace). * Examples updated to reflect the latest definition format. * Documentation updated. @@ -82,7 +107,7 @@ **Changes:** * Added XML/SOAP support -* Added datagenerator +* Added data generator * Added Automatic Type Conversion * Added Schema Support for XML, Avro, JSON, Schema * Added Basic Error Handling @@ -102,7 +127,7 @@ * Bug fix for windowed objects * Store improvements * Support Liberica NIK -* Switch from Travis CI to Github workflow +* Switch from Travis CI to GitHub workflow * Build snapshot Docker image on pull request merged ### 0.0.3 (2021-07-30) diff --git a/docs/runners.md b/docs/runners.md index 4d344ff8..7efc0ee1 100644 --- a/docs/runners.md +++ b/docs/runners.md @@ -1,15 +1,19 @@ # Runners ### Table of Contents + 1. [Introduction](#introduction) 2. [Configuration](#configuration) - - [Namespace support](#using-with-axual-platform-or-other-namespaced-kafka-clusters) + - [Namespace support](#using-with-axual-platform-or-other-namespaced-kafka-clusters) 3. [Starting a container](#starting-a-container) ## Introduction -The core of KSML is a library that allows KSML definition files to be parsed and translated into Kafka Streams topologies. Because we wanted to keep KSML low-overhead, KSML does not run these topologies itself. A runner application is provided separately to execute the generated topologies. -The runner supports plain Kafka connections, which can be configured using normal Kafka properties, and contains an advanced configurations that helps running against Kafka clusters using namespacing. +The core of KSML is a library that allows KSML definition files to be parsed and translated into Kafka Streams +topologies. Because we wanted to keep KSML low-overhead, KSML does not run these topologies itself. A runner application +is provided separately to execute the generated topologies. +The runner supports plain Kafka connections, which can be configured using normal Kafka properties, and contains an +advanced configurations that helps running against Kafka clusters using namespacing. Examples of runner configurations are shown below. @@ -42,12 +46,14 @@ ksml: logPayload: true # log message payloads upon error loggerName: ProduceError # logger name handler: continueOnFail # continue or stop on error + enableProducers: true # False to disable producers in the KSML definition + enablePipelines: true # False to disable pipelines in the KSML definition definitions: # KSML definition files from the working directory namedDefinition1: definition1.yaml namedDefinition2: definition2.yaml namedDefinition3: -kafka: # Kafka streams configuration options +kafka: # Kafka streams configuration options application.id: io.ksml.example.processor bootstrap.servers: broker-1:9092,broker-2:9092 security.protocol: SSL @@ -56,7 +62,7 @@ kafka: # Kafka streams configuration options ssl.endpoint.identification.algorithm: "" ssl.truststore.location: /ksml/config/truststore.jks ssl.truststore.password: password-for-truststore - + # Schema Registry client configuration, needed when schema registry is used schema.registry.url: http://schema-registry:8083 schema.registry.ssl.truststore.location: /ksml/config/truststore.jks @@ -73,22 +79,24 @@ The following config will resolve the backing topic of a stream or table ```yaml kafka: - # The patterns for topics, groups and transactional ids. - # Each field between the curly braces must be specified in the configuration, except the topic, - # group.id and transactional.id fields, which is used to identify the place where the resource name - # is used - topic.pattern: "{tenant}-{instance}-{environment}-{topic}" - group.id.pattern: "{tenant}-{instance}-{environment}-{group.id}" - transactional.id.pattern: "{tenant}-{instance}-{environment}-{transactional.id}" - - # Additional configuration options used for resolving the pattern to values - tenant: "ksmldemo" - instance: "dta" - environment: "dev" + # The patterns for topics, groups and transactional ids. + # Each field between the curly braces must be specified in the configuration, except the topic, + # group.id and transactional.id fields, which is used to identify the place where the resource name + # is used + topic.pattern: "{tenant}-{instance}-{environment}-{topic}" + group.id.pattern: "{tenant}-{instance}-{environment}-{group.id}" + transactional.id.pattern: "{tenant}-{instance}-{environment}-{transactional.id}" + + # Additional configuration options used for resolving the pattern to values + tenant: "ksmldemo" + instance: "dta" + environment: "dev" ``` ## Starting a container -To start a container the KSML definitions and Runner configuration files need to be available in a directory mounted inside the docker container. + +To start a container the KSML definitions and Runner configuration files need to be available in a directory mounted +inside the docker container. The default Runner configuration filename is **_ksml-runner.yaml_**. If no arguments are given, the runner will look for this file in the home directory diff --git a/docs/streams.md b/docs/streams.md index 35631d70..3cce18ca 100644 --- a/docs/streams.md +++ b/docs/streams.md @@ -1,6 +1,7 @@ # Streams ### Table of Contents + 1. [Introduction](#introduction) 2. [Stream](#stream) 3. [Table](#table) @@ -28,6 +29,8 @@ streams: topic: some_kafka_topic keyType: string valueType: string + offsetResetPolicy: earliest + timestampExtractor: my_timestamp_extractor ``` ## Table diff --git a/docs/types.md b/docs/types.md index 79e42f22..edc8a04a 100644 --- a/docs/types.md +++ b/docs/types.md @@ -1,15 +1,16 @@ # Types ### Table of Contents + 1. [Introduction](#introduction) -1. [Primitives](#primitives) -1. [Any](#any) -1. [Duration](#duration) -1. [Enum](#enum) -1. [List](#list) -1. [Struct](#struct) -1. [Tuple](#tuple) -1. [Windowed](#windowed) +2. [Primitives](#primitives) +3. [Any](#any) +4. [Duration](#duration) +5. [Enum](#enum) +6. [List](#list) +7. [Struct](#struct) +8. [Tuple](#tuple) +9. [Windowed](#windowed) ## Introduction @@ -20,26 +21,39 @@ Types are very useful. The following native types are supported. -| Type | Description | -|---------|---------------------------------------------------------------| -| ? | Any type | -| boolean | Boolean values, ie. true or false | -| bytes | Byte array | -| double | Double precision floating point | -| float | Single precision floating point | -| byte | 8-bit integer | -| short | 16-bit integer | -| int | 32-bit integer | -| long | 64-bit long | -| string | String of characters | -| struct | Untyped key-value map, where keys are always of type `string` | +| Type | Description | +|---------------|------------------------------------------------------------------------------------------------------------| +| ?, or any | Any type | +| null, or none | Null type, available for variables without a value (eg. Kafka tombstone messages, or optional AVRO fields) | +| boolean | Boolean values, ie. true or false | +| double | Double precision floating point | +| float | Single precision floating point | +| byte | 8-bit integer | +| short | 16-bit integer | +| int | 32-bit integer | +| long | 64-bit long | +| bytes | Byte array | +| string | String of characters | +| struct | Key-value map, where with `string` keys and values of any type | ## Any -The special type `any` can be used in places where input is uncertain. Code that deals +The special type `?` or `any` can be used in places where input is uncertain. Code that deals with input of this type should always perform proper type checking before assuming any specific underlying type. +## Duration + +Some fields in the KSML spec are of type `duration`. These fields have a fixed format `123x`, where `123` is an +integer and `x` is any of the following: + +* __: milliseconds +* `s`: seconds +* `m`: minutes +* `h`: hours +* `d`: days +* `w`: weeks + ## Enum Enumerations can be defined as individual types, through: @@ -51,11 +65,13 @@ enum(literal1, literal2, ...) ## List Lists contain elements of the same type. They are defined using: + ``` [elementType] ``` Examples: + ``` [string] [long] @@ -71,10 +87,12 @@ using: ## Tuple -Tuples combine multiple subtypes into one. For example `(1, "text")` is a tuple containing an integer and a string element. +Tuples combine multiple subtypes into one. For example `(1, "text")` is a tuple containing an integer and a string +element. Tuple types always have a fixed number of elements. Examples: + ``` (long, string) (avro:SensorData, string, long, string) @@ -91,6 +109,7 @@ union(type1, type2, ...) ``` Examples: + ``` union(null, string) union(avro:SensorData, long, string) @@ -103,6 +122,7 @@ _Windowed\_ type to group Kafka messages with similar keys together. The resu window, with a defined start and end time. KSML can convert the internal _Windowed\_ type into a struct type with five fields: + * `start`: The window start timestamp (type `long`) * `end`: The window end timestamp (type `long`) * `startTime`: The window start time (type `string`) @@ -111,11 +131,13 @@ KSML can convert the internal _Windowed\_ type into a struct type with five f However, in pipelines or topic definitions users may need to refer to this type explicitly. This is done in the following manner: + ``` notation:windowed(keytype) ``` For example: + ``` avro:windowed(avro:SensorData) xml:windowed(long) diff --git a/ksml-runner/src/main/java/io/axual/ksml/runner/KsmlInfo.java b/ksml-runner/src/main/java/io/axual/ksml/runner/KsmlInfo.java index 451abd08..cd573607 100644 --- a/ksml-runner/src/main/java/io/axual/ksml/runner/KsmlInfo.java +++ b/ksml-runner/src/main/java/io/axual/ksml/runner/KsmlInfo.java @@ -20,7 +20,7 @@ * =========================LICENSE_END================================== */ -import io.axual.ksml.metric.KSMLMetrics; +import io.axual.ksml.metric.Metrics; import lombok.extern.slf4j.Slf4j; import javax.management.*; @@ -68,7 +68,7 @@ public class KsmlInfo { } public static void registerKsmlAppInfo(String appId) { - var beanName = "%s:type=app-info,app-id=%s,app-name=%s,app-version=%s,build-time=%s".formatted(KSMLMetrics.DOMAIN, appId, ObjectName.quote(APP_NAME), ObjectName.quote(APP_VERSION), ObjectName.quote(BUILD_TIME)); + var beanName = "%s:type=app-info,app-id=%s,app-name=%s,app-version=%s,build-time=%s".formatted(Metrics.DOMAIN, appId, ObjectName.quote(APP_NAME), ObjectName.quote(APP_VERSION), ObjectName.quote(BUILD_TIME)); try { var objectName = ObjectName.getInstance(beanName); var beanServer = ManagementFactory.getPlatformMBeanServer(); diff --git a/ksml-runner/src/main/java/io/axual/ksml/runner/prometheus/PrometheusExport.java b/ksml-runner/src/main/java/io/axual/ksml/runner/prometheus/PrometheusExport.java index c1fc895e..8963d569 100644 --- a/ksml-runner/src/main/java/io/axual/ksml/runner/prometheus/PrometheusExport.java +++ b/ksml-runner/src/main/java/io/axual/ksml/runner/prometheus/PrometheusExport.java @@ -20,7 +20,7 @@ * =========================LICENSE_END================================== */ -import io.axual.ksml.metric.KSMLMetrics; +import io.axual.ksml.metric.Metrics; import io.axual.ksml.runner.config.PrometheusConfig; import io.prometheus.jmx.BuildInfoMetrics; import io.prometheus.jmx.JmxCollector; @@ -53,7 +53,7 @@ public PrometheusExport(PrometheusConfig config) { @Synchronized public void start() throws Exception { - KSMLMetrics.init(); + Metrics.init(); if (!config.isEnabled()) { log.info("Prometheus export is disabled"); return; diff --git a/ksml/NOTICE.txt b/ksml/NOTICE.txt index f9b500c5..b7675010 100644 --- a/ksml/NOTICE.txt +++ b/ksml/NOTICE.txt @@ -1,5 +1,5 @@ -Lists of 71 third-party dependencies. +Lists of 72 third-party dependencies. (MIT License) minimal-json (com.eclipsesource.minimal-json:minimal-json:0.9.5 - https://github.com/ralfstx/minimal-json) (The Apache Software License, Version 2.0) Jackson-annotations (com.fasterxml.jackson.core:jackson-annotations:2.17.1 - https://github.com/FasterXML/jackson) (The Apache Software License, Version 2.0) Jackson-core (com.fasterxml.jackson.core:jackson-core:2.17.1 - https://github.com/FasterXML/jackson-core) @@ -34,6 +34,7 @@ Lists of 71 third-party dependencies. (Apache License 2.0) Metrics Integration with JMX (io.dropwizard.metrics:metrics-jmx:4.2.25 - https://metrics.dropwizard.io/metrics-jmx) (Apache License 2.0) swagger-annotations (io.swagger.core.v3:swagger-annotations:2.1.10 - https://github.com/swagger-api/swagger-core/modules/swagger-annotations) (EDL 1.0) Jakarta Activation API (jakarta.activation:jakarta.activation-api:2.1.3 - https://github.com/jakartaee/jaf-api) + (Apache License 2.0) Jakarta Bean Validation API (jakarta.validation:jakarta.validation-api:3.0.2 - https://beanvalidation.org) (Eclipse Distribution License - v 1.0) Jakarta SOAP with Attachments API (jakarta.xml.soap:jakarta.xml.soap-api:3.0.2 - https://github.com/jakartaee/saaj-api) (Apache-2.0) Apache Avro (org.apache.avro:avro:1.11.3 - https://avro.apache.org) (Apache-2.0) Apache Commons Compress (org.apache.commons:commons-compress:1.26.2 - https://commons.apache.org/proper/commons-compress/) diff --git a/ksml/pom.xml b/ksml/pom.xml index ad666d8f..d61278b1 100644 --- a/ksml/pom.xml +++ b/ksml/pom.xml @@ -166,6 +166,12 @@ slf4j-simple test + + jakarta.validation + jakarta.validation-api + 3.0.2 + compile + diff --git a/ksml/src/main/java/io/axual/ksml/definition/parser/FunctionDefinitionParser.java b/ksml/src/main/java/io/axual/ksml/definition/parser/FunctionDefinitionParser.java index 74bbad6a..8a744b7b 100644 --- a/ksml/src/main/java/io/axual/ksml/definition/parser/FunctionDefinitionParser.java +++ b/ksml/src/main/java/io/axual/ksml/definition/parser/FunctionDefinitionParser.java @@ -28,9 +28,9 @@ import io.axual.ksml.definition.ParameterDefinition; import io.axual.ksml.dsl.KSMLDSL; import io.axual.ksml.parser.DefinitionParser; +import io.axual.ksml.parser.IgnoreParser; import io.axual.ksml.parser.StringValueParser; import io.axual.ksml.parser.StructsParser; -import lombok.Getter; import java.util.List; @@ -44,14 +44,14 @@ public FunctionDefinitionParser(boolean requireType) { } protected StructsParser parserWithStores(Class resultClass, String description, Constructor1 constructor) { - return parser(resultClass, description, (name, params, globalCode, code, expression, resultType, stores, tags) -> FunctionDefinition.as(name, params, globalCode, code, expression, resultType, stores), constructor); + return parser(resultClass, description, true, (name, params, globalCode, code, expression, resultType, stores, tags) -> FunctionDefinition.as(name, params, globalCode, code, expression, resultType, stores), constructor); } protected StructsParser parserWithoutStores(Class resultClass, String description, Constructor1 constructor) { - return parser(resultClass, description, (name, params, globalCode, code, expression, resultType, stores, tags) -> FunctionDefinition.as(name, params, globalCode, code, expression, resultType, null), constructor); + return parser(resultClass, description, false, (name, params, globalCode, code, expression, resultType, stores, tags) -> FunctionDefinition.as(name, params, globalCode, code, expression, resultType, null), constructor); } - private StructsParser parser(Class resultClass, String description, Constructor7, String, String, String, UserType, List> innerConstructor, Constructor1 outerConstructor) { + private StructsParser parser(Class resultClass, String description, boolean includeStores, Constructor7, String, String, String, UserType, List> innerConstructor, Constructor1 outerConstructor) { final var parseType = resultClass == FunctionDefinition.class; final var doc = "Defines a " + description + " function, that gets injected into the Kafka Streams topology"; final var name = optional(stringField(Functions.NAME, "The name of the " + description + ". If this field is not defined, then the name is derived from the context.")); @@ -60,8 +60,10 @@ private StructsParser parser(Class resultClass, String description, Constr final var code = optional(codeField(Functions.CODE, "The (multiline) code of the " + description)); final var expression = optional(codeField(Functions.EXPRESSION, "The expression returned by the " + description + ". Only required for functions that return values.")); final var resultType = optional(userTypeField(Functions.RESULT_TYPE, "The data type returned by the " + description + ". Only required for function types, which are not pre-defined.")); - final var stores = optional(listField(Functions.STORES, "store-name", "store", "A list of store names that the " + description + " uses. Only required if the function wants to use a state store.", new StringValueParser())); - // We assume that the resultClass is always either using stores, or not using stores, but not a combination of both. Hence we do not provide a definitionVariant extension to distinguish between the two. + final var stores = includeStores + ? optional(listField(Functions.STORES, "store-name", "store", "A list of store names that the " + description + " uses. Only required if the function wants to use a state store.", new StringValueParser())) + : new IgnoreParser>(); + // We assume that the resultClass is always either using stores, or not using stores, but not a combination of both. Hence, we do not provide a definitionVariant extension to distinguish between the two. final var parser = structsParser(resultClass, parseType || requireType ? "" : KSMLDSL.Types.WITH_IMPLICIT_TYPE_POSTFIX, doc, name, params, globalCode, code, expression, resultType, stores, innerConstructor); return new StructsParser<>() { @Override diff --git a/ksml/src/main/java/io/axual/ksml/definition/parser/PipelineDefinitionParser.java b/ksml/src/main/java/io/axual/ksml/definition/parser/PipelineDefinitionParser.java index 2948a4a3..e3518df9 100644 --- a/ksml/src/main/java/io/axual/ksml/definition/parser/PipelineDefinitionParser.java +++ b/ksml/src/main/java/io/axual/ksml/definition/parser/PipelineDefinitionParser.java @@ -90,7 +90,7 @@ public StructsParser parser() { // If no sink operation was specified, then we create an AS operation here with the name provided. // This means that pipeline results can be referred to by other pipelines using the pipeline's name // as identifier. - var sinkOperation = shortName != null ? new AsOperation(new OperationConfig(resources().getUniqueOperationName(longName), tags, null), shortName) : null; + var sinkOperation = shortName != null ? new AsOperation(new OperationConfig(resources().getUniqueOperationName(longName), tags), shortName) : null; return new PipelineDefinition(name, from, via, sinkOperation); }); } diff --git a/ksml/src/main/java/io/axual/ksml/definition/parser/PredicateDefinitionParser.java b/ksml/src/main/java/io/axual/ksml/definition/parser/PredicateDefinitionParser.java index 7ee914c2..f83ab3a1 100644 --- a/ksml/src/main/java/io/axual/ksml/definition/parser/PredicateDefinitionParser.java +++ b/ksml/src/main/java/io/axual/ksml/definition/parser/PredicateDefinitionParser.java @@ -33,7 +33,7 @@ public PredicateDefinitionParser(boolean requireType) { public StructsParser parser() { return parserWithStores( PredicateDefinition.class, - "Function that returns true or false based on key/value input", + "predicate", (function, tags) -> new PredicateDefinition(function)); } } diff --git a/ksml/src/main/java/io/axual/ksml/dsl/KSMLDSL.java b/ksml/src/main/java/io/axual/ksml/dsl/KSMLDSL.java index 84257f33..596e3ecb 100644 --- a/ksml/src/main/java/io/axual/ksml/dsl/KSMLDSL.java +++ b/ksml/src/main/java/io/axual/ksml/dsl/KSMLDSL.java @@ -86,7 +86,6 @@ public static class Operations { public static final String NAME_ATTRIBUTE = "name"; public static final String TYPE_ATTRIBUTE = "type"; public static final String STORE_ATTRIBUTE = "store"; - public static final String STORE_NAMES_ATTRIBUTE = "stores"; public static final String AGGREGATE = "aggregate"; public static final String COGROUP = "cogroup"; @@ -205,6 +204,7 @@ public static class Reduce { @NoArgsConstructor(access = AccessLevel.PRIVATE) public static class Repartition { + public static final String NUMBER_OF_PARTITIONS = "numberOfPartitions"; public static final String PARTITIONER = "partitioner"; } diff --git a/ksml/src/main/java/io/axual/ksml/metric/KSMLMetrics.java b/ksml/src/main/java/io/axual/ksml/metric/Metrics.java similarity index 95% rename from ksml/src/main/java/io/axual/ksml/metric/KSMLMetrics.java rename to ksml/src/main/java/io/axual/ksml/metric/Metrics.java index a901e73a..5f4f0678 100644 --- a/ksml/src/main/java/io/axual/ksml/metric/KSMLMetrics.java +++ b/ksml/src/main/java/io/axual/ksml/metric/Metrics.java @@ -22,10 +22,10 @@ import java.util.Collections; -public class KSMLMetrics { +public class Metrics { public static final String DOMAIN = "ksml"; - private KSMLMetrics() { + private Metrics() { } private static final MetricsRegistry REGISTRY = new MetricsRegistry(); diff --git a/ksml/src/main/java/io/axual/ksml/operation/AggregateOperation.java b/ksml/src/main/java/io/axual/ksml/operation/AggregateOperation.java index 4ef75373..55b5ebbb 100644 --- a/ksml/src/main/java/io/axual/ksml/operation/AggregateOperation.java +++ b/ksml/src/main/java/io/axual/ksml/operation/AggregateOperation.java @@ -211,61 +211,63 @@ public StreamWrapper apply(CogroupedKStreamWrapper input, TopologyBuildContext c } @Override - public StreamWrapper apply(TimeWindowedCogroupedKStreamWrapper input, TopologyBuildContext context) { + public StreamWrapper apply(SessionWindowedCogroupedKStreamWrapper input, TopologyBuildContext context) { /* * Kafka Streams method signature: * KTable, V> aggregate( * final Initializer initializer, + * final Merger sessionMerger, * final Named named, - * final Materialized> materialized) + * final Materialized> materialized) */ checkNotNull(initializer, INITIALIZER_NAME.toLowerCase()); final var k = input.keyType(); final var v = input.valueType(); + final var windowedK = windowedTypeOf(k); final var init = userFunctionOf(context, INITIALIZER_NAME, initializer, v); final var userInit = new UserInitializer(init, tags); - final var kvStore = validateWindowStore(store(), k, v); - final var mat = materializedOf(context, kvStore); + final var merg = userFunctionOf(context, MERGER_NAME, merger, v); + final var userMerg = new UserMerger(merg, tags); + final var sessionStore = validateSessionStore(store(), k, v); + final var mat = materializedOf(context, sessionStore); final var named = namedOf(); final KTable, Object> output = named != null ? mat != null - ? input.timeWindowedCogroupedKStream.aggregate(userInit, named, mat) - : input.timeWindowedCogroupedKStream.aggregate(userInit, named) + ? input.sessionWindowedCogroupedKStream.aggregate(userInit, userMerg, named, mat) + : input.sessionWindowedCogroupedKStream.aggregate(userInit, userMerg, named) : mat != null - ? input.timeWindowedCogroupedKStream.aggregate(userInit, mat) - : input.timeWindowedCogroupedKStream.aggregate(userInit); - return new KTableWrapper((KTable) output, k, v); + ? input.sessionWindowedCogroupedKStream.aggregate(userInit, userMerg, mat) + : input.sessionWindowedCogroupedKStream.aggregate(userInit, userMerg); + return new KTableWrapper((KTable) output, windowedK, v); } @Override - public StreamWrapper apply(SessionWindowedCogroupedKStreamWrapper input, TopologyBuildContext context) { + public StreamWrapper apply(TimeWindowedCogroupedKStreamWrapper input, TopologyBuildContext context) { /* * Kafka Streams method signature: * KTable, V> aggregate( * final Initializer initializer, - * final Merger sessionMerger, * final Named named, - * final Materialized> materialized) + * final Materialized> materialized) */ checkNotNull(initializer, INITIALIZER_NAME.toLowerCase()); final var k = input.keyType(); final var v = input.valueType(); + final var windowedK = windowedTypeOf(k); final var init = userFunctionOf(context, INITIALIZER_NAME, initializer, v); final var userInit = new UserInitializer(init, tags); - final var merg = userFunctionOf(context, MERGER_NAME, merger, v); - final var userMerg = new UserMerger(merg, tags); - final var sessionStore = validateSessionStore(store(), k, v); - final var mat = materializedOf(context, sessionStore); + final var kvStore = validateWindowStore(store(), k, v); + final var mat = materializedOf(context, kvStore); final var named = namedOf(); final KTable, Object> output = named != null ? mat != null - ? input.sessionWindowedCogroupedKStream.aggregate(userInit, userMerg, named, mat) - : input.sessionWindowedCogroupedKStream.aggregate(userInit, userMerg, named) + ? input.timeWindowedCogroupedKStream.aggregate(userInit, named, mat) + : input.timeWindowedCogroupedKStream.aggregate(userInit, named) : mat != null - ? input.sessionWindowedCogroupedKStream.aggregate(userInit, userMerg, mat) - : input.sessionWindowedCogroupedKStream.aggregate(userInit, userMerg); - return new KTableWrapper((KTable) output, k, v); + ? input.timeWindowedCogroupedKStream.aggregate(userInit, mat) + : input.timeWindowedCogroupedKStream.aggregate(userInit); + return new KTableWrapper((KTable) output, windowedK, v); } } diff --git a/ksml/src/main/java/io/axual/ksml/operation/BaseOperation.java b/ksml/src/main/java/io/axual/ksml/operation/BaseOperation.java index e0bc49b9..93b23708 100644 --- a/ksml/src/main/java/io/axual/ksml/operation/BaseOperation.java +++ b/ksml/src/main/java/io/axual/ksml/operation/BaseOperation.java @@ -40,8 +40,6 @@ import org.apache.kafka.streams.state.WindowStore; import java.util.Arrays; -import java.util.Collections; -import java.util.TreeSet; @Slf4j public class BaseOperation implements StreamOperation { @@ -67,7 +65,6 @@ public static String validateNameAndReturnError(String name) { protected final String name; protected final ContextTags tags; - protected final String[] storeNames; public BaseOperation(OperationConfig config) { var error = NameValidator.validateNameAndReturnError(config.name()); @@ -78,7 +75,6 @@ public BaseOperation(OperationConfig config) { name = config.name(); } tags = config.tags().append("operation-name", name); - storeNames = config.storeNames() != null ? config.storeNames() : new String[0]; } @Override @@ -287,14 +283,6 @@ protected void checkTuple(String faultDescription, DataType type, DataType... el } } - protected String[] combineStoreNames(String[]... storeNameArrays) { - final var storeNames = new TreeSet(); - for (String[] storeNameArray : storeNameArrays) { - if (storeNameArray != null) Collections.addAll(storeNames, storeNameArray); - } - return storeNames.toArray(TEMPLATE); - } - protected StreamDataType streamDataTypeOf(DataType dataType, boolean isKey) { return streamDataTypeOf(new UserType(dataType), isKey); } @@ -355,9 +343,10 @@ protected Materialized> materializedOf return null; } - protected Repartitioned repartitionedOf(StreamDataType k, StreamDataType v, StreamPartitioner partitioner) { + protected Repartitioned repartitionedOf(StreamDataType k, StreamDataType v, Integer numberOfPartitions, StreamPartitioner partitioner) { if (partitioner == null) return null; var repartitioned = Repartitioned.with(k.serde(), v.serde()).withStreamPartitioner(partitioner); + if (numberOfPartitions != null) repartitioned = repartitioned.withNumberOfPartitions(numberOfPartitions); if (name != null) repartitioned = repartitioned.withName(name); return repartitioned; } diff --git a/ksml/src/main/java/io/axual/ksml/operation/CogroupOperation.java b/ksml/src/main/java/io/axual/ksml/operation/CogroupOperation.java index 6aaecec0..60b67be1 100644 --- a/ksml/src/main/java/io/axual/ksml/operation/CogroupOperation.java +++ b/ksml/src/main/java/io/axual/ksml/operation/CogroupOperation.java @@ -69,6 +69,6 @@ public StreamWrapper apply(CogroupedKStreamWrapper input, TopologyBuildContext c // of another pipeline, which then happens to be of Java-type KGroupedStream. It seems a little far-fetched // for KSML users to grasp the technicalities under the hood well enough to use this properly. Therefore, this // is unsupported for now and we throw an exception with this explicit message. - throw new TopologyException("Cogroup operations are not supported for CogroupStreams"); + throw new TopologyException("Cogroup operations are not supported for CogroupedStreams"); } } diff --git a/ksml/src/main/java/io/axual/ksml/operation/FilterNotOperation.java b/ksml/src/main/java/io/axual/ksml/operation/FilterNotOperation.java index 4372317f..cf6277e4 100644 --- a/ksml/src/main/java/io/axual/ksml/operation/FilterNotOperation.java +++ b/ksml/src/main/java/io/axual/ksml/operation/FilterNotOperation.java @@ -54,7 +54,7 @@ public StreamWrapper apply(KStreamWrapper input, TopologyBuildContext context) { final var v = input.valueType(); final var pred = userFunctionOf(context, PREDICATE_NAME, predicate, new UserType(DataBoolean.DATATYPE), superOf(k), superOf(v)); final var userPred = new UserPredicate(pred, tags); - final var storeNames = combineStoreNames(this.storeNames, predicate.storeNames().toArray(TEMPLATE)); + final var storeNames = predicate.storeNames().toArray(String[]::new); final var supplier = new FixedKeyOperationProcessorSupplier<>( name, FilterNotProcessor::new, diff --git a/ksml/src/main/java/io/axual/ksml/operation/FilterOperation.java b/ksml/src/main/java/io/axual/ksml/operation/FilterOperation.java index fa78b4bd..a450b10e 100644 --- a/ksml/src/main/java/io/axual/ksml/operation/FilterOperation.java +++ b/ksml/src/main/java/io/axual/ksml/operation/FilterOperation.java @@ -54,7 +54,7 @@ public StreamWrapper apply(KStreamWrapper input, TopologyBuildContext context) { final var v = input.valueType(); final var pred = userFunctionOf(context, PREDICATE_NAME, predicate, new UserType(DataBoolean.DATATYPE), superOf(k), superOf(v)); final var userPred = new UserPredicate(pred, tags); - final var storeNames = combineStoreNames(this.storeNames, predicate.storeNames().toArray(TEMPLATE)); + final var storeNames = predicate.storeNames().toArray(String[]::new); final var supplier = new FixedKeyOperationProcessorSupplier<>( name, FilterProcessor::new, diff --git a/ksml/src/main/java/io/axual/ksml/operation/JoinOperation.java b/ksml/src/main/java/io/axual/ksml/operation/JoinOperation.java index 8bcd9495..41afffe3 100644 --- a/ksml/src/main/java/io/axual/ksml/operation/JoinOperation.java +++ b/ksml/src/main/java/io/axual/ksml/operation/JoinOperation.java @@ -107,14 +107,13 @@ public StreamWrapper apply(KStreamWrapper input, TopologyBuildContext context) { final var vr = streamDataTypeOf(firstSpecificType(valueJoiner, vo, v), false); checkType("Join stream keyType", ko, equalTo(k)); final var joiner = userFunctionOf(context, VALUEJOINER_NAME, valueJoiner, vr, superOf(k), superOf(v), superOf(vo)); - final var windowedK = windowedTypeOf(k); final var windowStore = validateWindowStore(store(), k, vr); final var streamJoined = streamJoinedOf(windowStore, k, v, vo); final var userJoiner = new UserValueJoinerWithKey(joiner, tags); final KStream output = streamJoined != null ? input.stream.join(otherStream.stream, userJoiner, joinWindows, streamJoined) : input.stream.join(otherStream.stream, userJoiner, joinWindows); - return new KStreamWrapper(output, windowedK, vr); + return new KStreamWrapper(output, k, vr); } if (joinTopic instanceof TableDefinition joinTable) { diff --git a/ksml/src/main/java/io/axual/ksml/operation/OperationConfig.java b/ksml/src/main/java/io/axual/ksml/operation/OperationConfig.java index 1525a529..399e313a 100644 --- a/ksml/src/main/java/io/axual/ksml/operation/OperationConfig.java +++ b/ksml/src/main/java/io/axual/ksml/operation/OperationConfig.java @@ -29,14 +29,10 @@ public class OperationConfig { private final String name; private final ContextTags tags; - private final boolean allowStores; - private final String[] storeNames; - public OperationConfig(String name, ContextTags tags, String[] storeNames) { + public OperationConfig(String name, ContextTags tags) { this.name = name; this.tags = tags; log.debug("Generated operation name: {}", this.name); - this.allowStores = storeNames != null; - this.storeNames = storeNames; } } diff --git a/ksml/src/main/java/io/axual/ksml/operation/PeekOperation.java b/ksml/src/main/java/io/axual/ksml/operation/PeekOperation.java index 2d12dd09..20bfe34d 100644 --- a/ksml/src/main/java/io/axual/ksml/operation/PeekOperation.java +++ b/ksml/src/main/java/io/axual/ksml/operation/PeekOperation.java @@ -46,7 +46,7 @@ public StreamWrapper apply(KStreamWrapper input, TopologyBuildContext context) { final var v = input.valueType(); final var action = userFunctionOf(context, FOREACHACTION_NAME, forEachAction, equalTo(DataNull.DATATYPE), superOf(k), superOf(v)); final var userAction = new UserForeachAction(action, tags); - final var storeNames = combineStoreNames(this.storeNames, forEachAction.storeNames().toArray(TEMPLATE)); + final var storeNames = forEachAction.storeNames().toArray(String[]::new); final var supplier = new FixedKeyOperationProcessorSupplier<>( name, PeekProcessor::new, diff --git a/ksml/src/main/java/io/axual/ksml/operation/ReduceOperation.java b/ksml/src/main/java/io/axual/ksml/operation/ReduceOperation.java index 657beb54..b49f5bae 100644 --- a/ksml/src/main/java/io/axual/ksml/operation/ReduceOperation.java +++ b/ksml/src/main/java/io/axual/ksml/operation/ReduceOperation.java @@ -36,9 +36,16 @@ public class ReduceOperation extends StoreOperation { private final FunctionDefinition adder; private final FunctionDefinition subtractor; - public ReduceOperation(StoreOperationConfig config, FunctionDefinition reducer, FunctionDefinition adder, FunctionDefinition subtractor) { + public ReduceOperation(StoreOperationConfig config, FunctionDefinition reducer) { super(config); this.reducer = reducer; + this.adder = null; + this.subtractor = null; + } + + public ReduceOperation(StoreOperationConfig config, FunctionDefinition adder, FunctionDefinition subtractor) { + super(config); + this.reducer = null; this.adder = adder; this.subtractor = subtractor; } diff --git a/ksml/src/main/java/io/axual/ksml/operation/RepartitionOperation.java b/ksml/src/main/java/io/axual/ksml/operation/RepartitionOperation.java index 8c5ce7b0..230613c3 100644 --- a/ksml/src/main/java/io/axual/ksml/operation/RepartitionOperation.java +++ b/ksml/src/main/java/io/axual/ksml/operation/RepartitionOperation.java @@ -31,10 +31,12 @@ public class RepartitionOperation extends BaseOperation { private static final String PARTITIONER_NAME = "Partitioner"; + private final Integer numberOfPartitions; private final FunctionDefinition partitioner; - public RepartitionOperation(OperationConfig config, FunctionDefinition partitioner) { + public RepartitionOperation(OperationConfig config, Integer numberOfPartitions, FunctionDefinition partitioner) { super(config); + this.numberOfPartitions = numberOfPartitions; this.partitioner = partitioner; } @@ -50,7 +52,7 @@ public StreamWrapper apply(KStreamWrapper input, TopologyBuildContext context) { final var v = input.valueType(); final var part = userFunctionOf(context, PARTITIONER_NAME, partitioner, equalTo(DataInteger.DATATYPE), equalTo(DataString.DATATYPE), superOf(k), superOf(v), equalTo(DataInteger.DATATYPE)); final var userPart = part != null ? new UserStreamPartitioner(part, tags) : null; - final var repartitioned = repartitionedOf(k, v, userPart); + final var repartitioned = repartitionedOf(k, v, numberOfPartitions,userPart); final KStream output = repartitioned != null ? input.stream.repartition(repartitioned) : input.stream.repartition(); diff --git a/ksml/src/main/java/io/axual/ksml/operation/StoreOperationConfig.java b/ksml/src/main/java/io/axual/ksml/operation/StoreOperationConfig.java index 46cbd182..79204588 100644 --- a/ksml/src/main/java/io/axual/ksml/operation/StoreOperationConfig.java +++ b/ksml/src/main/java/io/axual/ksml/operation/StoreOperationConfig.java @@ -23,13 +23,11 @@ import io.axual.ksml.data.tag.ContextTags; import io.axual.ksml.definition.StateStoreDefinition; -import java.util.List; - public class StoreOperationConfig extends OperationConfig { public final StateStoreDefinition store; - public StoreOperationConfig(String name, ContextTags context, StateStoreDefinition store, List storeNames) { - super(name, context, storeNames != null ? storeNames.toArray(new String[]{}) : null); + public StoreOperationConfig(String name, ContextTags context, StateStoreDefinition store) { + super(name, context); this.store = store; } } diff --git a/ksml/src/main/java/io/axual/ksml/operation/TransformKeyOperation.java b/ksml/src/main/java/io/axual/ksml/operation/TransformKeyOperation.java index bd2547aa..ff719721 100644 --- a/ksml/src/main/java/io/axual/ksml/operation/TransformKeyOperation.java +++ b/ksml/src/main/java/io/axual/ksml/operation/TransformKeyOperation.java @@ -53,7 +53,7 @@ public StreamWrapper apply(KStreamWrapper input, TopologyBuildContext context) { final var kr = streamDataTypeOf(firstSpecificType(mapper, k), true); final var map = userFunctionOf(context, MAPPER_NAME, mapper, kr, superOf(k), superOf(v)); final var userMap = new UserKeyTransformer(map, tags); - final var storeNames = combineStoreNames(this.storeNames, mapper.storeNames().toArray(TEMPLATE)); + final var storeNames = mapper.storeNames().toArray(String[]::new); final var supplier = new OperationProcessorSupplier<>( name, TransformKeyProcessor::new, diff --git a/ksml/src/main/java/io/axual/ksml/operation/TransformKeyValueOperation.java b/ksml/src/main/java/io/axual/ksml/operation/TransformKeyValueOperation.java index 78b7769e..0c506fc6 100644 --- a/ksml/src/main/java/io/axual/ksml/operation/TransformKeyValueOperation.java +++ b/ksml/src/main/java/io/axual/ksml/operation/TransformKeyValueOperation.java @@ -62,7 +62,7 @@ public BaseStreamWrapper apply(KStreamWrapper input, TopologyBuildContext contex final var kr = streamDataTypeOf(userTupleType.getUserType(0), true); final var vr = streamDataTypeOf(userTupleType.getUserType(1), false); final var userMap = new UserKeyValueTransformer(map, tags); - final var storeNames = combineStoreNames(this.storeNames, mapper.storeNames().toArray(TEMPLATE)); + final var storeNames = mapper.storeNames().toArray(String[]::new); final var supplier = new OperationProcessorSupplier<>( name, TransformKeyValueProcessor::new, diff --git a/ksml/src/main/java/io/axual/ksml/operation/TransformKeyValueToKeyValueListOperation.java b/ksml/src/main/java/io/axual/ksml/operation/TransformKeyValueToKeyValueListOperation.java index 548bdc02..dd48afad 100644 --- a/ksml/src/main/java/io/axual/ksml/operation/TransformKeyValueToKeyValueListOperation.java +++ b/ksml/src/main/java/io/axual/ksml/operation/TransformKeyValueToKeyValueListOperation.java @@ -64,7 +64,7 @@ public StreamWrapper apply(KStreamWrapper input, TopologyBuildContext context) { final var kr = streamDataTypeOf(mapperResultListTupleValueType.getUserType(0), true); final var vr = streamDataTypeOf(mapperResultListTupleValueType.getUserType(1), false); final var userMap = new UserKeyValueToKeyValueListTransformer(map, tags); - final var storeNames = combineStoreNames(this.storeNames, mapper.storeNames().toArray(TEMPLATE)); + final var storeNames = mapper.storeNames().toArray(String[]::new); final var supplier = new OperationProcessorSupplier<>( name, TransformKeyValueToKeyValueListProcessor::new, diff --git a/ksml/src/main/java/io/axual/ksml/operation/TransformKeyValueToValueListOperation.java b/ksml/src/main/java/io/axual/ksml/operation/TransformKeyValueToValueListOperation.java index 16256b93..defd083a 100644 --- a/ksml/src/main/java/io/axual/ksml/operation/TransformKeyValueToValueListOperation.java +++ b/ksml/src/main/java/io/axual/ksml/operation/TransformKeyValueToValueListOperation.java @@ -55,7 +55,7 @@ public StreamWrapper apply(KStreamWrapper input, TopologyBuildContext context) { final var vr = streamDataTypeOf(firstSpecificType(mapper, new UserType(new ListType(DataType.UNKNOWN))), false); final var map = userFunctionOf(context, MAPPER_NAME, mapper, subOf(vr), superOf(k), superOf(v)); final var userMap = new UserKeyValueToValueListTransformer(map, tags); - final var storeNames = combineStoreNames(this.storeNames, mapper.storeNames().toArray(TEMPLATE)); + final var storeNames = mapper.storeNames().toArray(String[]::new); final var supplier = new FixedKeyOperationProcessorSupplier<>( name, TransformKeyValueToValueListProcessor::new, diff --git a/ksml/src/main/java/io/axual/ksml/operation/TransformMetadataOperation.java b/ksml/src/main/java/io/axual/ksml/operation/TransformMetadataOperation.java index 8f766978..7b057ac6 100644 --- a/ksml/src/main/java/io/axual/ksml/operation/TransformMetadataOperation.java +++ b/ksml/src/main/java/io/axual/ksml/operation/TransformMetadataOperation.java @@ -54,7 +54,7 @@ public StreamWrapper apply(KStreamWrapper input, TopologyBuildContext context) { final var meta = new UserType(RecordMetadata.DATATYPE); final var map = userFunctionOf(context, MAPPER_NAME, mapper, subOf(meta), superOf(k), superOf(v), superOf(meta)); final var userMap = new UserMetadataTransformer(map, tags); - final var storeNames = combineStoreNames(this.storeNames, mapper.storeNames().toArray(TEMPLATE)); + final var storeNames = mapper.storeNames().toArray(String[]::new); final var supplier = new FixedKeyOperationProcessorSupplier<>( name, TransformMetadataProcessor::new, diff --git a/ksml/src/main/java/io/axual/ksml/operation/TransformValueOperation.java b/ksml/src/main/java/io/axual/ksml/operation/TransformValueOperation.java index 73c55aab..0bdea0d8 100644 --- a/ksml/src/main/java/io/axual/ksml/operation/TransformValueOperation.java +++ b/ksml/src/main/java/io/axual/ksml/operation/TransformValueOperation.java @@ -57,7 +57,7 @@ public StreamWrapper apply(KStreamWrapper input, TopologyBuildContext context) { final var vr = streamDataTypeOf(firstSpecificType(mapper, v.userType()), false); final var map = userFunctionOf(context, MAPPER_NAME, mapper, vr, superOf(k), superOf(v)); final var userMap = new UserValueTransformer(map, tags); - final var storeNames = combineStoreNames(this.storeNames, mapper.storeNames().toArray(TEMPLATE)); + final var storeNames = mapper.storeNames().toArray(String[]::new); final var supplier = new FixedKeyOperationProcessorSupplier<>( name, TransformValueProcessor::new, @@ -89,6 +89,7 @@ public StreamWrapper apply(KTableWrapper input, TopologyBuildContext context) { final ValueTransformerWithKeySupplier supplier = () -> userMap; final var named = namedOf(); final var mat = materializedOf(context, kvStore); + final var storeNames = mapper.storeNames().toArray(String[]::new); final KTable output = named != null ? mat != null ? input.table.transformValues(supplier, mat, named, storeNames) diff --git a/ksml/src/main/java/io/axual/ksml/operation/parser/FilterNotOperationParser.java b/ksml/src/main/java/io/axual/ksml/operation/parser/FilterNotOperationParser.java index 63262994..01712eec 100644 --- a/ksml/src/main/java/io/axual/ksml/operation/parser/FilterNotOperationParser.java +++ b/ksml/src/main/java/io/axual/ksml/operation/parser/FilterNotOperationParser.java @@ -42,10 +42,9 @@ public StructsParser parser() { operationNameField(), functionField(KSMLDSL.Operations.Filter.PREDICATE, "A function that returns \"false\" when records are accepted, \"true\" otherwise", new PredicateDefinitionParser(false)), storeField(false, "Materialized view of the filtered table (only applies to tables, ignored for streams)", StoreType.KEYVALUE_STORE), - storeNamesField(), - (name, pred, store, stores, tags) -> { + (name, pred, store, tags) -> { if (pred != null) - return new FilterNotOperation(storeOperationConfig(name, tags, store, stores), pred); + return new FilterNotOperation(storeOperationConfig(name, tags, store), pred); throw new ExecutionException("Predicate not defined for " + type + " operation"); }); } diff --git a/ksml/src/main/java/io/axual/ksml/operation/parser/FilterOperationParser.java b/ksml/src/main/java/io/axual/ksml/operation/parser/FilterOperationParser.java index b4d93161..af5e043f 100644 --- a/ksml/src/main/java/io/axual/ksml/operation/parser/FilterOperationParser.java +++ b/ksml/src/main/java/io/axual/ksml/operation/parser/FilterOperationParser.java @@ -42,10 +42,9 @@ public StructsParser parser() { operationNameField(), functionField(KSMLDSL.Operations.Filter.PREDICATE, "A function that returns \"true\" when records are accepted, \"false\" otherwise", new PredicateDefinitionParser(false)), storeField(false, "Materialized view of the filtered table (only applies to tables, ignored for streams)", StoreType.KEYVALUE_STORE), - storeNamesField(), - (name, pred, store, stores, tags) -> { + (name, pred, store, tags) -> { if (pred != null) - return new FilterOperation(storeOperationConfig(name, tags, store, stores), pred); + return new FilterOperation(storeOperationConfig(name, tags, store), pred); throw new ExecutionException("Predicate not defined for " + type + " operation"); }); } diff --git a/ksml/src/main/java/io/axual/ksml/operation/parser/ForEachOperationParser.java b/ksml/src/main/java/io/axual/ksml/operation/parser/ForEachOperationParser.java index b04e06c3..f364c03f 100644 --- a/ksml/src/main/java/io/axual/ksml/operation/parser/ForEachOperationParser.java +++ b/ksml/src/main/java/io/axual/ksml/operation/parser/ForEachOperationParser.java @@ -38,7 +38,6 @@ public StructsParser parser() { "Operation to call a function for every record in the stream", operationNameField(), functionField(KSMLDSL.Operations.FOR_EACH, "A function that gets called for every message in the stream", new ForEachActionDefinitionParser(false)), - storeNamesField(), - (name, action, stores, tags) -> action != null ? new ForEachOperation(operationConfig(name, tags, stores), action) : null); + (name, action, tags) -> action != null ? new ForEachOperation(operationConfig(name, tags), action) : null); } } diff --git a/ksml/src/main/java/io/axual/ksml/operation/parser/JoinOperationParser.java b/ksml/src/main/java/io/axual/ksml/operation/parser/JoinOperationParser.java index 45caf946..79fe0ff2 100644 --- a/ksml/src/main/java/io/axual/ksml/operation/parser/JoinOperationParser.java +++ b/ksml/src/main/java/io/axual/ksml/operation/parser/JoinOperationParser.java @@ -68,7 +68,6 @@ public JoinOperationParser(TopologyResources resources) { JoinOperation.class, KSMLDSL.Types.WITH_TABLE, "Operation to join with a table", - stringField(KSMLDSL.Operations.TYPE_ATTRIBUTE, "The type of the operation, fixed value \"" + Operations.JOIN + "\""), operationNameField(), topicField(Operations.Join.WITH_TABLE, "A reference to the table, or an inline definition of the table to join with", new TableDefinitionParser(resources(), false)), optional(functionField(Operations.Join.FOREIGN_KEY_EXTRACTOR, "A function that can translate the join table value to a primary key", new ValueJoinerDefinitionParser(false))), @@ -77,7 +76,7 @@ public JoinOperationParser(TopologyResources resources) { optional(functionField(Operations.Join.PARTITIONER, "A function that partitions the records on the primary table", new StreamPartitionerDefinitionParser(false))), optional(functionField(Operations.Join.OTHER_PARTITIONER, "A function that partitions the records on the join table", new StreamPartitionerDefinitionParser(false))), storeField(false, "Materialized view of the joined streams", null), - (type, name, table, foreignKeyExtractor, valueJoiner, grace, partitioner, otherPartitioner, store, tags) -> { + (name, table, foreignKeyExtractor, valueJoiner, grace, partitioner, otherPartitioner, store, tags) -> { if (table instanceof TableDefinition tableDef) { return new JoinOperation(storeOperationConfig(name, tags, store), tableDef, foreignKeyExtractor, valueJoiner, grace, partitioner, otherPartitioner); } @@ -88,15 +87,14 @@ public JoinOperationParser(TopologyResources resources) { JoinOperation.class, KSMLDSL.Types.WITH_GLOBAL_TABLE, "Operation to join with a table", - stringField(KSMLDSL.Operations.TYPE_ATTRIBUTE, "The type of the operation, fixed value \"" + Operations.JOIN + "\""), operationNameField(), topicField(Operations.Join.WITH_GLOBAL_TABLE, "A reference to the globalTable, or an inline definition of the globalTable to join with", new GlobalTableDefinitionParser(resources(), false)), - functionField(Operations.Join.MAPPER, "A function that maps the key value from the stream with the primary key of the globalTable", new ValueJoinerDefinitionParser(false)), + functionField(Operations.Join.MAPPER, "A function that maps the key value from the stream to the primary key type of the globalTable", new ValueJoinerDefinitionParser(false)), functionField(Operations.Join.VALUE_JOINER, "A function that joins two values", new ValueJoinerDefinitionParser(false)), - storeField(false, "Materialized view of the joined streams", null), - (type, name, globalTable, mapper, valueJoiner, store, tags) -> { + // GlobalTable joins do not use/require a state store + (name, globalTable, mapper, valueJoiner, tags) -> { if (globalTable instanceof GlobalTableDefinition globalTableDef) { - return new JoinOperation(storeOperationConfig(name, tags, store), globalTableDef, mapper, valueJoiner); + return new JoinOperation(storeOperationConfig(name, tags, null), globalTableDef, mapper, valueJoiner); } throw new TopologyException("Join globalTable not correct, should be a defined globalTable"); }); diff --git a/ksml/src/main/java/io/axual/ksml/operation/parser/LeftJoinOperationParser.java b/ksml/src/main/java/io/axual/ksml/operation/parser/LeftJoinOperationParser.java index f94c116e..23b460c4 100644 --- a/ksml/src/main/java/io/axual/ksml/operation/parser/LeftJoinOperationParser.java +++ b/ksml/src/main/java/io/axual/ksml/operation/parser/LeftJoinOperationParser.java @@ -91,10 +91,10 @@ public LeftJoinOperationParser(TopologyResources resources) { topicField(Operations.Join.WITH_GLOBAL_TABLE, "A reference to the globalTable, or an inline definition of the globalTable to join with", new GlobalTableDefinitionParser(resources(), false)), functionField(Operations.Join.MAPPER, "A function that maps the key value from the stream with the primary key of the globalTable", new ValueJoinerDefinitionParser(false)), functionField(Operations.Join.VALUE_JOINER, "A function that joins two values", new ValueJoinerDefinitionParser(false)), - storeField(false, "Materialized view of the leftJoined streams", null), - (name, globalTable, mapper, valueJoiner, store, tags) -> { + // GlobalTable joins do not use/require a state store + (name, globalTable, mapper, valueJoiner, tags) -> { if (globalTable instanceof GlobalTableDefinition globalTableDef) { - return new LeftJoinOperation(storeOperationConfig(name, tags, store), globalTableDef, mapper, valueJoiner); + return new LeftJoinOperation(storeOperationConfig(name, tags, null), globalTableDef, mapper, valueJoiner); } throw new TopologyException("LeftJoin globalTable not correct, should be a defined globalTable"); }); diff --git a/ksml/src/main/java/io/axual/ksml/operation/parser/OperationParser.java b/ksml/src/main/java/io/axual/ksml/operation/parser/OperationParser.java index 568b49b3..d1f06469 100644 --- a/ksml/src/main/java/io/axual/ksml/operation/parser/OperationParser.java +++ b/ksml/src/main/java/io/axual/ksml/operation/parser/OperationParser.java @@ -26,7 +26,6 @@ import io.axual.ksml.generator.TopologyResources; import io.axual.ksml.operation.BaseOperation; import io.axual.ksml.operation.OperationConfig; -import io.axual.ksml.parser.StringValueParser; import io.axual.ksml.parser.StructsParser; import io.axual.ksml.parser.TopologyResourceAwareParser; import lombok.Getter; @@ -48,10 +47,6 @@ protected StructsParser operationNameField() { return optional(stringField(KSMLDSL.Operations.NAME_ATTRIBUTE, false, type, "The name of the operation processor")); } - protected StructsParser> storeNamesField() { - return optional(listField(KSMLDSL.Operations.STORE_NAMES_ATTRIBUTE, "store", "state store name", "The names of all state stores used by the function", new StringValueParser())); - } - protected OperationConfig operationConfig(String name, ContextTags tags) { return operationConfig(name, tags, null); } @@ -60,8 +55,7 @@ protected OperationConfig operationConfig(String name, ContextTags tags, List parser() { "Operation to peek into a stream, without modifying the stream contents", operationNameField(), functionField(KSMLDSL.Operations.FOR_EACH, "A function that gets called for every message in the stream", new ForEachActionDefinitionParser(false)), - storeNamesField(), - (name, action, stores, tags) -> new PeekOperation(operationConfig(name, tags, stores), action)); + (name, action, tags) -> new PeekOperation(operationConfig(name, tags), action)); } } diff --git a/ksml/src/main/java/io/axual/ksml/operation/parser/PrintOperationParser.java b/ksml/src/main/java/io/axual/ksml/operation/parser/PrintOperationParser.java index a9c2d121..c4c359a0 100644 --- a/ksml/src/main/java/io/axual/ksml/operation/parser/PrintOperationParser.java +++ b/ksml/src/main/java/io/axual/ksml/operation/parser/PrintOperationParser.java @@ -38,7 +38,7 @@ public StructsParser parser() { "", "Operation to print the contents of a pipeline on the screen or to write them to a file", operationNameField(), - optional(stringField(KSMLDSL.Operations.Print.FILENAME, "The filename to output records to. If nothing is specified, then messages will be printed on stdout")), + optional(stringField(KSMLDSL.Operations.Print.FILENAME, "The filename to output records to. If nothing is specified, then messages will be printed on stdout.")), optional(stringField(KSMLDSL.Operations.Print.LABEL, "A label to attach to the output records")), optional(functionField(KSMLDSL.Operations.Print.MAPPER, "A function to convert record into a string for output", new KeyValuePrinterDefinitionParser(false))), (name, filename, label, mapper, tags) -> new PrintOperation(operationConfig(name, tags), filename, label, mapper)); diff --git a/ksml/src/main/java/io/axual/ksml/operation/parser/ReduceOperationParser.java b/ksml/src/main/java/io/axual/ksml/operation/parser/ReduceOperationParser.java index 20c29cee..b1558bf0 100644 --- a/ksml/src/main/java/io/axual/ksml/operation/parser/ReduceOperationParser.java +++ b/ksml/src/main/java/io/axual/ksml/operation/parser/ReduceOperationParser.java @@ -21,30 +21,59 @@ */ +import io.axual.ksml.data.schema.StructSchema; import io.axual.ksml.definition.parser.ReducerDefinitionParser; import io.axual.ksml.dsl.KSMLDSL; +import io.axual.ksml.exception.TopologyException; import io.axual.ksml.generator.TopologyResources; import io.axual.ksml.operation.ReduceOperation; import io.axual.ksml.parser.StructsParser; import io.axual.ksml.store.StoreType; +import java.util.ArrayList; +import java.util.List; + public class ReduceOperationParser extends StoreOperationParser { + private static final String DOC = "Operation to reduce a series of records into a single aggregate result"; + private final StructsParser reducerParser; + private final StructsParser addedSubtractorParser; + private final List schemas; + public ReduceOperationParser(TopologyResources resources) { super(KSMLDSL.Operations.REDUCE, resources); - } - - @Override - public StructsParser parser() { final var storeField = storeField(false, "Materialized view of the aggregation", StoreType.WINDOW_STORE); - return structsParser( + reducerParser = structsParser( ReduceOperation.class, - "", - "Operation to reduce a series of records into a single aggregate result", + "WithReducer", + DOC, operationNameField(), functionField(KSMLDSL.Operations.Reduce.REDUCER, "A function that computes a new aggregate result", new ReducerDefinitionParser(false)), + storeField, + (name, reducer, store, tags) -> new ReduceOperation(storeOperationConfig(name, tags, store), reducer)); + addedSubtractorParser = structsParser( + ReduceOperation.class, + "WithAdderAndSubtractor", + DOC, + operationNameField(), functionField(KSMLDSL.Operations.Reduce.ADDER, "A function that adds a record to the aggregate result", new ReducerDefinitionParser(false)), functionField(KSMLDSL.Operations.Reduce.SUBTRACTOR, "A function that removes a record from the aggregate result", new ReducerDefinitionParser(false)), storeField, - (name, reducer, add, sub, store, tags) -> new ReduceOperation(storeOperationConfig(name, tags, store), reducer, add, sub)); + (name, add, sub, store, tags) -> new ReduceOperation(storeOperationConfig(name, tags, store), add, sub)); + schemas = new ArrayList<>(); + schemas.addAll(reducerParser.schemas()); + schemas.addAll(addedSubtractorParser.schemas()); + } + + @Override + public StructsParser parser() { + return StructsParser.of( + node -> { + final var result1 = reducerParser.parse(node); + if (result1 != null) return result1; + final var result2 = addedSubtractorParser.parse(node); + if (result2 != null) return result2; + throw new TopologyException("Error in reducer operation: " + node); + }, + schemas); } } diff --git a/ksml/src/main/java/io/axual/ksml/operation/parser/RepartitionOperationParser.java b/ksml/src/main/java/io/axual/ksml/operation/parser/RepartitionOperationParser.java index 7ce67ed2..1a6774d4 100644 --- a/ksml/src/main/java/io/axual/ksml/operation/parser/RepartitionOperationParser.java +++ b/ksml/src/main/java/io/axual/ksml/operation/parser/RepartitionOperationParser.java @@ -39,7 +39,8 @@ public StructsParser parser() { "", "Operation to (re)partition a stream", operationNameField(), - functionField(KSMLDSL.Operations.Repartition.PARTITIONER, "A function that partitions stream records", new StreamPartitionerDefinitionParser(false)), - (name, partitioner, tags) -> new RepartitionOperation(operationConfig(name, tags), partitioner)); + optional(integerField(KSMLDSL.Operations.Repartition.NUMBER_OF_PARTITIONS, "The target number of partitions")), + optional(functionField(KSMLDSL.Operations.Repartition.PARTITIONER, "A function that partitions stream records", new StreamPartitionerDefinitionParser(false))), + (name, numberOfPartitions, partitioner, tags) -> new RepartitionOperation(operationConfig(name, tags), numberOfPartitions, partitioner)); } } diff --git a/ksml/src/main/java/io/axual/ksml/operation/parser/StoreOperationParser.java b/ksml/src/main/java/io/axual/ksml/operation/parser/StoreOperationParser.java index a23b7af3..5176eb79 100644 --- a/ksml/src/main/java/io/axual/ksml/operation/parser/StoreOperationParser.java +++ b/ksml/src/main/java/io/axual/ksml/operation/parser/StoreOperationParser.java @@ -44,12 +44,8 @@ public StoreOperationParser(String type, TopologyResources resources) { } protected StoreOperationConfig storeOperationConfig(String name, ContextTags tags, StateStoreDefinition store) { - return storeOperationConfig(name, tags, store, null); - } - - protected StoreOperationConfig storeOperationConfig(String name, ContextTags tags, StateStoreDefinition store, List storeNames) { name = validateName("Store", name, defaultShortName(), true); - return new StoreOperationConfig(name != null ? resources().getUniqueOperationName(name) : resources().getUniqueOperationName(tags), tags, store, storeNames); + return new StoreOperationConfig(name != null ? resources().getUniqueOperationName(name) : resources().getUniqueOperationName(tags), tags, store); } protected StructsParser storeField(boolean required, String doc, StoreType expectedStoreType) { diff --git a/ksml/src/main/java/io/axual/ksml/operation/parser/SuppressOperationParser.java b/ksml/src/main/java/io/axual/ksml/operation/parser/SuppressOperationParser.java index 27f989b2..8c775460 100644 --- a/ksml/src/main/java/io/axual/ksml/operation/parser/SuppressOperationParser.java +++ b/ksml/src/main/java/io/axual/ksml/operation/parser/SuppressOperationParser.java @@ -21,50 +21,73 @@ */ +import io.axual.ksml.data.schema.EnumSchema; import io.axual.ksml.dsl.KSMLDSL; import io.axual.ksml.exception.TopologyException; import io.axual.ksml.generator.TopologyResources; import io.axual.ksml.operation.SuppressOperation; +import io.axual.ksml.parser.ChoiceParser; import io.axual.ksml.parser.StructsParser; import org.apache.kafka.streams.kstream.Suppressed; +import java.util.List; +import java.util.Map; + import static io.axual.ksml.dsl.KSMLDSL.Operations; public class SuppressOperationParser extends OperationParser { + private final StructsParser untilTimeLimitParser; + private final StructsParser untilWindowClosesParser; + public SuppressOperationParser(TopologyResources resources) { super(KSMLDSL.Operations.SUPPRESS, resources); - } + final var bufferFullSchema = new EnumSchema( + SCHEMA_NAMESPACE, + "BufferFullStrategy", + "What to do when the buffer is full", + List.of(Operations.Suppress.BUFFER_FULL_STRATEGY_EMIT, Operations.Suppress.BUFFER_FULL_STRATEGY_SHUTDOWN)); - public StructsParser parser() { - return structsParser( + untilTimeLimitParser = structsParser( SuppressOperation.class, - "", - "Operation to suppress messages in the source stream until a certain limit is reached", + "UntilTimeLimit", + "Operation to suppress messages in the source stream until a time limit is reached", operationNameField(), - stringField(Operations.Suppress.UNTIL, "The method by which messages are held, either \"" + Operations.Suppress.UNTIL_TIME_LIMIT + "\", or \"" + Operations.Suppress.UNTIL_WINDOW_CLOSES + "\""), durationField(Operations.Suppress.DURATION, "The duration for which messages are suppressed"), optional(stringField(Operations.Suppress.BUFFER_MAXBYTES, "The maximum number of bytes in the buffer")), optional(stringField(Operations.Suppress.BUFFER_MAXRECORDS, "The maximum number of records in the buffer")), - optional(stringField(Operations.Suppress.BUFFER_FULL_STRATEGY, "What to do when the buffer is full, either \"" + Operations.Suppress.BUFFER_FULL_STRATEGY_EMIT + "\", or \"" + Operations.Suppress.BUFFER_FULL_STRATEGY_SHUTDOWN + "\"")), - (name, until, duration, maxBytes, maxRecords, strategy, tags) -> { - switch (until) { - case Operations.Suppress.UNTIL_TIME_LIMIT -> { - final var bufferConfig = bufferConfig(maxBytes, maxRecords, strategy); - return SuppressOperation.create( - operationConfig(name, tags), - Suppressed.untilTimeLimit(duration, bufferConfig)); - } - case Operations.Suppress.UNTIL_WINDOW_CLOSES -> { - final var bufferConfig = strictBufferConfig(bufferConfig(maxBytes, maxRecords, strategy)); - return SuppressOperation.createWindowed( - operationConfig(name, tags), - Suppressed.untilWindowCloses(bufferConfig)); - } - } - throw new TopologyException("Unknown Until type for suppress operation: " + until); + optional(enumField(Operations.Suppress.BUFFER_FULL_STRATEGY, bufferFullSchema)), + (name, duration, maxBytes, maxRecords, strategy, tags) -> { + final var bufferConfig = bufferConfig(maxBytes, maxRecords, strategy); + return SuppressOperation.create( + operationConfig(name, tags), + Suppressed.untilTimeLimit(duration, bufferConfig)); + }); + untilWindowClosesParser = structsParser( + SuppressOperation.class, + "UntilWindowCloses", + "Operation to suppress messages in the source stream until a window limit is reached", + operationNameField(), + optional(stringField(Operations.Suppress.BUFFER_MAXBYTES, "The maximum number of bytes in the buffer")), + optional(stringField(Operations.Suppress.BUFFER_MAXRECORDS, "The maximum number of records in the buffer")), + optional(enumField(Operations.Suppress.BUFFER_FULL_STRATEGY, bufferFullSchema)), + (name, maxBytes, maxRecords, strategy, tags) -> { + final var bufferConfig = strictBufferConfig(bufferConfig(maxBytes, maxRecords, strategy)); + return SuppressOperation.createWindowed( + operationConfig(name, tags), + Suppressed.untilWindowCloses(bufferConfig)); }); } + public StructsParser parser() { + return new ChoiceParser<>( + Operations.Suppress.UNTIL, + "SuppressType", + "Operation to suppress messages in the source stream until a certain limit is reached", + null, + Map.of(Operations.Suppress.UNTIL_TIME_LIMIT, untilTimeLimitParser, + Operations.Suppress.UNTIL_WINDOW_CLOSES, untilWindowClosesParser)); + } + private Suppressed.EagerBufferConfig bufferConfig(String maxBytes, String maxRecords, String bufferFullStrategy) { Suppressed.EagerBufferConfig result = null; diff --git a/ksml/src/main/java/io/axual/ksml/operation/parser/TransformKeyOperationParser.java b/ksml/src/main/java/io/axual/ksml/operation/parser/TransformKeyOperationParser.java index 55ec3abc..6de24116 100644 --- a/ksml/src/main/java/io/axual/ksml/operation/parser/TransformKeyOperationParser.java +++ b/ksml/src/main/java/io/axual/ksml/operation/parser/TransformKeyOperationParser.java @@ -40,7 +40,6 @@ protected StructsParser parser() { "Convert the key of every record in the stream to another key", operationNameField(), functionField(KSMLDSL.Operations.Transform.MAPPER, "A function that computes a new key for each record", new KeyTransformerDefinitionParser(false)), - storeNamesField(), - (name, mapper, storeNames, tags) -> new TransformKeyOperation(operationConfig(name, tags, storeNames), mapper)); + (name, mapper, tags) -> new TransformKeyOperation(operationConfig(name, tags), mapper)); } } diff --git a/ksml/src/main/java/io/axual/ksml/operation/parser/TransformKeyValueOperationParser.java b/ksml/src/main/java/io/axual/ksml/operation/parser/TransformKeyValueOperationParser.java index ca23868f..f784ecec 100644 --- a/ksml/src/main/java/io/axual/ksml/operation/parser/TransformKeyValueOperationParser.java +++ b/ksml/src/main/java/io/axual/ksml/operation/parser/TransformKeyValueOperationParser.java @@ -40,7 +40,6 @@ protected StructsParser parser() { "Convert the key/value of every record in the stream to another key/value", operationNameField(), functionField(KSMLDSL.Operations.Transform.MAPPER, "A function that computes a new key/value for each record", new KeyValueTransformerDefinitionParser(false)), - storeNamesField(), - (name, mapper, storeNames, tags) -> new TransformKeyValueOperation(operationConfig(name, tags, storeNames), mapper)); + (name, mapper, tags) -> new TransformKeyValueOperation(operationConfig(name, tags), mapper)); } } diff --git a/ksml/src/main/java/io/axual/ksml/operation/parser/TransformKeyValueToKeyValueListOperationParser.java b/ksml/src/main/java/io/axual/ksml/operation/parser/TransformKeyValueToKeyValueListOperationParser.java index 63a36204..89bd0267 100644 --- a/ksml/src/main/java/io/axual/ksml/operation/parser/TransformKeyValueToKeyValueListOperationParser.java +++ b/ksml/src/main/java/io/axual/ksml/operation/parser/TransformKeyValueToKeyValueListOperationParser.java @@ -40,7 +40,6 @@ protected StructsParser parser() { "Convert a stream by transforming every record into a list of derived records", operationNameField(), functionField(KSMLDSL.Operations.Transform.MAPPER, "A function that converts every record of a stream to a list of output records.", new KeyValueToKeyValueListTransformerDefinitionParser(false)), - storeNamesField(), - (name, mapper, storeNames, tags) -> new TransformKeyValueToKeyValueListOperation(operationConfig(name, tags, storeNames), mapper)); + (name, mapper, tags) -> new TransformKeyValueToKeyValueListOperation(operationConfig(name, tags), mapper)); } } diff --git a/ksml/src/main/java/io/axual/ksml/operation/parser/TransformKeyValueToValueListOperationParser.java b/ksml/src/main/java/io/axual/ksml/operation/parser/TransformKeyValueToValueListOperationParser.java index e5590ca0..4201652c 100644 --- a/ksml/src/main/java/io/axual/ksml/operation/parser/TransformKeyValueToValueListOperationParser.java +++ b/ksml/src/main/java/io/axual/ksml/operation/parser/TransformKeyValueToValueListOperationParser.java @@ -40,7 +40,6 @@ protected StructsParser parser() { "Convert every record in the stream to a list of output records with the same key", operationNameField(), functionField(KSMLDSL.Operations.Transform.MAPPER, "A function that converts every key/value into a list of result values, each of which will be combined with the original key to form a new message in the output stream", new KeyValueToValueListTransformerDefinitionParser(false)), - storeNamesField(), - (name, mapper, storeNames, tags) -> new TransformKeyValueToValueListOperation(operationConfig(name, tags, storeNames), mapper)); + (name, mapper, tags) -> new TransformKeyValueToValueListOperation(operationConfig(name, tags), mapper)); } } diff --git a/ksml/src/main/java/io/axual/ksml/operation/parser/TransformMetadataOperationParser.java b/ksml/src/main/java/io/axual/ksml/operation/parser/TransformMetadataOperationParser.java index f3a63163..492513b3 100644 --- a/ksml/src/main/java/io/axual/ksml/operation/parser/TransformMetadataOperationParser.java +++ b/ksml/src/main/java/io/axual/ksml/operation/parser/TransformMetadataOperationParser.java @@ -39,7 +39,6 @@ protected StructsParser parser() { "Convert the metadata of every record in the stream", operationNameField(), functionField(KSMLDSL.Operations.Transform.MAPPER, "A function that converts the metadata (Kafka headers, timestamp) of every record in the stream", new MetadataTransformerDefinitionParser(false)), - storeNamesField(), - (name, mapper, storeNames, tags) -> new TransformMetadataOperation(operationConfig(name, tags, storeNames), mapper)); + (name, mapper, tags) -> new TransformMetadataOperation(operationConfig(name, tags), mapper)); } } diff --git a/ksml/src/main/java/io/axual/ksml/operation/parser/TransformValueOperationParser.java b/ksml/src/main/java/io/axual/ksml/operation/parser/TransformValueOperationParser.java index a2efdd6c..eaaac2b1 100644 --- a/ksml/src/main/java/io/axual/ksml/operation/parser/TransformValueOperationParser.java +++ b/ksml/src/main/java/io/axual/ksml/operation/parser/TransformValueOperationParser.java @@ -42,7 +42,6 @@ protected StructsParser parser() { operationNameField(), functionField(KSMLDSL.Operations.Transform.MAPPER, "A function that converts the value of every record into another value", new ValueTransformerDefinitionParser(false)), storeField(false, "Materialized view of the transformed table (only applies to tables, ignored for streams)", StoreType.KEYVALUE_STORE), - storeNamesField(), - (name, mapper, store, storeNames, tags) -> new TransformValueOperation(storeOperationConfig(name, tags, store, storeNames), mapper)); + (name, mapper, store, tags) -> new TransformValueOperation(storeOperationConfig(name, tags, store), mapper)); } } diff --git a/ksml/src/main/java/io/axual/ksml/operation/parser/WindowByTimeOperationParser.java b/ksml/src/main/java/io/axual/ksml/operation/parser/WindowByTimeOperationParser.java index b5bebf3b..9a855317 100644 --- a/ksml/src/main/java/io/axual/ksml/operation/parser/WindowByTimeOperationParser.java +++ b/ksml/src/main/java/io/axual/ksml/operation/parser/WindowByTimeOperationParser.java @@ -25,52 +25,83 @@ import io.axual.ksml.exception.TopologyException; import io.axual.ksml.generator.TopologyResources; import io.axual.ksml.operation.WindowByTimeOperation; +import io.axual.ksml.parser.ChoiceParser; import io.axual.ksml.parser.StructsParser; import org.apache.kafka.streams.kstream.SlidingWindows; import org.apache.kafka.streams.kstream.TimeWindows; +import java.util.Map; + public class WindowByTimeOperationParser extends OperationParser { + private static final String DOC = "Operation to window records based on time criteria"; + private final StructsParser tumblingParser; + private final StructsParser hoppingParser; + private final StructsParser slidingParser; + public WindowByTimeOperationParser(TopologyResources resources) { super(KSMLDSL.Operations.WINDOW_BY_TIME, resources); - } - @Override - public StructsParser parser() { - return structsParser( + final var durationParser = durationField(KSMLDSL.TimeWindows.DURATION, "The duration of time windows"); + final var advanceByParser = durationField(KSMLDSL.TimeWindows.ADVANCE_BY, "The amount of time to increase time windows by"); + final var graceParser = durationField(KSMLDSL.TimeWindows.GRACE, "The grace period, during which out-of-order records can still be processed"); + final var timeDifferenceParser = durationField(KSMLDSL.TimeWindows.TIME_DIFFERENCE, "The maximum amount of time difference between two records"); + + tumblingParser = structsParser( + WindowByTimeOperation.class, + "WithTumblingWindow", + DOC, + operationNameField(), + durationParser, + optional(graceParser), + (name, duration, grace, tags) -> { + final var timeWindows = (grace != null && grace.toMillis() > 0) + ? TimeWindows.ofSizeAndGrace(duration, grace) + : TimeWindows.ofSizeWithNoGrace(duration); + return new WindowByTimeOperation(operationConfig(name, tags), timeWindows); + }); + + hoppingParser = structsParser( WindowByTimeOperation.class, - "", - "Operation to reduce a series of records into a single aggregate result", + "WithHoppingWindow", + DOC, operationNameField(), - stringField(KSMLDSL.TimeWindows.WINDOW_TYPE, "The type of the operation, either \"" + KSMLDSL.TimeWindows.TYPE_TUMBLING + "\", or \"" + KSMLDSL.TimeWindows.TYPE_HOPPING + "\", or \"" + KSMLDSL.TimeWindows.TYPE_SLIDING + "\""), - optional(durationField(KSMLDSL.TimeWindows.DURATION, "(Tumbling) The duration of time windows")), - optional(durationField(KSMLDSL.TimeWindows.ADVANCE_BY, "(Hopping) The amount of time to increase time windows by")), - optional(durationField(KSMLDSL.TimeWindows.GRACE, "(Tumbling, Hopping, Sliding) The grace period, during which out-of-order records can still be processed")), - optional(durationField(KSMLDSL.TimeWindows.TIME_DIFFERENCE, "(Sliding) The maximum amount of time difference between two records")), - (name, windowType, duration, advanceBy, grace, timeDifference, tags) -> { - switch (windowType) { - case KSMLDSL.TimeWindows.TYPE_TUMBLING -> { - final var timeWindows = (grace != null && grace.toMillis() > 0) - ? TimeWindows.ofSizeAndGrace(duration, grace) - : TimeWindows.ofSizeWithNoGrace(duration); - return new WindowByTimeOperation(operationConfig(name, tags), timeWindows); - } - case KSMLDSL.TimeWindows.TYPE_HOPPING -> { - if (advanceBy.toMillis() > duration.toMillis()) { - throw new TopologyException("A hopping window can not advanceBy more than its duration"); - } - final var timeWindows = (grace != null && grace.toMillis() > 0) - ? org.apache.kafka.streams.kstream.TimeWindows.ofSizeAndGrace(duration, grace).advanceBy(advanceBy) - : org.apache.kafka.streams.kstream.TimeWindows.ofSizeWithNoGrace(duration).advanceBy(advanceBy); - return new WindowByTimeOperation(operationConfig(name, tags), timeWindows); - } - case KSMLDSL.TimeWindows.TYPE_SLIDING -> { - final var slidingWindows = (grace != null && grace.toMillis() > 0) - ? SlidingWindows.ofTimeDifferenceAndGrace(timeDifference, grace) - : SlidingWindows.ofTimeDifferenceWithNoGrace(timeDifference); - return new WindowByTimeOperation(operationConfig(name, tags), slidingWindows); - } + durationParser, + advanceByParser, + optional(graceParser), + (name, duration, advanceBy, grace, tags) -> { + if (advanceBy.toMillis() > duration.toMillis()) { + throw new TopologyException("A hopping window can not advanceBy more than its duration"); } - throw new TopologyException("Unknown WindowType for windowByTime operation: " + windowType); + final var timeWindows = (grace != null && grace.toMillis() > 0) + ? org.apache.kafka.streams.kstream.TimeWindows.ofSizeAndGrace(duration, grace).advanceBy(advanceBy) + : org.apache.kafka.streams.kstream.TimeWindows.ofSizeWithNoGrace(duration).advanceBy(advanceBy); + return new WindowByTimeOperation(operationConfig(name, tags), timeWindows); + }); + + slidingParser = structsParser( + WindowByTimeOperation.class, + "WithSlidingWindow", + DOC, + operationNameField(), + timeDifferenceParser, + optional(graceParser), + (name, timeDifference, grace, tags) -> { + final var slidingWindows = (grace != null && grace.toMillis() > 0) + ? SlidingWindows.ofTimeDifferenceAndGrace(timeDifference, grace) + : SlidingWindows.ofTimeDifferenceWithNoGrace(timeDifference); + return new WindowByTimeOperation(operationConfig(name, tags), slidingWindows); }); } + + @Override + public StructsParser parser() { + return new ChoiceParser<>( + KSMLDSL.TimeWindows.WINDOW_TYPE, + "WindowType", + "time window", + null, + Map.of(KSMLDSL.TimeWindows.TYPE_TUMBLING, tumblingParser, + KSMLDSL.TimeWindows.TYPE_HOPPING, hoppingParser, + KSMLDSL.TimeWindows.TYPE_SLIDING, slidingParser)); + } } diff --git a/ksml/src/main/java/io/axual/ksml/parser/ChoiceParser.java b/ksml/src/main/java/io/axual/ksml/parser/ChoiceParser.java index b8fd990c..4c5caf00 100644 --- a/ksml/src/main/java/io/axual/ksml/parser/ChoiceParser.java +++ b/ksml/src/main/java/io/axual/ksml/parser/ChoiceParser.java @@ -45,7 +45,6 @@ public ChoiceParser(String childName, String enumType, String description, Strin this.childName = childName; this.parsedType = description; this.defaultValue = defaultValue; - this.parsers = new HashMap<>(parsers); // To generate proper JSON Schema, first map all parseable schema to fixed values associated with them diff --git a/ksml/src/main/java/io/axual/ksml/parser/DefinitionParser.java b/ksml/src/main/java/io/axual/ksml/parser/DefinitionParser.java index 154b82b5..2f7b47b9 100644 --- a/ksml/src/main/java/io/axual/ksml/parser/DefinitionParser.java +++ b/ksml/src/main/java/io/axual/ksml/parser/DefinitionParser.java @@ -30,10 +30,7 @@ import lombok.Getter; import java.time.Duration; -import java.util.ArrayList; -import java.util.HashMap; -import java.util.List; -import java.util.Map; +import java.util.*; public abstract class DefinitionParser extends BaseParser implements StructsParser { public static final String SCHEMA_NAMESPACE = "io.axual.ksml"; @@ -152,6 +149,21 @@ protected StructsParser durationField(String childName, String doc) { return freeField(childName, null, doc, durationParser); } + protected StructsParser enumField(String childName, EnumSchema schema) { + final var stringParser = new StringValueParser(); + final ParserWithSchema parser = ParserWithSchema.of( + node -> { + final var value = stringParser.parse(node); + if (value == null) + throw new ParseException(node, "Empty value not allowed for enum " + schema.name()); + if (!Set.copyOf(schema.symbols()).contains(value)) + throw new ParseException(node, "Illegal value for enum " + schema.name() + ": " + value); + return value; + }, + schema); + return new FieldParser<>(childName, false, null, schema.doc(), parser); + } + protected StructsParser integerField(String childName, String doc) { return integerField(childName, null, doc); } diff --git a/ksml/src/main/java/io/axual/ksml/python/Invoker.java b/ksml/src/main/java/io/axual/ksml/python/Invoker.java index 82c73374..6033641b 100644 --- a/ksml/src/main/java/io/axual/ksml/python/Invoker.java +++ b/ksml/src/main/java/io/axual/ksml/python/Invoker.java @@ -27,8 +27,8 @@ import io.axual.ksml.data.tag.ContextTags; import io.axual.ksml.data.type.DataType; import io.axual.ksml.exception.TopologyException; -import io.axual.ksml.metric.KSMLMetrics; import io.axual.ksml.metric.MetricName; +import io.axual.ksml.metric.Metrics; import io.axual.ksml.user.UserFunction; import org.slf4j.Logger; import org.slf4j.LoggerFactory; @@ -46,10 +46,10 @@ protected Invoker(UserFunction function, ContextTags metricTags, String function } final var metricName = new MetricName("execution-time", metricTags.append("function-type", functionType).append("function-name", function.name)); - if (KSMLMetrics.registry().getTimer(metricName) == null) { - timer = KSMLMetrics.registry().registerTimer(metricName); + if (Metrics.registry().getTimer(metricName) == null) { + timer = Metrics.registry().registerTimer(metricName); } else { - timer = KSMLMetrics.registry().getTimer(metricName); + timer = Metrics.registry().getTimer(metricName); } this.function = function; } diff --git a/ksml/src/main/java/io/axual/ksml/python/PythonContext.java b/ksml/src/main/java/io/axual/ksml/python/PythonContext.java index 8b217221..b0c9befe 100644 --- a/ksml/src/main/java/io/axual/ksml/python/PythonContext.java +++ b/ksml/src/main/java/io/axual/ksml/python/PythonContext.java @@ -22,7 +22,7 @@ import io.axual.ksml.data.exception.ExecutionException; import io.axual.ksml.data.mapper.DataObjectConverter; -import io.axual.ksml.metric.KSMLMetrics; +import io.axual.ksml.metric.Metrics; import lombok.Getter; import lombok.extern.slf4j.Slf4j; import org.graalvm.polyglot.*; @@ -33,7 +33,7 @@ @Slf4j public class PythonContext { private static final LoggerBridge LOGGER_BRIDGE = new LoggerBridge(); - private static final MetricsBridge METRICS_BRIDGE = new MetricsBridge(KSMLMetrics.registry()); + private static final MetricsBridge METRICS_BRIDGE = new MetricsBridge(Metrics.registry()); private static final String PYTHON = "python"; private static final List ALLOWED_JAVA_CLASSES = List.of( "java.util.ArrayList", @@ -114,7 +114,7 @@ def register_ksml_bridges(lb, mb): if (register == null) { throw new ExecutionException("Could not register global code for loggerBridge:\n" + pyCode); } - // Load the global LOGGER_BRIDGE variable into the context + // Pass the global LOGGER_BRIDGE and METRICS_BRIDGE variables into global variables of the Python context register.execute(LOGGER_BRIDGE, METRICS_BRIDGE); } } diff --git a/ksml/src/test/java/io/axual/ksml/python/MetricsBridgeTest.java b/ksml/src/test/java/io/axual/ksml/python/MetricsBridgeTest.java index 0d280497..37121a9a 100644 --- a/ksml/src/test/java/io/axual/ksml/python/MetricsBridgeTest.java +++ b/ksml/src/test/java/io/axual/ksml/python/MetricsBridgeTest.java @@ -20,7 +20,7 @@ * =========================LICENSE_END================================== */ -import io.axual.ksml.metric.KSMLMetrics; +import io.axual.ksml.metric.Metrics; import io.axual.ksml.metric.MetricsRegistry; import org.junit.jupiter.api.BeforeEach; import org.junit.jupiter.api.Test; @@ -39,7 +39,7 @@ class MetricsBridgeTest { @BeforeEach void setUp() { - metricsBridge = new MetricsBridge(KSMLMetrics.registry()); + metricsBridge = new MetricsBridge(Metrics.registry()); } @Test