diff --git a/CHANGELOG.md b/CHANGELOG.md index b54021e5f77b..101d45b676ba 100644 --- a/CHANGELOG.md +++ b/CHANGELOG.md @@ -1,5 +1,50 @@ # Change Log +## [0.25.1](https://github.com/confluentinc/ksql/releases/tag/v0.25.1-ksqldb) (2022-04-07) + +### Features + +* add java client support for push query v2 ALOS through continue method ([#8785](https://github.com/confluentinc/ksql/pull/8785)) ([11ede7c](https://github.com/confluentinc/ksql/commit/11ede7c1fb9169496e9db42459105b98084bdce7)) +* add cleanup service metrics ([#8779](https://github.com/confluentinc/ksql/pull/8779)) ([1cb7846](https://github.com/confluentinc/ksql/commit/1cb7846715d5055e3aad936bcac90b37d51c9b38)) +* add rate-limiting to ksql command topic ([#8809](https://github.com/confluentinc/ksql/pull/8809)) ([b2f1540](https://github.com/confluentinc/ksql/commit/b2f15400a179126a73f9ae4b7a5a0abf4db8735d)) +* allow users to specify custom migrations dir location ([#8844](https://github.com/confluentinc/ksql/pull/8844)) ([cbe447e](https://github.com/confluentinc/ksql/commit/cbe447e5506f93c2c425794c9be322be8ea7a6a9)) +* cleanup transient query resources ([#8694](https://github.com/confluentinc/ksql/pull/8694)) ([24b2a7a](https://github.com/confluentinc/ksql/commit/24b2a7a2297bddd94c86c2a4e7b5a6a92298825e)) +* create command topic with command topic configs ([#8742](https://github.com/confluentinc/ksql/pull/8742)) ([ea5d6d7](https://github.com/confluentinc/ksql/commit/ea5d6d74ac159ef1c17533563f279dd0060fd494)) +* Extend Udaf interface to allow for polymorphic UDAFs. ([#8871](https://github.com/confluentinc/ksql/pull/8871)) ([2dae2a1](https://github.com/confluentinc/ksql/commit/2dae2a1a8f5d5277a829f29a001478e50278a481)) +* Generalize the UDAFs collect_list and collect_set ([#8877](https://github.com/confluentinc/ksql/pull/8877)) ([1416ecd](https://github.com/confluentinc/ksql/commit/1416ecdd9a1077cd79a95225764facaa9d895af3)) +* Generalize the UDAFs earliest_by_offset and latest_by_offset ([#8878](https://github.com/confluentinc/ksql/pull/8878)) ([adac458](https://github.com/confluentinc/ksql/commit/adac45855dce1c413073e5cbeb474cb022013a2b)) +* include checking the the config during validation of SET ([#8718](https://github.com/confluentinc/ksql/pull/8718)) ([d8ff588](https://github.com/confluentinc/ksql/commit/d8ff588d6b8afcc7059f08a7c6eb0002fc6ecd43)) +* support custom request headers from java client and migrations tool ([#8787](https://github.com/confluentinc/ksql/pull/8787)) ([ffe57f5](https://github.com/confluentinc/ksql/commit/ffe57f5ba13efcec735112382a6a5056f118fc2d)) + + + +### Bug Fixes + +* preserve old schema behavior for protobuf wrapped primitives ([#8934](https://github.com/confluentinc/ksql/pull/8934)) ([36485e2](https://github.com/confluentinc/ksql/commit/36485e263d8988e73c8150aaa3c59252fcdedc69)) +* Add null handling to functions ([#8726](https://github.com/confluentinc/ksql/pull/8726)) ([6117604](https://github.com/confluentinc/ksql/commit/6117604fa424dd051f882458a0c95abcdcd8e170)) +* Adds error handling for nested functions ([#8850](https://github.com/confluentinc/ksql/pull/8850)) ([2a60269](https://github.com/confluentinc/ksql/commit/2a60269cadddc8a11522d103d800ae2a6769d690)) +* Apply the ExtensionSecurityManager to UDAFs ([#8776](https://github.com/confluentinc/ksql/pull/8776)) ([a37688c](https://github.com/confluentinc/ksql/commit/a37688cdd33428c1f5c4e67bbc776835f72b0403)) +* bug preventing decimals in (-1, 1) from being inserted / queried ([#8720](https://github.com/confluentinc/ksql/pull/8720)) ([de8284a](https://github.com/confluentinc/ksql/commit/de8284a127de323567276eb6daa2f963a30a6352)) +* CLI should return non-zero error code on failure ([#8892](https://github.com/confluentinc/ksql/pull/8892)) ([238f4fd](https://github.com/confluentinc/ksql/commit/238f4fdd929541ccdd4aeee45cd331964f73b9b3)) +* coerce property values to correct type ([#8765](https://github.com/confluentinc/ksql/pull/8765)) ([7f7a076](https://github.com/confluentinc/ksql/commit/7f7a07619a6ed28f022980ad078607502f4a1f8d)) +* do not include schema id in session config ([#8869](https://github.com/confluentinc/ksql/pull/8869)) ([5ddb852](https://github.com/confluentinc/ksql/commit/5ddb852afc37939baa72d9cf6e55dfde9d1e6400)) +* Ensures response end handler is invoked just once ([#8849](https://github.com/confluentinc/ksql/pull/8849)) ([a2efcc5](https://github.com/confluentinc/ksql/commit/a2efcc53f05de45daf0724ff7c4e1467f2548224)) +* Fix bugs in sample standard deviation UDAF ([#8728](https://github.com/confluentinc/ksql/pull/8728)) ([b2f993b](https://github.com/confluentinc/ksql/commit/b2f993b860a4c8075d23f8a77a828eae12af499f)) +* Gives a query completed message for stream pull queries ([#8612](https://github.com/confluentinc/ksql/pull/8612)) ([a44a16b](https://github.com/confluentinc/ksql/commit/a44a16be46d3c61e7f9d8724c4900752e9fc5337)) +* **ksql:** add ifExists/ifNotExist parameters to java client connector functions ([#8851](https://github.com/confluentinc/ksql/pull/8851)) ([eaf2b1f](https://github.com/confluentinc/ksql/commit/eaf2b1f047a126de642f06a0cde30e6c5123762a)) +* **ksql:** allow migrations tool to run connector commands with IF [NOT] EXISTS clauses ([#8855](https://github.com/confluentinc/ksql/pull/8855)) ([a7c8689](https://github.com/confluentinc/ksql/commit/a7c8689a498bba812a9fc6dbc66c8a3b9d814176)) +* make writes to the backup file atomic ([#8566](https://github.com/confluentinc/ksql/pull/8566)) ([b113e9e](https://github.com/confluentinc/ksql/commit/b113e9ed4a81670424019dcff567aa01697ddca9)) +* pass DCN_NULLPOINTER_EXCEPTION spotbugs error ([#8775](https://github.com/confluentinc/ksql/pull/8775)) ([afd7fb1](https://github.com/confluentinc/ksql/commit/afd7fb1c72dc9ed52be3c991213277a401408bca)) +* prevent hanging stream pull queries on truncated topics ([#8740](https://github.com/confluentinc/ksql/pull/8740)) ([d66107c](https://github.com/confluentinc/ksql/commit/d66107ced9809b852e6c75935aa623499a793164)) +* re-order cache usage error ([#8909](https://github.com/confluentinc/ksql/pull/8909)) ([336c690](https://github.com/confluentinc/ksql/commit/336c690f81932fe263e224344a671fe71b85a0ab)) +* register schema within sandbox ([#8614](https://github.com/confluentinc/ksql/pull/8614)) ([ba572e0](https://github.com/confluentinc/ksql/commit/ba572e0e5bd8873e538d40a58153ba345eb1573d)), closes [#1394](https://github.com/confluentinc/ksql/issues/1394) +* reinstate the old KsqlRestClient.create overload ([#8761](https://github.com/confluentinc/ksql/pull/8761)) ([ee4a1bc](https://github.com/confluentinc/ksql/commit/ee4a1bcf4ae4487e01bd4a6341feac402ffe6ddc)) +* resolve schema registry issue for pull query ([#8876](https://github.com/confluentinc/ksql/pull/8876)) ([6a1c2ae](https://github.com/confluentinc/ksql/commit/6a1c2ae36d1257b050798b2a1f3cdbec4b68470d)) +* restore process fails due to DROP constraints ([#8803](https://github.com/confluentinc/ksql/pull/8803)) ([db070a2](https://github.com/confluentinc/ksql/commit/db070a211b51cfb9ef7138c9b05238ba41798f4b)) +* Set the sslFactory properly for the SR REST client. ([#8830](https://github.com/confluentinc/ksql/pull/8830)) ([e69a545](https://github.com/confluentinc/ksql/commit/e69a545a7765e9056da223075d4336d482b36614)) +* update rate limiting test so it's not flaky ([#8872](https://github.com/confluentinc/ksql/pull/8872)) ([466f8fe](https://github.com/confluentinc/ksql/commit/466f8fe92b5ec18209e37c511a91b633d07ca96e)) +* update restore command topic tool to work with command topic configs ([#8802](https://github.com/confluentinc/ksql/pull/8802)) ([371b200](https://github.com/confluentinc/ksql/commit/371b200d92b9fd6083bb453af74a31d4b754ecf2)) + ## [0.24.0](https://github.com/confluentinc/ksql/releases/tag/v0.24.0) (2022-02-11) ### Features diff --git a/Jenkinsfile b/Jenkinsfile index 09870c2b1f28..e191d765cdfe 100755 --- a/Jenkinsfile +++ b/Jenkinsfile @@ -8,7 +8,7 @@ common { dockerPush = false dockerScan = false dockerImageClean = false - downStreamRepos = ["confluent-security-plugins", "confluent-cloud-plugins"] + downStreamRepos = ["confluent-security-plugins", "confluent-cloud-plugins", "cc-docker-ksql"] downStreamValidate = false nanoVersion = true maxBuildsToKeep = 99 diff --git a/design-proposals/README.md b/design-proposals/README.md index 5652761c692d..061746d3b8e2 100644 --- a/design-proposals/README.md +++ b/design-proposals/README.md @@ -37,7 +37,7 @@ This is the guts of our improvement proposal process: The progression of statuses should be: Proposal, Discussion, Approved, Merged -Next KLIP number: **63** +Next KLIP number: **64** | KLIP | Status | Community Release | CP Release | Discussion PR | |----------------------------------------------------------------------------------------------------------------------------------|:--------:|:-----------------:|:----------:|--------------------------------------------------------------| @@ -103,3 +103,4 @@ Next KLIP number: **63** | [KLIP-60: Support input topics with multiple message types](klip-60-multiple-message-types.md) | Proposal | | | | | [KLIP-61: Bootstrapped TABLE subscriptions](klip-61-bootstrapped-subscriptions.md) | Proposal | | | | | [KLIP-62: ksqlDB Ruby Client](klip-62-ksqldb-ruby-client.md) | Proposal | | | [Discussion](https://github.com/confluentinc/ksql/pull/8865) | +| [KLIP-63: PAUSE and RESUME for persistent queries](klip-63-pause-resume-persistent-queries.md) | Proposal | | | | diff --git a/docs/operate-and-deploy/changelog.md b/docs/operate-and-deploy/changelog.md index 300fe3f65fa9..c3636597b2a5 100644 --- a/docs/operate-and-deploy/changelog.md +++ b/docs/operate-and-deploy/changelog.md @@ -6,6 +6,12 @@ description: Lists changes to the ksqlDB codebase keywords: ksqldb, changelog --- +Version 0.25.1 +-------------- + +- [Announcing ksqlDB 0.25.1](https://www.confluent.io/blog/announcing-ksqldb-0-25-1/) +- [ksqlDB v0.25.1 changelog](https://github.com/confluentinc/ksql/blob/master/CHANGELOG.md#0251-2022-04-07) + Version 0.24.0 -------------- diff --git a/docs/operate-and-deploy/exactly-once-semantics.md b/docs/operate-and-deploy/exactly-once-semantics.md index bd83f2cc417a..18f17fd0d146 100644 --- a/docs/operate-and-deploy/exactly-once-semantics.md +++ b/docs/operate-and-deploy/exactly-once-semantics.md @@ -28,12 +28,14 @@ operation exactly one time. All of the processing happens exactly once, including the processing and the materialized state created by the processing job that is written back to {{ site.ak }}. -To enable exactly-once semantics, set `processing.guarantee="exactly_once"` in -your ksqlDB configuration. +To enable exactly-once semantics, set `processing.guarantee="exactly_once_v2"` in +your ksqlDB configuration. Your {{ site.ak }} broker version must be 2.5 or later. +If you're using the {{ site.cp }} distribution of ksqlDB, you need {{ site.cp }} +version 5.5 or later. !!! important - Use the `exactly_once` setting with care. To achieve a true exactly-once + Use the `exactly_once_v2` setting with care. To achieve a true exactly-once system, end consumers and producers must also implement exactly-once semantics. @@ -73,7 +75,7 @@ Content-Type: application/vnd.ksql.v1+json { "ksql": "SELECT * FROM pageviews EMIT CHANGES;", "streamsProperties": { - "processing.guarantee": "exactly_once" + "processing.guarantee": "exactly_once_v2" } } ``` diff --git a/docs/reference/sql/time.md b/docs/reference/sql/time.md index 45ec6b9d231f..62ccb6012e64 100644 --- a/docs/reference/sql/time.md +++ b/docs/reference/sql/time.md @@ -11,11 +11,11 @@ keywords: time, datetime, timestamp, format, window The following list shows valid time units for the `SIZE`, `ADVANCE BY`, `SESSION`, and `WITHIN` clauses, or to pass as time unit parameters in functions. -- `DAY`, `DAYS` -- `HOUR`, `HOURS` -- `MINUTE`, `MINUTES` -- `SECOND`, `SECONDS` -- `MILLISECOND`, `MILLISECONDS` +- `DAYS` +- `HOURS` +- `MINUTES` +- `SECONDS` +- `MILLISECONDS` For more information, see [Windows in SQL Queries](/concepts/time-and-windows-in-ksqldb-queries#windows-in-sql-queries). diff --git a/ksqldb-api-client/src/main/java/io/confluent/ksql/api/client/Client.java b/ksqldb-api-client/src/main/java/io/confluent/ksql/api/client/Client.java index c95af8c3d1fb..bd13573db4c5 100644 --- a/ksqldb-api-client/src/main/java/io/confluent/ksql/api/client/Client.java +++ b/ksqldb-api-client/src/main/java/io/confluent/ksql/api/client/Client.java @@ -19,6 +19,7 @@ import io.confluent.ksql.api.client.impl.ClientImpl; import io.vertx.core.Vertx; import java.io.Closeable; +import java.time.Duration; import java.util.List; import java.util.Map; import java.util.concurrent.CompletableFuture; @@ -311,6 +312,146 @@ CompletableFuture createConnector( */ CompletableFuture describeConnector(String connectorName); + /** + * Asserts the existence of a schema with the given subject name. If the assertion fails, + * then the {@code CompletableFuture} will complete exceptionally + * + *

If a non-200 response is received from the server, the {@code CompletableFuture} will be + * failed. + * + * @param subject subject name + * @param exists true to check that a schema exists, false to check that a schema does not exist + * @return a future that completes once the server response is received + */ + CompletableFuture assertSchema(String subject, boolean exists); + + /** + * Asserts the existence of a schema with the given id. If the assertion fails, + * then the {@code CompletableFuture} will complete exceptionally + * + *

If a non-200 response is received from the server, the {@code CompletableFuture} will be + * failed. + * + * @param id schema id + * @param exists true to check that a schema exists, false to check that a schema does not exist + * @return a future that completes once the server response is received + */ + CompletableFuture assertSchema(int id, boolean exists); + + /** + * Asserts the existence of a schema with the given subject name and id. If the assertion fails, + * then the {@code CompletableFuture} will complete exceptionally + * + *

If a non-200 response is received from the server, the {@code CompletableFuture} will be + * failed. + * + * @param subject subject name + * @param id schema id + * @param exists true to check that a schema exists, false to check that a schema does not exist + * @return a future that completes once the server response is received + */ + CompletableFuture assertSchema(String subject, int id, boolean exists); + + /** + * Asserts the existence of a schema with the given subject name within the specified timeout. + * If the assertion fails, then the {@code CompletableFuture} will complete exceptionally + * + *

If a non-200 response is received from the server, the {@code CompletableFuture} will be + * failed. + * + * @param subject subject name + * @param exists true to check that a schema exists, false to check that a schema does not exist + * @param timeout the amount of time to wait for a successful assertion + * @return a future that completes once the server response is received + */ + CompletableFuture assertSchema(String subject, boolean exists, Duration timeout); + + /** + * Asserts the existence of a schema with the given id within the specified timeout. + * If the assertion fails, then the {@code CompletableFuture} will complete exceptionally + * + *

If a non-200 response is received from the server, the {@code CompletableFuture} will be + * failed. + * + * @param id schema id + * @param exists true to check that a schema exists, false to check that a schema does not exist + * @param timeout the amount of time to wait for a successful assertion + * @return a future that completes once the server response is received + */ + CompletableFuture assertSchema(int id, boolean exists, Duration timeout); + + /** + * Asserts the existence of a schema with the given subject name and id within the specified + * timeout. If the assertion fails, then the {@code CompletableFuture} will complete exceptionally + * + *

If a non-200 response is received from the server, the {@code CompletableFuture} will be + * failed. + * + * @param subject subject name + * @param id schema id + * @param exists true to check that a schema exists, false to check that a schema does not exist + * @param timeout the amount of time to wait for a successful assertion + * @return a future that completes once the server response is received + */ + CompletableFuture assertSchema(String subject, int id, boolean exists, Duration timeout); + + /** + * Asserts the existence of a topic. If the assertion fails, then the {@code CompletableFuture} + * will complete exceptionally + * + *

If a non-200 response is received from the server, the {@code CompletableFuture} will be + * failed. + * + * @param topic topic name + * @param exists true to check that a schema exists, false to check that a schema does not exist + * @return a future that completes once the server response is received + */ + CompletableFuture assertTopic(String topic, boolean exists); + + /** + * Asserts the existence of a topic within the specified timeout. If the assertion fails, then + * the {@code CompletableFuture} will complete exceptionally + * + *

If a non-200 response is received from the server, the {@code CompletableFuture} will be + * failed. + * + * @param topic topic name + * @param exists true to check that a schema exists, false to check that a schema does not exist + * @param timeout the amount of time to wait for a successful assertion + * @return a future that completes once the server response is received + */ + CompletableFuture assertTopic(String topic, boolean exists, Duration timeout); + + /** + * Asserts the existence of a topic with the given configurations. If the assertion fails, then + * the {@code CompletableFuture} will complete exceptionally + * + *

If a non-200 response is received from the server, the {@code CompletableFuture} will be + * failed. + * + * @param topic topic name + * @param configs map of topic configurations to check + * @param exists true to check that a schema exists, false to check that a schema does not exist + * @return a future that completes once the server response is received + */ + CompletableFuture assertTopic(String topic, Map configs, boolean exists); + + /** + * Asserts the existence of a topic with the given configurations within the specified + * timeout. If the assertion fails, then the {@code CompletableFuture} will complete exceptionally + * + *

If a non-200 response is received from the server, the {@code CompletableFuture} will be + * failed. + * + * @param topic topic name + * @param configs map of topic configurations to check + * @param exists true to check that a schema exists, false to check that a schema does not exist + * @param timeout the amount of time to wait for a successful assertion + * @return a future that completes once the server response is received + */ + CompletableFuture assertTopic( + String topic, Map configs, boolean exists, Duration timeout); + /** * A factory to construct {@link HttpRequest} objects. Instances of {@link HttpRequest} are * used to make direct HTTP requests to ksqlDB server's REST API. diff --git a/ksqldb-api-client/src/main/java/io/confluent/ksql/api/client/impl/AssertResponseHandler.java b/ksqldb-api-client/src/main/java/io/confluent/ksql/api/client/impl/AssertResponseHandler.java new file mode 100644 index 000000000000..3ead1f22e3a7 --- /dev/null +++ b/ksqldb-api-client/src/main/java/io/confluent/ksql/api/client/impl/AssertResponseHandler.java @@ -0,0 +1,57 @@ +/* + * Copyright 2022 Confluent Inc. + * + * Licensed under the Confluent Community License (the "License"); you may not use + * this file except in compliance with the License. You may obtain a copy of the + * License at + * + * http://www.confluent.io/confluent-community-license + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, WITHOUT + * WARRANTIES OF ANY KIND, either express or implied. See the License for the + * specific language governing permissions and limitations under the License. + */ + +package io.confluent.ksql.api.client.impl; + +import io.vertx.core.json.JsonObject; +import java.util.concurrent.CompletableFuture; + +public final class AssertResponseHandler { + + private AssertResponseHandler() { + + } + + static void handleAssertSchemaResponse( + final JsonObject assertSchemaResponse, + final CompletableFuture cf + ) { + if (assertSchemaResponse.getBoolean("exists") != null + && assertSchemaResponse.containsKey("subject") + && assertSchemaResponse.containsKey("id") + ) { + cf.complete(null); + } else { + cf.completeExceptionally(new IllegalStateException( + "Unexpected server response format. Response: " + assertSchemaResponse + )); + } + } + + static void handleAssertTopicResponse( + final JsonObject assertTopicResponse, + final CompletableFuture cf + ) { + if (assertTopicResponse.getBoolean("exists") != null + && assertTopicResponse.getString("topicName") != null + ) { + cf.complete(null); + } else { + cf.completeExceptionally(new IllegalStateException( + "Unexpected server response format. Response: " + assertTopicResponse + )); + } + } +} diff --git a/ksqldb-api-client/src/main/java/io/confluent/ksql/api/client/impl/ClientImpl.java b/ksqldb-api-client/src/main/java/io/confluent/ksql/api/client/impl/ClientImpl.java index f3833fbe6a1c..90d4cd415b7d 100644 --- a/ksqldb-api-client/src/main/java/io/confluent/ksql/api/client/impl/ClientImpl.java +++ b/ksqldb-api-client/src/main/java/io/confluent/ksql/api/client/impl/ClientImpl.java @@ -16,11 +16,13 @@ package io.confluent.ksql.api.client.impl; import static io.confluent.ksql.api.client.impl.DdlDmlRequestValidators.validateExecuteStatementRequest; +import static io.netty.handler.codec.http.HttpHeaderNames.ACCEPT; import static io.netty.handler.codec.http.HttpHeaderNames.AUTHORIZATION; import static io.netty.handler.codec.http.HttpHeaderNames.USER_AGENT; import static io.netty.handler.codec.http.HttpResponseStatus.OK; import com.google.common.annotations.VisibleForTesting; +import com.google.common.collect.ImmutableMap; import io.confluent.ksql.api.client.AcksPublisher; import io.confluent.ksql.api.client.BatchedQueryResult; import io.confluent.ksql.api.client.Client; @@ -37,6 +39,7 @@ import io.confluent.ksql.api.client.TableInfo; import io.confluent.ksql.api.client.TopicInfo; import io.confluent.ksql.api.client.exception.KsqlClientException; +import io.confluent.ksql.rest.entity.KsqlMediaType; import io.confluent.ksql.util.AppInfo; import io.confluent.ksql.util.KsqlConfig; import io.confluent.ksql.util.KsqlRequestConfig; @@ -58,6 +61,7 @@ import io.vertx.core.net.SocketAddress; import io.vertx.core.parsetools.RecordParser; import java.nio.charset.Charset; +import java.time.Duration; import java.util.Base64; import java.util.Collections; import java.util.HashMap; @@ -479,6 +483,127 @@ public CompletableFuture describeConnector(final String na return cf; } + @Override + public CompletableFuture assertSchema(final String subject, final boolean exists) { + return assertSchema(Optional.of(subject), Optional.empty(), exists, Optional.empty()); + } + + @Override + public CompletableFuture assertSchema(final int id, final boolean exists) { + return assertSchema(Optional.empty(), Optional.of(id), exists, Optional.empty()); + } + + @Override + public CompletableFuture assertSchema( + final String subject, final int id, final boolean exists) { + return assertSchema(Optional.of(subject), Optional.of(id), exists, Optional.empty()); + } + + @Override + public CompletableFuture assertSchema( + final String subject, final boolean exists, final Duration timeout) { + return assertSchema(Optional.of(subject), Optional.empty(), exists, Optional.of(timeout)); + } + + @Override + public CompletableFuture assertSchema( + final int id, final boolean exists, final Duration timeout) { + return assertSchema(Optional.empty(), Optional.of(id), exists, Optional.of(timeout)); + } + + @Override + public CompletableFuture assertSchema( + final String subject, final int id, final boolean exists, final Duration timeout) { + return assertSchema(Optional.of(subject), Optional.of(id), exists, Optional.of(timeout)); + } + + private CompletableFuture assertSchema( + final Optional subject, + final Optional id, + final boolean exists, + final Optional timeout + ) { + final CompletableFuture cf = new CompletableFuture<>(); + final String existClause = exists ? "" : " not exists"; + final String subjectClause = subject.isPresent() ? " subject '" + subject.get() + "'" : ""; + final String idClause = id.isPresent() ? " id " + id.get() : ""; + final String timeoutClause = + timeout.isPresent() ? " timeout " + timeout.get().getSeconds() + " seconds" : ""; + final String command = + "assert" + existClause + " schema" + subjectClause + idClause + timeoutClause + ";"; + makePostRequest( + KSQL_ENDPOINT, + new JsonObject() + .put("ksql", command) + .put("sessionVariables", sessionVariables), + cf, + response -> handleSingleEntityResponse( + response, cf, AssertResponseHandler::handleAssertSchemaResponse) + ); + return cf; + } + + @Override + public CompletableFuture assertTopic(final String topic, final boolean exists) { + return assertTopic(topic, ImmutableMap.of(), exists, Optional.empty()); + } + + @Override + public CompletableFuture assertTopic( + final String topic, final boolean exists, final Duration timeout) { + return assertTopic(topic, ImmutableMap.of(), exists, Optional.of(timeout)); + } + + @Override + public CompletableFuture assertTopic( + final String topic, final Map configs, final boolean exists) { + return assertTopic(topic, configs, exists, Optional.empty()); + } + + @Override + public CompletableFuture assertTopic( + final String topic, + final Map configs, + final boolean exists, + final Duration timeout + ) { + return assertTopic(topic, configs, exists, Optional.of(timeout)); + } + + private CompletableFuture assertTopic( + final String topic, + final Map configs, + final boolean exists, + final Optional timeout + ) { + final CompletableFuture cf = new CompletableFuture<>(); + final String existClause = exists ? "" : " not exists"; + final String configString = configs.size() > 0 ? createConfigString(configs) : ""; + final String timeoutClause = + timeout.isPresent() ? " timeout " + timeout.get().getSeconds() + " seconds" : ""; + final String command = + "assert" + existClause + " topic '" + topic + "'" + configString + timeoutClause + ";"; + makePostRequest( + KSQL_ENDPOINT, + new JsonObject() + .put("ksql", command) + .put("sessionVariables", sessionVariables), + cf, + response -> handleSingleEntityResponse( + response, cf, AssertResponseHandler::handleAssertTopicResponse) + ); + return cf; + } + + private String createConfigString(final Map configs) { + return " with (" + + configs.entrySet() + .stream() + .map((entry) -> entry.getKey() + "=" + entry.getValue()) + .collect(Collectors.joining(",")) + + ")"; + } + @Override public void define(final String variable, final Object value) { sessionVariables.put(variable, value); @@ -613,6 +738,9 @@ private > void makeRequest( if (clientOptions.isUseBasicAuth()) { request = configureBasicAuth(request); } + if (path.equals(QUERY_STREAM_ENDPOINT)) { + request = configureAcceptTypeToLatestMediaType(request); + } if (clientOptions.getRequestHeaders() != null) { for (final Entry entry : clientOptions.getRequestHeaders().entrySet()) { request.putHeader(entry.getKey(), entry.getValue()); @@ -630,6 +758,10 @@ private HttpClientRequest configureBasicAuth(final HttpClientRequest request) { return request.putHeader(AUTHORIZATION.toString(), basicAuthHeader); } + private HttpClientRequest configureAcceptTypeToLatestMediaType(final HttpClientRequest request) { + return request.putHeader(ACCEPT.toString(), KsqlMediaType.LATEST_FORMAT.mediaType()); + } + private HttpClientRequest configureUserAgent(final HttpClientRequest request) { final String clientVersion = AppInfo.getVersion(); return request.putHeader(USER_AGENT.toString(), "ksqlDB Java Client v" + clientVersion); diff --git a/ksqldb-api-client/src/main/java/io/confluent/ksql/api/client/impl/ExecuteQueryResponseHandler.java b/ksqldb-api-client/src/main/java/io/confluent/ksql/api/client/impl/ExecuteQueryResponseHandler.java index c99358761ea5..1c8a5a8c1282 100644 --- a/ksqldb-api-client/src/main/java/io/confluent/ksql/api/client/impl/ExecuteQueryResponseHandler.java +++ b/ksqldb-api-client/src/main/java/io/confluent/ksql/api/client/impl/ExecuteQueryResponseHandler.java @@ -67,32 +67,35 @@ protected void handleMetadata(final QueryResponseMetadata queryResponseMetadata) @Override protected void handleRow(final Buffer buff) { - final Row row; - final Object json = buff.toJson(); + final JsonObject json = buff.toJsonObject(); - if (json instanceof JsonObject) { - final JsonObject jsonObject = (JsonObject) json; - // This is the serialized consistency vector - // Don't add it to the result list since the user should not see it - if (jsonObject.getMap() != null && jsonObject.getMap().containsKey("consistencyToken")) { - log.info("Response contains consistency vector " + jsonObject); - serializedConsistencyVector.set((String) ((JsonObject) json).getMap().get( - "consistencyToken")); + if (!json.containsKey("finalMessage")) { + if (json.containsKey("row")) { + if (rows.size() < maxRows) { + rows.add(new RowImpl( + columnNames, + columnTypes, + new JsonArray((List)((Map) json.getMap().get("row")).get("columns")), + columnNameToIndex)); + } else { + throw new KsqlClientException( + "Reached max number of rows that may be returned by executeQuery(). " + + "Increase the limit via ClientOptions#setExecuteQueryMaxResultRows(). " + + "Current limit: " + maxRows); + } + } else if (json.getMap() != null) { + // This is the serialized consistency vector + // Don't add it to the result list since the user should not see it + if (json.getMap().containsKey("consistencyToken")) { + log.info("Response contains consistency vector " + json); + serializedConsistencyVector.set((String) json.getMap().get( + "consistencyToken")); + } else { + throw new RuntimeException("Could not decode JSON, expected consistency token: " + json); + } } else { - throw new RuntimeException("Could not decode JSON, expected consistency toke: " + json); + throw new RuntimeException("Could not decode JSON: " + json); } - } else if (json instanceof JsonArray) { - final JsonArray values = new JsonArray(buff); - if (rows.size() < maxRows) { - rows.add(new RowImpl(columnNames, columnTypes, values, columnNameToIndex)); - } else { - throw new KsqlClientException( - "Reached max number of rows that may be returned by executeQuery(). " - + "Increase the limit via ClientOptions#setExecuteQueryMaxResultRows(). " - + "Current limit: " + maxRows); - } - } else { - throw new RuntimeException("Could not decode JSON: " + json); } } diff --git a/ksqldb-api-client/src/main/java/io/confluent/ksql/api/client/impl/QueryResponseHandler.java b/ksqldb-api-client/src/main/java/io/confluent/ksql/api/client/impl/QueryResponseHandler.java index c19053a4af30..962064402b76 100644 --- a/ksqldb-api-client/src/main/java/io/confluent/ksql/api/client/impl/QueryResponseHandler.java +++ b/ksqldb-api-client/src/main/java/io/confluent/ksql/api/client/impl/QueryResponseHandler.java @@ -15,12 +15,17 @@ package io.confluent.ksql.api.client.impl; +import static io.confluent.ksql.util.BytesUtils.toJsonMsg; + import com.fasterxml.jackson.databind.ObjectMapper; import io.confluent.ksql.api.client.util.JsonMapper; +import io.confluent.ksql.api.client.util.RowUtil; import io.confluent.ksql.rest.entity.QueryResponseMetadata; import io.vertx.core.Context; import io.vertx.core.buffer.Buffer; import io.vertx.core.parsetools.RecordParser; +import java.util.Map; +import java.util.Objects; import java.util.concurrent.CompletableFuture; abstract class QueryResponseHandler> extends ResponseHandler { @@ -35,10 +40,14 @@ abstract class QueryResponseHandler> extends Resp @Override protected void doHandleBodyBuffer(final Buffer buff) { + final Buffer strippedValidJson = toJsonMsg(buff, true); + if (!hasReadArguments) { - handleArgs(buff); + handleArgs(strippedValidJson); } else { - handleRow(buff); + if (!Objects.equals(strippedValidJson.toString(), "")) { + handleRow(strippedValidJson); + } } } @@ -59,10 +68,16 @@ protected void doHandleException(final Throwable t) { private void handleArgs(final Buffer buff) { hasReadArguments = true; - final QueryResponseMetadata queryResponseMetadata; try { - queryResponseMetadata = JSON_MAPPER.readValue(buff.getBytes(), QueryResponseMetadata.class); + final Object header = buff.toJsonObject().getMap().get("header"); + final String queryId = (String) ((Map) header).get("queryId"); + final String schema = (String) ((Map) header).get("schema"); + queryResponseMetadata = + new QueryResponseMetadata(queryId, + RowUtil.colNamesFromSchema(schema), + RowUtil.colTypesFromSchema(schema), + null); } catch (Exception e) { cf.completeExceptionally(e); return; diff --git a/ksqldb-api-client/src/main/java/io/confluent/ksql/api/client/impl/StreamQueryResponseHandler.java b/ksqldb-api-client/src/main/java/io/confluent/ksql/api/client/impl/StreamQueryResponseHandler.java index be8cd0d1bbe2..9ea7ef818e08 100644 --- a/ksqldb-api-client/src/main/java/io/confluent/ksql/api/client/impl/StreamQueryResponseHandler.java +++ b/ksqldb-api-client/src/main/java/io/confluent/ksql/api/client/impl/StreamQueryResponseHandler.java @@ -25,6 +25,7 @@ import io.vertx.core.json.JsonArray; import io.vertx.core.json.JsonObject; import io.vertx.core.parsetools.RecordParser; +import java.util.List; import java.util.Map; import java.util.Objects; import java.util.concurrent.CompletableFuture; @@ -83,44 +84,43 @@ protected void handleRow(final Buffer buff) { if (queryResult == null) { throw new IllegalStateException("handleRow called before metadata processed"); } - final Object json = buff.toJson(); + final Row row; - if (json instanceof JsonArray) { - row = new RowImpl( - queryResult.columnNames(), - queryResult.columnTypes(), - (JsonArray) json, - columnNameToIndex - ); - final boolean full = queryResult.accept(row); - if (full && !paused) { - recordParser.pause(); - queryResult.drainHandler(this::publisherReceptive); - paused = true; - } - } else if (json instanceof JsonObject) { - final JsonObject jsonObject = (JsonObject) json; - // This is the serialized consistency vector - // Don't add it to the publisher's buffer since the user should not see it - if (jsonObject.getMap() != null && jsonObject.getMap().containsKey("consistencyToken")) { - LOG.info("Response contains consistency vector " + jsonObject); - serializedConsistencyVector.set((String) ((JsonObject) json).getMap().get( - "consistencyToken")); - } - if (jsonObject.getMap() != null && jsonObject.getMap().containsKey("continuationToken")) { - LOG.info("Response contains continuation token " + jsonObject); - continuationToken.set((String) ((JsonObject) json).getMap().get( - "continuationToken")); - } - if (!(jsonObject.getMap() != null && jsonObject.getMap().containsKey("consistencyToken")) - && !(jsonObject.getMap() != null && jsonObject.getMap().containsKey("continuationToken")) - ) { - queryResult.handleError(new KsqlException( - jsonObject.getString("message") - )); + final JsonObject jsonObject = buff.toJsonObject(); + + if (!jsonObject.containsKey("finalMessage")) { + if (jsonObject.containsKey("row")) { + row = new RowImpl( + queryResult.columnNames(), + queryResult.columnTypes(), + new JsonArray((List)((Map) jsonObject.getMap().get("row")).get("columns")), + columnNameToIndex + ); + final boolean full = queryResult.accept(row); + if (full && !paused) { + recordParser.pause(); + queryResult.drainHandler(this::publisherReceptive); + paused = true; + } + } else if (jsonObject.getMap() != null) { + if (jsonObject.getMap().containsKey("consistencyToken")) { + LOG.info("Response contains consistency vector " + jsonObject); + serializedConsistencyVector.set((String) jsonObject.getMap().get("consistencyToken")); + } + if (jsonObject.getMap().containsKey("continuationToken")) { + LOG.info("Response contains continuation token " + jsonObject); + continuationToken.set( + (String) ((Map) jsonObject.getMap() + .get("continuationToken")) + .get("continuationToken")); + } + if (jsonObject.getMap().containsKey("errorMessage")) { + queryResult.handleError(new KsqlException( + (String) ((Map) jsonObject.getMap().get("errorMessage")).get("message"))); + } + } else { + throw new RuntimeException("Could not decode JSON: " + jsonObject); } - } else { - throw new RuntimeException("Could not decode JSON: " + json); } } diff --git a/ksqldb-api-client/src/main/java/io/confluent/ksql/api/client/util/RowUtil.java b/ksqldb-api-client/src/main/java/io/confluent/ksql/api/client/util/RowUtil.java index f42d4f87ce08..5925e0a22a34 100644 --- a/ksqldb-api-client/src/main/java/io/confluent/ksql/api/client/util/RowUtil.java +++ b/ksqldb-api-client/src/main/java/io/confluent/ksql/api/client/util/RowUtil.java @@ -18,6 +18,7 @@ import io.confluent.ksql.api.client.ColumnType; import io.confluent.ksql.api.client.impl.ColumnTypeImpl; import java.util.HashMap; +import java.util.LinkedList; import java.util.List; import java.util.Map; import java.util.stream.Collectors; @@ -50,4 +51,69 @@ private static ColumnType columnTypeFromString(final String columnType) { )); return new ColumnTypeImpl(primaryType); } + + public static List colNamesFromSchema(final String schema) { + return splitAcrossOneLevelDeepComma(schema).stream() + .map(RowUtil::removeBackTickAndKeyTrim) + .map(RowUtil::splitAndGetFirst) + .collect(Collectors.toList()); + } + + public static List colTypesFromSchema(final String schema) { + return splitAcrossOneLevelDeepComma(schema).stream() + .map(RowUtil::removeBackTickAndKeyTrim) + .map(RowUtil::splitAndGetSecond) + .collect(Collectors.toList()); + } + + private static List splitAcrossOneLevelDeepComma(final String stringWithCommas) { + final List listCols = new LinkedList<>(); + final StringBuilder sb = new StringBuilder(); + int nestRound = 0; + int nestChevron = 0; + for (int i = 0; i < stringWithCommas.length(); ++i) { + final char ch = stringWithCommas.charAt(i); + if (i == stringWithCommas.length() - 1) { + sb.append(ch); + listCols.add(sb.toString()); + break; + } + switch (ch) { + case ',': + if (nestRound == 0 && nestChevron == 0) { + listCols.add(sb.toString()); + sb.setLength(0); + continue; + } + break; + case '(': + ++nestRound; + break; + case ')': + --nestRound; + break; + case '<': + ++nestChevron; + break; + case '>': + --nestChevron; + break; + default: + } + sb.append(ch); + } + return listCols; + } + + private static String removeBackTickAndKeyTrim(final String dirtyString) { + return dirtyString.replace(" KEY", "").replace("`", "").trim(); + } + + private static String splitAndGetFirst(final String stringToSplit) { + return stringToSplit.split(" ", 2)[0]; + } + + private static String splitAndGetSecond(final String stringToSplit) { + return stringToSplit.split(" ", 2)[1]; + } } \ No newline at end of file diff --git a/ksqldb-api-client/src/test/java/io/confluent/ksql/api/client/ClientTest.java b/ksqldb-api-client/src/test/java/io/confluent/ksql/api/client/ClientTest.java index 5f0fe0c1b004..891425ee1dfc 100644 --- a/ksqldb-api-client/src/test/java/io/confluent/ksql/api/client/ClientTest.java +++ b/ksqldb-api-client/src/test/java/io/confluent/ksql/api/client/ClientTest.java @@ -52,6 +52,8 @@ import io.confluent.ksql.model.WindowType; import io.confluent.ksql.parser.exception.ParseFailedException; import io.confluent.ksql.query.QueryId; +import io.confluent.ksql.rest.entity.AssertSchemaEntity; +import io.confluent.ksql.rest.entity.AssertTopicEntity; import io.confluent.ksql.rest.entity.CommandId; import io.confluent.ksql.rest.entity.CommandStatus; import io.confluent.ksql.rest.entity.CommandStatusEntity; @@ -1798,6 +1800,110 @@ public void shouldSendCustomRequestHeaders() throws Exception { } } + @Test + public void shouldSendAssertSchemaWithSubjectAndId() throws Exception { + // Given + final AssertSchemaEntity entity = new AssertSchemaEntity("assert schema;", Optional.of("name"), Optional.of(3), true); + testEndpoints.setKsqlEndpointResponse(Collections.singletonList(entity)); + + // When: + javaClient.assertSchema("name", 3, true).get(); + + // Then: + assertThat(testEndpoints.getLastSql(), is("assert schema subject 'name' id 3;")); + } + + @Test + public void shouldSendAssertSchemaWithSubject() throws Exception { + // Given + final AssertSchemaEntity entity = new AssertSchemaEntity("assert schema;", Optional.of("name"), Optional.empty(), true); + testEndpoints.setKsqlEndpointResponse(Collections.singletonList(entity)); + + // When: + javaClient.assertSchema("name", true).get(); + + // Then: + assertThat(testEndpoints.getLastSql(), is("assert schema subject 'name';")); + } + + @Test + public void shouldSendAssertSchemaWithId() throws Exception { + // Given + final AssertSchemaEntity entity = new AssertSchemaEntity("assert schema;", Optional.empty(), Optional.of(3), true); + testEndpoints.setKsqlEndpointResponse(Collections.singletonList(entity)); + + // When: + javaClient.assertSchema(3, true).get(); + + // Then: + assertThat(testEndpoints.getLastSql(), is("assert schema id 3;")); + } + + @Test + public void shouldSendAssertNotExistSchema() throws Exception { + // Given + final AssertSchemaEntity entity = new AssertSchemaEntity("assert schema;", Optional.of("name"), Optional.empty(), false); + testEndpoints.setKsqlEndpointResponse(Collections.singletonList(entity)); + + // When: + javaClient.assertSchema("name", false).get(); + + // Then: + assertThat(testEndpoints.getLastSql(), is("assert not exists schema subject 'name';")); + } + + @Test + public void shouldSendAssertSchemaWithTimeout() throws Exception { + // Given + final AssertSchemaEntity entity = new AssertSchemaEntity("assert schema;", Optional.empty(), Optional.of(3), true); + testEndpoints.setKsqlEndpointResponse(Collections.singletonList(entity)); + + // When: + javaClient.assertSchema(3, true, Duration.ofSeconds(10)).get(); + + // Then: + assertThat(testEndpoints.getLastSql(), is("assert schema id 3 timeout 10 seconds;")); + } + + @Test + public void shouldSendAssertTopic() throws Exception { + // Given + final AssertTopicEntity entity = new AssertTopicEntity("assert topic;", "name", true); + testEndpoints.setKsqlEndpointResponse(Collections.singletonList(entity)); + + // When: + javaClient.assertTopic("name", true).get(); + + // Then: + assertThat(testEndpoints.getLastSql(), is("assert topic 'name';")); + } + + @Test + public void shouldSendAssertTopicWithConfigs() throws Exception { + // Given + final AssertTopicEntity entity = new AssertTopicEntity("assert topic;", "name", true); + testEndpoints.setKsqlEndpointResponse(Collections.singletonList(entity)); + + // When: + javaClient.assertTopic("name", ImmutableMap.of("foo", 3, "bar", 5),true).get(); + + // Then: + assertThat(testEndpoints.getLastSql(), is("assert topic 'name' with (foo=3,bar=5);")); + } + + @Test + public void shouldSendAssertNotExistsTopicWithTimeout() throws Exception { + // Given + final AssertTopicEntity entity = new AssertTopicEntity("assert topic;", "name", false); + testEndpoints.setKsqlEndpointResponse(Collections.singletonList(entity)); + + // When: + javaClient.assertTopic("name",false, Duration.ofSeconds(10)).get(); + + // Then: + assertThat(testEndpoints.getLastSql(), is("assert not exists topic 'name' timeout 10 seconds;")); + } + protected Client createJavaClient() { return Client.create(createJavaClientOptions(), vertx); } diff --git a/ksqldb-api-client/src/test/java/io/confluent/ksql/api/client/impl/ClientImplTest.java b/ksqldb-api-client/src/test/java/io/confluent/ksql/api/client/impl/ClientImplTest.java index bfb7c331b752..737d597925cd 100644 --- a/ksqldb-api-client/src/test/java/io/confluent/ksql/api/client/impl/ClientImplTest.java +++ b/ksqldb-api-client/src/test/java/io/confluent/ksql/api/client/impl/ClientImplTest.java @@ -15,6 +15,7 @@ package io.confluent.ksql.api.client.impl; +import static io.netty.handler.codec.http.HttpHeaderNames.ACCEPT; import static io.netty.handler.codec.http.HttpHeaderNames.USER_AGENT; import static org.hamcrest.MatcherAssert.assertThat; import static org.hamcrest.Matchers.is; @@ -25,6 +26,7 @@ import com.google.common.testing.EqualsTester; import io.confluent.ksql.api.client.Client; import io.confluent.ksql.api.client.ClientOptions; +import io.confluent.ksql.rest.entity.KsqlMediaType; import io.vertx.core.Vertx; import io.vertx.core.http.HttpClient; import io.vertx.core.http.HttpClientRequest; @@ -62,7 +64,7 @@ public void shouldImplementHashCodeAndEquals() { } @Test - public void shouldSetUserAgent() { + public void shouldSetUserAgentAndAcceptHeaders() { // Given Vertx vertx = Mockito.mock(Vertx.class); HttpClient httpClient = Mockito.mock(HttpClient.class); @@ -83,10 +85,14 @@ public void shouldSetUserAgent() { // Then Client client = Client.create(OPTIONS_1, vertx); client.streamQuery("SELECT * from STREAM1 EMIT CHANGES;"); - assertThat(headers.size(), is(1)); + assertThat(headers.size(), is(2)); assertThat(headers.containsKey(USER_AGENT.toString()), is(true)); assertThat(headers.get(USER_AGENT.toString()).matches("ksqlDB Java Client v\\d\\.\\d\\.\\d.*"), is(true)); + + assertThat(headers.containsKey(ACCEPT.toString()), is(true)); + assertThat(headers.get(ACCEPT.toString()).equals(KsqlMediaType.LATEST_FORMAT.mediaType()), + is(true)); } } \ No newline at end of file diff --git a/ksqldb-api-client/src/test/java/io/confluent/ksql/api/client/integration/AssertClientIntegrationTest.java b/ksqldb-api-client/src/test/java/io/confluent/ksql/api/client/integration/AssertClientIntegrationTest.java new file mode 100644 index 000000000000..b88653df6fbb --- /dev/null +++ b/ksqldb-api-client/src/test/java/io/confluent/ksql/api/client/integration/AssertClientIntegrationTest.java @@ -0,0 +1,217 @@ +/* + * Copyright 2022 Confluent Inc. + * + * Licensed under the Confluent Community License (the "License"); you may not use + * this file except in compliance with the License. You may obtain a copy of the + * License at + * + * http://www.confluent.io/confluent-community-license + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, WITHOUT + * WARRANTIES OF ANY KIND, either express or implied. See the License for the + * specific language governing permissions and limitations under the License. + */ + +package io.confluent.ksql.api.client.integration; + +import static io.confluent.ksql.util.KsqlConfig.KSQL_DEFAULT_KEY_FORMAT_CONFIG; +import static io.confluent.ksql.util.KsqlConfig.KSQL_HEADERS_COLUMNS_ENABLED; +import static io.confluent.ksql.util.KsqlConfig.KSQL_STREAMS_PREFIX; +import static org.hamcrest.MatcherAssert.assertThat; +import static org.hamcrest.Matchers.containsString; +import static org.hamcrest.Matchers.instanceOf; +import static org.junit.Assert.assertThrows; + +import com.google.common.collect.ImmutableMap; +import io.confluent.common.utils.IntegrationTest; +import io.confluent.ksql.api.client.Client; +import io.confluent.ksql.api.client.ClientOptions; +import io.confluent.ksql.api.client.exception.KsqlClientException; +import io.confluent.ksql.integration.IntegrationTestHarness; +import io.confluent.ksql.integration.Retry; +import io.confluent.ksql.name.ColumnName; +import io.confluent.ksql.rest.server.TestKsqlRestApp; +import io.confluent.ksql.schema.ksql.LogicalSchema; +import io.confluent.ksql.schema.ksql.PhysicalSchema; +import io.confluent.ksql.schema.ksql.types.SqlTypes; +import io.confluent.ksql.serde.SerdeFeatures; +import io.confluent.ksql.util.KsqlConfig; +import io.vertx.core.Vertx; +import java.time.Duration; +import java.util.concurrent.ExecutionException; +import java.util.concurrent.TimeUnit; +import kafka.zookeeper.ZooKeeperClientException; +import org.apache.kafka.streams.StreamsConfig; +import org.junit.After; +import org.junit.Before; +import org.junit.BeforeClass; +import org.junit.ClassRule; +import org.junit.Test; +import org.junit.experimental.categories.Category; +import org.junit.rules.RuleChain; + +@Category({IntegrationTest.class}) +public class AssertClientIntegrationTest { + + private static final PhysicalSchema SCHEMA = PhysicalSchema.from( + LogicalSchema.builder() + .keyColumn(ColumnName.of("K"), SqlTypes.INTEGER) + .valueColumn(ColumnName.of("LONG"), SqlTypes.BIGINT) + .build(), + SerdeFeatures.of(), + SerdeFeatures.of() + ); + + private static final IntegrationTestHarness TEST_HARNESS = IntegrationTestHarness.build(); + + private static final TestKsqlRestApp REST_APP = TestKsqlRestApp + .builder(TEST_HARNESS::kafkaBootstrapServers) + .withProperty(KSQL_STREAMS_PREFIX + StreamsConfig.NUM_STREAM_THREADS_CONFIG, 1) + .withProperty(KSQL_DEFAULT_KEY_FORMAT_CONFIG, "JSON") + .withProperty(KSQL_HEADERS_COLUMNS_ENABLED, true) + .withStaticServiceContext(TEST_HARNESS::getServiceContext) + .withProperty(KsqlConfig.SCHEMA_REGISTRY_URL_PROPERTY, "http://foo:8080") + .build(); + + @ClassRule + public static final RuleChain CHAIN = RuleChain + .outerRule(Retry.of(3, ZooKeeperClientException.class, 3, TimeUnit.SECONDS)) + .around(TEST_HARNESS) + .around(REST_APP); + + @BeforeClass + public static void setUpClass() { + TEST_HARNESS.ensureTopics("abc"); + // registers a schema under the subject name "abc-value" + TEST_HARNESS.ensureSchema("abc", SCHEMA, false); + } + + private Vertx vertx; + private Client client; + + @Before + public void setUp() { + vertx = Vertx.vertx(); + client = createClient(); + } + + @After + public void tearDown() { + if (client != null) { + client.close(); + } + if (vertx != null) { + vertx.close(); + } + REST_APP.getServiceContext().close(); + } + + @Test + public void shouldThrowOnAssertNonexistentSchema() { + // When + final Exception e = assertThrows( + ExecutionException.class, // thrown from .get() when the future completes exceptionally + () -> client.assertSchema("NONEXISTENT", 3, true, Duration.ofSeconds(3)).get() + ); + + // Then: + assertThat(e.getCause(), instanceOf(KsqlClientException.class)); + assertThat(e.getCause().getMessage(), containsString("Received 417 response from server")); + assertThat(e.getCause().getMessage(), containsString("Schema with subject name NONEXISTENT id 3 does not exist")); + assertThat(e.getCause().getMessage(), containsString("Error code: 41700")); + } + + @Test + public void shouldThrowOnAssertNotExistsNonExistingSchema() { + // When + final Exception e = assertThrows( + ExecutionException.class, // thrown from .get() when the future completes exceptionally + () -> client.assertSchema("abc-value", 1, false, Duration.ofSeconds(3)).get() + ); + + // Then: + assertThat(e.getCause(), instanceOf(KsqlClientException.class)); + assertThat(e.getCause().getMessage(), containsString("Received 417 response from server")); + assertThat(e.getCause().getMessage(), containsString("Schema with subject name abc-value id 1 exists.")); + assertThat(e.getCause().getMessage(), containsString("Error code: 41700")); + } + + @Test + public void shouldAssertSchema() throws ExecutionException, InterruptedException { + client.assertSchema("abc-value", true).get(); + client.assertSchema("abc-value", 1, true).get(); + client.assertSchema(1, true, Duration.ofSeconds(3)).get(); + } + + @Test + public void shouldAssertNotExistsSchema() throws ExecutionException, InterruptedException { + client.assertSchema(34, false).get(); + client.assertSchema("NONEXISTENT", false, Duration.ofSeconds(3)).get(); + client.assertSchema("NONEXISTENT", 43, false, Duration.ofSeconds(3)).get(); + } + + @Test + public void shouldThrowOnAssertNonexistentTopic() { + // When + final Exception e1 = assertThrows( + ExecutionException.class, // thrown from .get() when the future completes exceptionally + () -> client.assertTopic("NONEXISTENT", true).get() + ); + final Exception e2 = assertThrows( + ExecutionException.class, // thrown from .get() when the future completes exceptionally + () -> client.assertTopic("abc", ImmutableMap.of("partitions", 7, "foo", 3), true).get() + ); + + // Then: + assertThat(e1.getCause(), instanceOf(KsqlClientException.class)); + assertThat(e1.getCause().getMessage(), containsString("Received 417 response from server")); + assertThat(e1.getCause().getMessage(), containsString("Topic NONEXISTENT does not exist.")); + assertThat(e1.getCause().getMessage(), containsString("Error code: 41700")); + + assertThat(e2.getCause(), instanceOf(KsqlClientException.class)); + assertThat(e2.getCause().getMessage(), containsString("Received 417 response from server")); + assertThat(e2.getCause().getMessage(), containsString("Mismatched configuration for topic abc: For config partitions, expected 7 got 1")); + assertThat(e2.getCause().getMessage(), containsString("Cannot assert unknown topic property: FOO.")); + assertThat(e2.getCause().getMessage(), containsString("Error code: 41700")); + } + + @Test + public void shouldThrowOnAssertNotExistsNonExistingTopic() { + // When + final Exception e = assertThrows( + ExecutionException.class, // thrown from .get() when the future completes exceptionally + () -> client.assertTopic("abc", false, Duration.ofSeconds(3)).get() + ); + + // Then: + assertThat(e.getCause(), instanceOf(KsqlClientException.class)); + assertThat(e.getCause().getMessage(), containsString("Received 417 response from server")); + assertThat(e.getCause().getMessage(), containsString("Topic abc exists")); + assertThat(e.getCause().getMessage(), containsString("Error code: 41700")); + } + + @Test + public void shouldAssertTopic() throws ExecutionException, InterruptedException { + // Given: + client.define("name", "abc"); + + // These should run without throwing any errors + client.assertTopic("abc", true).get(); + client.assertTopic("${name}", ImmutableMap.of("replicas", 1, "partitions", 1), true, Duration.ofSeconds(3)).get(); + } + + @Test + public void shouldAssertNotExistsTopic() throws ExecutionException, InterruptedException { + // These should run without throwing any errors + client.assertTopic("NONEXISTENT", ImmutableMap.of("replicas", 1, "partitions", 1), false).get(); + client.assertTopic("NONEXISTENT", false, Duration.ofSeconds(3)).get(); + } + + private Client createClient() { + final ClientOptions clientOptions = ClientOptions.create() + .setHost("localhost") + .setPort(REST_APP.getListeners().get(0).getPort()); + return Client.create(clientOptions, vertx); + } +} diff --git a/ksqldb-api-client/src/test/java/io/confluent/ksql/api/client/util/RowUtilTest.java b/ksqldb-api-client/src/test/java/io/confluent/ksql/api/client/util/RowUtilTest.java index 96db8a84baeb..6aa227a3369f 100644 --- a/ksqldb-api-client/src/test/java/io/confluent/ksql/api/client/util/RowUtilTest.java +++ b/ksqldb-api-client/src/test/java/io/confluent/ksql/api/client/util/RowUtilTest.java @@ -67,4 +67,84 @@ public void shouldGetColumnTypesFromStrings() { "TIME" )); } + + @Test + public void shouldGetColumnNamesFromSchema() { + // Given + final String schema = "`K` STRUCT<`F1` ARRAY>, " + + "`STR` STRING, " + + "`LONG` BIGINT, " + + "`DEC` DECIMAL(4, 2)," + + "`BYTES_` BYTES, " + + "`ARRAY` ARRAY, " + + "`MAP` MAP, " + + "`STRUCT` STRUCT<`F1` INTEGER>, " + + "`COMPLEX` STRUCT<`DECIMAL` DECIMAL(2, 1), `STRUCT` STRUCT<`F1` STRING, `F2` INTEGER>, `ARRAY_ARRAY` ARRAY>, `ARRAY_STRUCT` ARRAY>, `ARRAY_MAP` ARRAY>, `MAP_ARRAY` MAP>, `MAP_MAP` MAP>, `MAP_STRUCT` MAP>>, " + + "`TIMESTAMP` TIMESTAMP, " + + "`DATE` DATE, " + + "`TIME` TIME, " + + "`HEAD` BYTES"; + + // When + final List columnNames = RowUtil.colNamesFromSchema(schema); + + // Then + assertThat( + columnNames, + contains( + "K", + "STR", + "LONG", + "DEC", + "BYTES_", + "ARRAY", + "MAP", + "STRUCT", + "COMPLEX", + "TIMESTAMP", + "DATE", + "TIME", + "HEAD" + )); + } + + @Test + public void shouldGetColumnTypesFromSchema() { + // Given + final String schema = "`K` STRUCT<`F1` ARRAY>, " + + "`STR` STRING, " + + "`LONG` BIGINT, " + + "`DEC` DECIMAL(4, 2)," + + "`BYTES_` BYTES, " + + "`ARRAY` ARRAY, " + + "`MAP` MAP, " + + "`STRUCT` STRUCT<`F1` INTEGER>, " + + "`COMPLEX` STRUCT<`DECIMAL` DECIMAL(2, 1), `STRUCT` STRUCT<`F1` STRING, `F2` INTEGER>, `ARRAY_ARRAY` ARRAY>, `ARRAY_STRUCT` ARRAY>, `ARRAY_MAP` ARRAY>, `MAP_ARRAY` MAP>, `MAP_MAP` MAP>, `MAP_STRUCT` MAP>>, " + + "`TIMESTAMP` TIMESTAMP, " + + "`DATE` DATE, " + + "`TIME` TIME, " + + "`HEAD` BYTES"; + + // When + final List columnTypes = RowUtil.colTypesFromSchema(schema); + + // Then + assertThat( + columnTypes, + contains( + "STRUCT>", + "STRING", + "BIGINT", + "DECIMAL(4, 2)", + "BYTES", + "ARRAY", + "MAP", + "STRUCT", + "STRUCT, ARRAY_ARRAY ARRAY>, ARRAY_STRUCT ARRAY>, ARRAY_MAP ARRAY>, MAP_ARRAY MAP>, MAP_MAP MAP>, MAP_STRUCT MAP>>", + "TIMESTAMP", + "DATE", + "TIME", + "BYTES" + )); + } } \ No newline at end of file diff --git a/ksqldb-cli/src/main/java/io/confluent/ksql/cli/console/Console.java b/ksqldb-cli/src/main/java/io/confluent/ksql/cli/console/Console.java index ebf6e6b3d95c..02d0482fdb1a 100644 --- a/ksqldb-cli/src/main/java/io/confluent/ksql/cli/console/Console.java +++ b/ksqldb-cli/src/main/java/io/confluent/ksql/cli/console/Console.java @@ -51,6 +51,8 @@ import io.confluent.ksql.query.QueryError; import io.confluent.ksql.rest.ApiJsonMapper; import io.confluent.ksql.rest.entity.ArgumentInfo; +import io.confluent.ksql.rest.entity.AssertSchemaEntity; +import io.confluent.ksql.rest.entity.AssertTopicEntity; import io.confluent.ksql.rest.entity.CommandStatusEntity; import io.confluent.ksql.rest.entity.ConnectorDescription; import io.confluent.ksql.rest.entity.ConnectorList; @@ -191,6 +193,8 @@ public class Console implements Closeable { tablePrinter(VariablesList.class, ListVariablesTableBuilder::new)) .put(TerminateQueryEntity.class, tablePrinter(TerminateQueryEntity.class, TerminateQueryTableBuilder::new)) + .put(AssertTopicEntity.class, Console::printAssertTopic) + .put(AssertSchemaEntity.class, Console::printAssertSchema) .build(); private static Handler1 tablePrinter( @@ -904,6 +908,26 @@ private void printFunctionDescription(final FunctionDescriptionList describeFunc ); } + private void printAssertTopic(final AssertTopicEntity assertTopic) { + final String existence = assertTopic.getExists() ? " exists" : " does not exist"; + writer().printf("Topic " + assertTopic.getTopicName() + existence + ".\n"); + } + + private void printAssertSchema(final AssertSchemaEntity assertSchema) { + if (!assertSchema.getId().isPresent() && !assertSchema.getSubject().isPresent()) { + throw new RuntimeException("No subject or id found in AssertSchema response."); + } + + final String existence = assertSchema.getExists() ? " exists" : " does not exist"; + final String subject = assertSchema.getSubject().isPresent() + ? " subject " + assertSchema.getSubject().get() + : ""; + final String id = assertSchema.getId().isPresent() + ? " id " + assertSchema.getId().get() + : ""; + writer().printf("Schema with" + subject + id + existence + ".\n"); + } + private static String argToString(final ArgumentInfo arg) { final String type = arg.getType() + (arg.getIsVariadic() ? "[]" : ""); return arg.getName().isEmpty() ? type : (arg.getName() + " " + type); diff --git a/ksqldb-cli/src/test/java/io/confluent/ksql/cli/console/ConsoleTest.java b/ksqldb-cli/src/test/java/io/confluent/ksql/cli/console/ConsoleTest.java index df43473f8e9a..dd036d3d3598 100644 --- a/ksqldb-cli/src/test/java/io/confluent/ksql/cli/console/ConsoleTest.java +++ b/ksqldb-cli/src/test/java/io/confluent/ksql/cli/console/ConsoleTest.java @@ -26,8 +26,6 @@ import static org.mockito.Mockito.verify; import static org.mockito.Mockito.when; -import com.fasterxml.jackson.databind.ObjectMapper; -import com.google.common.base.Strings; import com.google.common.collect.ImmutableList; import com.google.common.collect.ImmutableMap; import com.google.common.collect.ImmutableSet; @@ -44,6 +42,8 @@ import io.confluent.ksql.query.QueryId; import io.confluent.ksql.rest.Errors; import io.confluent.ksql.rest.entity.ArgumentInfo; +import io.confluent.ksql.rest.entity.AssertSchemaEntity; +import io.confluent.ksql.rest.entity.AssertTopicEntity; import io.confluent.ksql.rest.entity.CommandId; import io.confluent.ksql.rest.entity.CommandStatus; import io.confluent.ksql.rest.entity.CommandStatusEntity; @@ -95,7 +95,6 @@ import io.confluent.ksql.util.KsqlConstants; import io.confluent.ksql.util.KsqlConstants.KsqlQueryStatus; import io.confluent.ksql.util.KsqlConstants.KsqlQueryType; -import java.io.IOException; import java.util.ArrayList; import java.util.Collection; import java.util.Collections; @@ -106,7 +105,6 @@ import java.util.function.Supplier; import java.util.stream.Collectors; import java.util.stream.IntStream; -import org.apache.commons.lang3.StringUtils; import org.apache.kafka.connect.runtime.rest.entities.ConnectorStateInfo; import org.apache.kafka.connect.runtime.rest.entities.ConnectorStateInfo.ConnectorState; import org.apache.kafka.connect.runtime.rest.entities.ConnectorStateInfo.TaskState; @@ -1094,4 +1092,64 @@ public void shouldThrowOnInvalidCliPropertyValue() { assertThat(terminal.getOutputString(), containsString("Invalid value BURRITO for configuration WRAP: String must be one of: ON, OFF, null")); } + + @Test + public void shouldPrintAssertTopicResult() { + // Given: + final KsqlEntityList entities = new KsqlEntityList(ImmutableList.of( + new AssertTopicEntity("statement", "name", true) + )); + + // When: + console.printKsqlEntityList(entities); + + // Then: + final String output = terminal.getOutputString(); + Approvals.verify(output, approvalOptions); + } + + @Test + public void shouldPrintAssertNotExistsTopicResult() { + // Given: + final KsqlEntityList entities = new KsqlEntityList(ImmutableList.of( + new AssertTopicEntity("statement", "name", false) + )); + + // When: + console.printKsqlEntityList(entities); + + // Then: + final String output = terminal.getOutputString(); + Approvals.verify(output, approvalOptions); + } + + @Test + public void shouldPrintAssertSchemaResult() { + // Given: + final KsqlEntityList entities = new KsqlEntityList(ImmutableList.of( + new AssertSchemaEntity("statement", Optional.of("abc"), Optional.of(55), true) + )); + + // When: + console.printKsqlEntityList(entities); + + // Then: + final String output = terminal.getOutputString(); + Approvals.verify(output, approvalOptions); + } + + @Test + public void shouldPrintAssertNotExistsSchemaResult() { + // Given: + final KsqlEntityList entities = new KsqlEntityList(ImmutableList.of( + new AssertSchemaEntity("statement", Optional.of("abc"), Optional.of(55), false) + )); + + // When: + console.printKsqlEntityList(entities); + + // Then: + final String output = terminal.getOutputString(); + Approvals.verify(output, approvalOptions); + } } diff --git a/ksqldb-cli/src/test/java/io/confluent/ksql/cli/console/ConsoleTest.shouldPrintAssertNotExistsSchemaResult.approved.json b/ksqldb-cli/src/test/java/io/confluent/ksql/cli/console/ConsoleTest.shouldPrintAssertNotExistsSchemaResult.approved.json new file mode 100644 index 000000000000..3c77ce574720 --- /dev/null +++ b/ksqldb-cli/src/test/java/io/confluent/ksql/cli/console/ConsoleTest.shouldPrintAssertNotExistsSchemaResult.approved.json @@ -0,0 +1,8 @@ +[ { + "@type" : "assert_schema", + "statementText" : "statement", + "subject" : "abc", + "id" : 55, + "exists" : false, + "warnings" : [ ] +} ] \ No newline at end of file diff --git a/ksqldb-cli/src/test/java/io/confluent/ksql/cli/console/ConsoleTest.shouldPrintAssertNotExistsSchemaResult.approved.tabular b/ksqldb-cli/src/test/java/io/confluent/ksql/cli/console/ConsoleTest.shouldPrintAssertNotExistsSchemaResult.approved.tabular new file mode 100644 index 000000000000..8b2b40e97469 --- /dev/null +++ b/ksqldb-cli/src/test/java/io/confluent/ksql/cli/console/ConsoleTest.shouldPrintAssertNotExistsSchemaResult.approved.tabular @@ -0,0 +1,2 @@ + +Schema with subject abc id 55 does not exist. diff --git a/ksqldb-cli/src/test/java/io/confluent/ksql/cli/console/ConsoleTest.shouldPrintAssertNotExistsTopicResult.approved.json b/ksqldb-cli/src/test/java/io/confluent/ksql/cli/console/ConsoleTest.shouldPrintAssertNotExistsTopicResult.approved.json new file mode 100644 index 000000000000..546903bc0ac8 --- /dev/null +++ b/ksqldb-cli/src/test/java/io/confluent/ksql/cli/console/ConsoleTest.shouldPrintAssertNotExistsTopicResult.approved.json @@ -0,0 +1,7 @@ +[ { + "@type" : "assert_topic", + "statementText" : "statement", + "topicName" : "name", + "exists" : false, + "warnings" : [ ] +} ] diff --git a/ksqldb-cli/src/test/java/io/confluent/ksql/cli/console/ConsoleTest.shouldPrintAssertNotExistsTopicResult.approved.tabular b/ksqldb-cli/src/test/java/io/confluent/ksql/cli/console/ConsoleTest.shouldPrintAssertNotExistsTopicResult.approved.tabular new file mode 100644 index 000000000000..675e854e00fb --- /dev/null +++ b/ksqldb-cli/src/test/java/io/confluent/ksql/cli/console/ConsoleTest.shouldPrintAssertNotExistsTopicResult.approved.tabular @@ -0,0 +1,2 @@ + +Topic name does not exist. diff --git a/ksqldb-cli/src/test/java/io/confluent/ksql/cli/console/ConsoleTest.shouldPrintAssertSchemaResult.approved.json b/ksqldb-cli/src/test/java/io/confluent/ksql/cli/console/ConsoleTest.shouldPrintAssertSchemaResult.approved.json new file mode 100644 index 000000000000..dda91cd7b1c6 --- /dev/null +++ b/ksqldb-cli/src/test/java/io/confluent/ksql/cli/console/ConsoleTest.shouldPrintAssertSchemaResult.approved.json @@ -0,0 +1,8 @@ +[ { + "@type" : "assert_schema", + "statementText" : "statement", + "subject" : "abc", + "id" : 55, + "exists" : true, + "warnings" : [ ] +} ] \ No newline at end of file diff --git a/ksqldb-cli/src/test/java/io/confluent/ksql/cli/console/ConsoleTest.shouldPrintAssertSchemaResult.approved.tabular b/ksqldb-cli/src/test/java/io/confluent/ksql/cli/console/ConsoleTest.shouldPrintAssertSchemaResult.approved.tabular new file mode 100644 index 000000000000..0a1012dd2030 --- /dev/null +++ b/ksqldb-cli/src/test/java/io/confluent/ksql/cli/console/ConsoleTest.shouldPrintAssertSchemaResult.approved.tabular @@ -0,0 +1,2 @@ + +Schema with subject abc id 55 exists. diff --git a/ksqldb-cli/src/test/java/io/confluent/ksql/cli/console/ConsoleTest.shouldPrintAssertTopicResult.approved.json b/ksqldb-cli/src/test/java/io/confluent/ksql/cli/console/ConsoleTest.shouldPrintAssertTopicResult.approved.json new file mode 100644 index 000000000000..1aea8557310f --- /dev/null +++ b/ksqldb-cli/src/test/java/io/confluent/ksql/cli/console/ConsoleTest.shouldPrintAssertTopicResult.approved.json @@ -0,0 +1,7 @@ +[ { + "@type" : "assert_topic", + "statementText" : "statement", + "topicName" : "name", + "exists" : true, + "warnings" : [ ] +} ] \ No newline at end of file diff --git a/ksqldb-cli/src/test/java/io/confluent/ksql/cli/console/ConsoleTest.shouldPrintAssertTopicResult.approved.tabular b/ksqldb-cli/src/test/java/io/confluent/ksql/cli/console/ConsoleTest.shouldPrintAssertTopicResult.approved.tabular new file mode 100644 index 000000000000..d18e0b2bdd25 --- /dev/null +++ b/ksqldb-cli/src/test/java/io/confluent/ksql/cli/console/ConsoleTest.shouldPrintAssertTopicResult.approved.tabular @@ -0,0 +1,2 @@ + +Topic name exists. diff --git a/ksqldb-common/src/main/java/io/confluent/ksql/function/AggregateFunctionFactory.java b/ksqldb-common/src/main/java/io/confluent/ksql/function/AggregateFunctionFactory.java index e4af23764c83..a6558a49799a 100644 --- a/ksqldb-common/src/main/java/io/confluent/ksql/function/AggregateFunctionFactory.java +++ b/ksqldb-common/src/main/java/io/confluent/ksql/function/AggregateFunctionFactory.java @@ -51,6 +51,13 @@ public abstract class AggregateFunctionFactory { .add(ImmutableList.of(ParamTypes.TIMESTAMP)) .build(); + protected static final ImmutableList> COMPARABLE_ARGS = ImmutableList + .>builder() + .addAll(NUMERICAL_TIME) + .add(ImmutableList.of(ParamTypes.STRING)) + .add(ImmutableList.of(ParamTypes.BYTES)) + .build(); + public AggregateFunctionFactory(final String functionName) { this(new UdfMetadata( functionName, diff --git a/ksqldb-common/src/main/java/io/confluent/ksql/logging/processing/MeteredProcessingLogger.java b/ksqldb-common/src/main/java/io/confluent/ksql/logging/processing/MeteredProcessingLogger.java new file mode 100644 index 000000000000..0dfc46dc11e0 --- /dev/null +++ b/ksqldb-common/src/main/java/io/confluent/ksql/logging/processing/MeteredProcessingLogger.java @@ -0,0 +1,54 @@ +/* + * Copyright 2022 Confluent Inc. + * + * Licensed under the Confluent Community License (the "License"); you may not use + * this file except in compliance with the License. You may obtain a copy of the + * License at + * + * http://www.confluent.io/confluent-community-license + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, WITHOUT + * WARRANTIES OF ANY KIND, either express or implied. See the License for the + * specific language governing permissions and limitations under the License. + */ + +package io.confluent.ksql.logging.processing; + +import edu.umd.cs.findbugs.annotations.SuppressFBWarnings; +import java.util.Objects; +import org.apache.kafka.common.metrics.Metrics; +import org.apache.kafka.common.metrics.Sensor; + +public class MeteredProcessingLogger implements ProcessingLogger { + private final ProcessingLogger logger; + private final Metrics metrics; + private final Sensor errorSensor; + + @SuppressFBWarnings(value = "EI_EXPOSE_REP2") + public MeteredProcessingLogger( + final ProcessingLogger logger, + final Metrics metrics, + final Sensor errorSensor + ) { + this.logger = Objects.requireNonNull(logger, "logger"); + this.metrics = metrics; + this.errorSensor = errorSensor; + } + + @Override + public void error(final ErrorMessage msg) { + if (errorSensor != null) { + errorSensor.record(); + } + logger.error(msg); + } + + @Override + public void close() { + if (metrics != null) { + metrics.removeSensor(errorSensor.name()); + } + logger.close(); + } +} diff --git a/ksqldb-common/src/main/java/io/confluent/ksql/logging/processing/MeteredProcessingLoggerFactory.java b/ksqldb-common/src/main/java/io/confluent/ksql/logging/processing/MeteredProcessingLoggerFactory.java new file mode 100644 index 000000000000..d88f54beccc4 --- /dev/null +++ b/ksqldb-common/src/main/java/io/confluent/ksql/logging/processing/MeteredProcessingLoggerFactory.java @@ -0,0 +1,141 @@ +/* + * Copyright 2018 Confluent Inc. + * + * Licensed under the Confluent Community License (the "License"); you may not use + * this file except in compliance with the License. You may obtain a copy of the + * License at + * + * http://www.confluent.io/confluent-community-license + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, WITHOUT + * WARRANTIES OF ANY KIND, either express or implied. See the License for the + * specific language governing permissions and limitations under the License. + */ + +package io.confluent.ksql.logging.processing; + +import io.confluent.common.logging.StructuredLogger; +import io.confluent.common.logging.StructuredLoggerFactory; +import io.confluent.ksql.util.MetricsTagsUtil; +import java.util.Collection; +import java.util.Collections; +import java.util.HashMap; +import java.util.Map; +import java.util.function.BiFunction; +import java.util.function.Function; +import org.apache.kafka.common.MetricName; +import org.apache.kafka.common.metrics.Metrics; +import org.apache.kafka.common.metrics.Sensor; +import org.apache.kafka.common.metrics.stats.CumulativeSum; + +public class MeteredProcessingLoggerFactory implements ProcessingLoggerFactory { + public static final String PROCESSING_LOG_ERROR_METRIC_NAME = "processing-error-total"; + public static final String PROCESSING_LOG_METRICS_GROUP_NAME = "processing-diagnostic-metrics"; + public static final String PROCESSING_LOG_METRIC_DESCRIPTION = + "The total number of errors emitted by the processing log."; + + private final ProcessingLogConfig config; + private final StructuredLoggerFactory innerFactory; + private final Metrics metrics; + private final Map metricsTags; + private final BiFunction loggerFactory; + private final Function> + loggerWithMetricsFactory; + private final Map processingLoggers; + + MeteredProcessingLoggerFactory( + final ProcessingLogConfig config, + final StructuredLoggerFactory innerFactory, + final Metrics metrics, + final Map metricsTags + ) { + this( + config, + innerFactory, + metrics, + ProcessingLoggerImpl::new, + metricObject -> (processingLogger, sensor) -> + new MeteredProcessingLogger(processingLogger, metricObject, sensor), + metricsTags + ); + } + + MeteredProcessingLoggerFactory( + final ProcessingLogConfig config, + final StructuredLoggerFactory innerFactory, + final Metrics metrics, + final BiFunction loggerFactory, + final Function> + loggerWithMetricsFactory, + final Map metricsTags + ) { + this.config = config; + this.innerFactory = innerFactory; + this.metrics = metrics; + this.loggerFactory = loggerFactory; + this.loggerWithMetricsFactory = loggerWithMetricsFactory; + this.metricsTags = metricsTags; + this.processingLoggers = new HashMap<>(); + } + + @Override + public ProcessingLogger getLogger( + final String name + ) { + return getLogger(name, Collections.emptyMap()); + } + + @Override + public synchronized ProcessingLogger getLogger( + final String name, + final Map additionalTags + ) { + if (processingLoggers.containsKey(name)) { + return processingLoggers.get(name); + } + + // the metrics may be null if this is factory is created from a SandboxedExecutionContext + Sensor errorSensor = null; + if (metrics != null) { + final Map combinedMetricsTags = new HashMap<>(additionalTags); + combinedMetricsTags.putAll(metricsTags); + errorSensor = configureProcessingErrorSensor(metrics, combinedMetricsTags, name); + } + final ProcessingLogger meteredProcessingLogger = loggerWithMetricsFactory.apply(metrics).apply( + getProcessLogger(name), + errorSensor + ); + processingLoggers.put(name, meteredProcessingLogger); + return meteredProcessingLogger; + } + + @Override + public synchronized Collection getLoggers() { + return processingLoggers.values(); + } + + private static Sensor configureProcessingErrorSensor( + final Metrics metrics, + final Map metricsTags, + final String loggerName + ) { + final Map metricsTagsWithLoggerId = MetricsTagsUtil.getMetricsTagsWithLoggerId( + loggerName, + metricsTags + ); + final MetricName errorMetric = metrics.metricName( + PROCESSING_LOG_ERROR_METRIC_NAME, + PROCESSING_LOG_METRICS_GROUP_NAME, + PROCESSING_LOG_METRIC_DESCRIPTION, + metricsTagsWithLoggerId + ); + final Sensor sensor = metrics.sensor(loggerName); + sensor.add(errorMetric, new CumulativeSum()); + return sensor; + } + + private ProcessingLogger getProcessLogger(final String name) { + return loggerFactory.apply(config, innerFactory.getLogger(name)); + } +} diff --git a/ksqldb-common/src/main/java/io/confluent/ksql/logging/processing/NoopProcessingLogContext.java b/ksqldb-common/src/main/java/io/confluent/ksql/logging/processing/NoopProcessingLogContext.java index 237b1d7c674a..501e3e7cf068 100644 --- a/ksqldb-common/src/main/java/io/confluent/ksql/logging/processing/NoopProcessingLogContext.java +++ b/ksqldb-common/src/main/java/io/confluent/ksql/logging/processing/NoopProcessingLogContext.java @@ -18,6 +18,7 @@ import com.google.common.collect.ImmutableList; import com.google.common.collect.ImmutableMap; import java.util.Collection; +import java.util.Map; /** * An implementation of {@code ProcessingLogContext} that does nothing. @@ -26,20 +27,41 @@ public final class NoopProcessingLogContext implements ProcessingLogContext { private static final ProcessingLogConfig NOOP_CONFIG = new ProcessingLogConfig(ImmutableMap.of()); - public static final ProcessingLogger NOOP_LOGGER = msgFactory -> { }; - - private static final ProcessingLoggerFactory NOOP_FACTORY = new ProcessingLoggerFactory() { + public static final ProcessingLogger NOOP_LOGGER = new ProcessingLogger() { @Override - public ProcessingLogger getLogger(final String name) { - return NOOP_LOGGER; + public void error(final ErrorMessage errorMessage) { + // no-op } @Override - public Collection getLoggers() { - return ImmutableList.of(); + public void close() { + // no-op } }; + private static final ProcessingLoggerFactory NOOP_FACTORY = + new ProcessingLoggerFactory() { + @Override + public ProcessingLogger getLogger( + final String name, + final Map additionalTags + ) { + return NOOP_LOGGER; + } + + @Override + public ProcessingLogger getLogger( + final String name + ) { + return NOOP_LOGGER; + } + + @Override + public Collection getLoggers() { + return ImmutableList.of(); + } + }; + public static final ProcessingLogContext INSTANCE = new NoopProcessingLogContext(); private NoopProcessingLogContext() { diff --git a/ksqldb-common/src/main/java/io/confluent/ksql/logging/processing/ProcessingLogContext.java b/ksqldb-common/src/main/java/io/confluent/ksql/logging/processing/ProcessingLogContext.java index 17858cfa0875..4a23d4e94ac5 100644 --- a/ksqldb-common/src/main/java/io/confluent/ksql/logging/processing/ProcessingLogContext.java +++ b/ksqldb-common/src/main/java/io/confluent/ksql/logging/processing/ProcessingLogContext.java @@ -16,6 +16,9 @@ package io.confluent.ksql.logging.processing; import java.util.Collections; +import java.util.Map; +import org.apache.kafka.common.metrics.Metrics; + public interface ProcessingLogContext { /** @@ -31,17 +34,27 @@ public interface ProcessingLogContext { /** * Creates a processing log context that uses the supplied config. * @param config the processing log config - * @return: A processing log context that uses the supplied config. + * @param metrics the object that emits metrics + * @param metricsTags the metricsTags to include with the metrics + * @return A processing log context that uses the supplied config and emits metrics */ - static ProcessingLogContext create(final ProcessingLogConfig config) { - return new ProcessingLogContextImpl(config); + static ProcessingLogContext create( + final ProcessingLogConfig config, + final Metrics metrics, + final Map metricsTags + ) { + return new ProcessingLogContextImpl(config, metrics, metricsTags); } /** * Creates a processing log context that uses the default processing log config. - * @return A processing log context that uses the default config + * @return A processing log context that uses the default config and doesn't emit metrics */ static ProcessingLogContext create() { - return new ProcessingLogContextImpl(new ProcessingLogConfig(Collections.emptyMap())); + return new ProcessingLogContextImpl( + new ProcessingLogConfig(Collections.emptyMap()), + null, + Collections.emptyMap() + ); } } diff --git a/ksqldb-common/src/main/java/io/confluent/ksql/logging/processing/ProcessingLogContextImpl.java b/ksqldb-common/src/main/java/io/confluent/ksql/logging/processing/ProcessingLogContextImpl.java index 18eae41dba0d..dd68dbf4204b 100644 --- a/ksqldb-common/src/main/java/io/confluent/ksql/logging/processing/ProcessingLogContextImpl.java +++ b/ksqldb-common/src/main/java/io/confluent/ksql/logging/processing/ProcessingLogContextImpl.java @@ -16,16 +16,23 @@ package io.confluent.ksql.logging.processing; import io.confluent.common.logging.StructuredLoggerFactory; +import java.util.Map; +import org.apache.kafka.common.metrics.Metrics; public final class ProcessingLogContextImpl implements ProcessingLogContext { private final ProcessingLogConfig config; private final ProcessingLoggerFactory loggerFactory; - ProcessingLogContextImpl(final ProcessingLogConfig config) { + ProcessingLogContextImpl( + final ProcessingLogConfig config, + final Metrics metrics, + final Map metricsTags) { this.config = config; - this.loggerFactory = new ProcessingLoggerFactoryImpl( + this.loggerFactory = new MeteredProcessingLoggerFactory( config, - new StructuredLoggerFactory(ProcessingLogConstants.PREFIX) + new StructuredLoggerFactory(ProcessingLogConstants.PREFIX), + metrics, + metricsTags ); } diff --git a/ksqldb-common/src/main/java/io/confluent/ksql/logging/processing/ProcessingLogger.java b/ksqldb-common/src/main/java/io/confluent/ksql/logging/processing/ProcessingLogger.java index 18ff671baf9f..373fcc83ee82 100644 --- a/ksqldb-common/src/main/java/io/confluent/ksql/logging/processing/ProcessingLogger.java +++ b/ksqldb-common/src/main/java/io/confluent/ksql/logging/processing/ProcessingLogger.java @@ -44,4 +44,9 @@ interface ErrorMessage { * @param msg the error to log */ void error(ErrorMessage msg); + + /** + * Close the processing logger + */ + void close(); } diff --git a/ksqldb-common/src/main/java/io/confluent/ksql/logging/processing/ProcessingLoggerFactory.java b/ksqldb-common/src/main/java/io/confluent/ksql/logging/processing/ProcessingLoggerFactory.java index 3e7a19c9413d..a4e8d98fa28b 100644 --- a/ksqldb-common/src/main/java/io/confluent/ksql/logging/processing/ProcessingLoggerFactory.java +++ b/ksqldb-common/src/main/java/io/confluent/ksql/logging/processing/ProcessingLoggerFactory.java @@ -16,17 +16,27 @@ package io.confluent.ksql.logging.processing; import java.util.Collection; +import java.util.Map; public interface ProcessingLoggerFactory { /** - * Get a processing logger for writing record processing log messages + * Get a processing logger for writing record processing log messages. * @param name The name of the logger to get. - * @return The logger with the given name. + * @return The logger with the given name */ ProcessingLogger getLogger(String name); + /** + * Get a processing logger for writing record processing log messages. + * Additional tags can be associated with the logger. + * @param name The name of the logger to get. + * @param additionalTags Additional tags to associate with the logger + * @return The logger with the given name and tags associated with it. + */ + ProcessingLogger getLogger(String name, Map additionalTags); + /** * @return A collection of all loggers that have been created by the factory */ - Collection getLoggers(); + Collection getLoggers(); } diff --git a/ksqldb-common/src/main/java/io/confluent/ksql/logging/processing/ProcessingLoggerFactoryImpl.java b/ksqldb-common/src/main/java/io/confluent/ksql/logging/processing/ProcessingLoggerFactoryImpl.java deleted file mode 100644 index 622210add575..000000000000 --- a/ksqldb-common/src/main/java/io/confluent/ksql/logging/processing/ProcessingLoggerFactoryImpl.java +++ /dev/null @@ -1,53 +0,0 @@ -/* - * Copyright 2018 Confluent Inc. - * - * Licensed under the Confluent Community License (the "License"); you may not use - * this file except in compliance with the License. You may obtain a copy of the - * License at - * - * http://www.confluent.io/confluent-community-license - * - * Unless required by applicable law or agreed to in writing, software - * distributed under the License is distributed on an "AS IS" BASIS, WITHOUT - * WARRANTIES OF ANY KIND, either express or implied. See the License for the - * specific language governing permissions and limitations under the License. - */ - -package io.confluent.ksql.logging.processing; - -import io.confluent.common.logging.StructuredLogger; -import io.confluent.common.logging.StructuredLoggerFactory; -import java.util.Collection; -import java.util.function.BiFunction; - -public class ProcessingLoggerFactoryImpl implements ProcessingLoggerFactory { - private final ProcessingLogConfig config; - private final StructuredLoggerFactory innerFactory; - private final BiFunction loggerFactory; - - ProcessingLoggerFactoryImpl( - final ProcessingLogConfig config, - final StructuredLoggerFactory innerFactory) { - this(config, innerFactory, ProcessingLoggerImpl::new); - } - - ProcessingLoggerFactoryImpl( - final ProcessingLogConfig config, - final StructuredLoggerFactory innerFactory, - final BiFunction loggerFactory - ) { - this.config = config; - this.innerFactory = innerFactory; - this.loggerFactory = loggerFactory; - } - - @Override - public ProcessingLogger getLogger(final String name) { - return loggerFactory.apply(config, innerFactory.getLogger(name)); - } - - @Override - public Collection getLoggers() { - return innerFactory.getLoggers(); - } -} diff --git a/ksqldb-common/src/main/java/io/confluent/ksql/logging/processing/ProcessingLoggerImpl.java b/ksqldb-common/src/main/java/io/confluent/ksql/logging/processing/ProcessingLoggerImpl.java index 6a7a11411fd3..b8c7664f0b7c 100644 --- a/ksqldb-common/src/main/java/io/confluent/ksql/logging/processing/ProcessingLoggerImpl.java +++ b/ksqldb-common/src/main/java/io/confluent/ksql/logging/processing/ProcessingLoggerImpl.java @@ -35,6 +35,11 @@ public void error(final ErrorMessage msg) { inner.error(() -> throwIfNotRightSchema(msg.get(config))); } + @Override + public void close() { + // no-op for now + } + private static SchemaAndValue throwIfNotRightSchema(final SchemaAndValue schemaAndValue) { if (!schemaAndValue.schema().equals(ProcessingLogMessageSchema.PROCESSING_LOG_SCHEMA)) { throw new RuntimeException("Received message with invalid schema"); diff --git a/ksqldb-common/src/main/java/io/confluent/ksql/util/BytesUtils.java b/ksqldb-common/src/main/java/io/confluent/ksql/util/BytesUtils.java index a16d359a85ad..d952d937cd86 100644 --- a/ksqldb-common/src/main/java/io/confluent/ksql/util/BytesUtils.java +++ b/ksqldb-common/src/main/java/io/confluent/ksql/util/BytesUtils.java @@ -15,6 +15,7 @@ package io.confluent.ksql.util; import com.google.common.collect.ImmutableMap; +import io.vertx.core.buffer.Buffer; import java.nio.ByteBuffer; import java.nio.ByteOrder; import java.nio.charset.StandardCharsets; @@ -122,6 +123,24 @@ public static byte[] getByteArray(final ByteBuffer buffer, final int start, fina return Arrays.copyOfRange(getByteArray(buffer), start, end); } + public static Buffer toJsonMsg(final Buffer responseLine, final boolean stripArray) { + + int start = 0; + int end = responseLine.length() - 1; + if (stripArray) { + if (responseLine.getByte(0) == (byte) '[') { + start = 1; + } + if (responseLine.getByte(end) == (byte) ']') { + end -= 1; + } + } + if (end > 0 && responseLine.getByte(end) == (byte) ',') { + end -= 1; + } + return responseLine.slice(start, end + 1); + } + public static List split(final byte[] b, final byte[] delim) { if (b.length == 0) { return Arrays.asList(Arrays.copyOf(b, b.length)); diff --git a/ksqldb-common/src/main/java/io/confluent/ksql/util/KsqlConfig.java b/ksqldb-common/src/main/java/io/confluent/ksql/util/KsqlConfig.java index 0da6301b188b..55a530b27646 100644 --- a/ksqldb-common/src/main/java/io/confluent/ksql/util/KsqlConfig.java +++ b/ksqldb-common/src/main/java/io/confluent/ksql/util/KsqlConfig.java @@ -661,6 +661,12 @@ public class KsqlConfig extends AbstractConfig { private static final boolean KSQL_ENDPOINT_MIGRATE_QUERY_DEFAULT = true; private static final String KSQL_ENDPOINT_MIGRATE_QUERY_DOC = "Migrates the /query endpoint to use the same handler as /query-stream."; + public static final String KSQL_ASSERT_TOPIC_DEFAULT_TIMEOUT_MS + = "ksql.assert.topic.default.timeout.ms"; + private static final int KSQL_ASSERT_TOPIC_DEFAULT_TIMEOUT_MS_DEFAULT = 1000; + private static final String KSQL_ASSERT_TOPIC_DEFAULT_TIMEOUT_MS_DOC + = "The default timeout for ASSERT TOPIC statements if not timeout is specified " + + "in the statement."; private enum ConfigGeneration { LEGACY, @@ -1428,6 +1434,13 @@ private static ConfigDef buildConfigDef(final ConfigGeneration generation) { Importance.LOW, KSQL_TRANSIENT_QUERY_CLEANUP_SERVICE_PERIOD_SECONDS_DOC ) + .define( + KSQL_ASSERT_TOPIC_DEFAULT_TIMEOUT_MS, + Type.INT, + KSQL_ASSERT_TOPIC_DEFAULT_TIMEOUT_MS_DEFAULT, + Importance.LOW, + KSQL_ASSERT_TOPIC_DEFAULT_TIMEOUT_MS_DOC + ) .withClientSslSupport(); for (final CompatibilityBreakingConfigDef compatibilityBreakingConfigDef diff --git a/ksqldb-common/src/main/java/io/confluent/ksql/util/MetricsTagsUtil.java b/ksqldb-common/src/main/java/io/confluent/ksql/util/MetricsTagsUtil.java index f85470d5ee42..f97a3a1212ec 100644 --- a/ksqldb-common/src/main/java/io/confluent/ksql/util/MetricsTagsUtil.java +++ b/ksqldb-common/src/main/java/io/confluent/ksql/util/MetricsTagsUtil.java @@ -15,6 +15,7 @@ package io.confluent.ksql.util; +import com.google.common.collect.ImmutableMap; import java.util.HashMap; import java.util.Map; @@ -22,22 +23,31 @@ public final class MetricsTagsUtil { private MetricsTagsUtil() {} - public static Map getCustomMetricsTagsForQuery( - final String id, - final KsqlConfig config + public static Map getMetricsTagsWithQueryId( + final String queryId, + final Map tags ) { - return getCustomMetricsTagsForQuery( - id, - config.getStringAsMap(KsqlConfig.KSQL_CUSTOM_METRICS_TAGS)); + if (queryId.equals("")) { + return tags; + } + return addMetricTagToMap("query-id", queryId, tags); } - public static Map getCustomMetricsTagsForQuery( - final String id, - final Map metricsTags + public static Map getMetricsTagsWithLoggerId( + final String loggerId, + final Map tags ) { - final Map customMetricsTags = - new HashMap<>(metricsTags); - customMetricsTags.put("query_id", id); - return customMetricsTags; + return addMetricTagToMap("logger-id", loggerId, tags); + } + + private static Map addMetricTagToMap( + final String tagName, + final String tagValue, + final Map tags + ) { + final Map newMetricsTags = + new HashMap<>(tags); + newMetricsTags.put(tagName, tagValue); + return ImmutableMap.copyOf(newMetricsTags); } } diff --git a/ksqldb-common/src/test/java/io/confluent/ksql/logging/processing/MeteredProcessingLoggerFactoryTest.java b/ksqldb-common/src/test/java/io/confluent/ksql/logging/processing/MeteredProcessingLoggerFactoryTest.java new file mode 100644 index 000000000000..af8d1cad8659 --- /dev/null +++ b/ksqldb-common/src/test/java/io/confluent/ksql/logging/processing/MeteredProcessingLoggerFactoryTest.java @@ -0,0 +1,183 @@ +/* + * Copyright 2018 Confluent Inc. + * + * Licensed under the Confluent Community License (the "License"); you may not use + * this file except in compliance with the License. You may obtain a copy of the + * License at + * + * http://www.confluent.io/confluent-community-license + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, WITHOUT + * WARRANTIES OF ANY KIND, either express or implied. See the License for the + * specific language governing permissions and limitations under the License. + */ + +package io.confluent.ksql.logging.processing; + +import static org.hamcrest.MatcherAssert.assertThat; +import static org.hamcrest.Matchers.contains; +import static org.hamcrest.Matchers.equalTo; +import static org.hamcrest.Matchers.is; +import static org.mockito.ArgumentMatchers.any; +import static org.mockito.ArgumentMatchers.anyString; +import static org.mockito.Mockito.times; +import static org.mockito.Mockito.verify; +import static org.mockito.Mockito.when; + +import io.confluent.common.logging.StructuredLogger; +import io.confluent.common.logging.StructuredLoggerFactory; +import java.util.Collection; +import java.util.Collections; +import java.util.HashMap; +import java.util.Map; +import java.util.function.BiFunction; +import java.util.function.Function; + +import io.confluent.ksql.metrics.MetricCollectors; +import org.apache.kafka.common.metrics.Metrics; +import org.apache.kafka.common.metrics.Sensor; +import org.junit.Before; +import org.junit.Test; +import org.junit.runner.RunWith; +import org.mockito.Mock; +import org.mockito.junit.MockitoJUnitRunner; + +@RunWith(MockitoJUnitRunner.class) +public class MeteredProcessingLoggerFactoryTest { + @Mock + private StructuredLoggerFactory innerFactory; + @Mock + private StructuredLogger innerLogger; + @Mock + private ProcessingLogConfig config; + @Mock + private BiFunction loggerFactory; + @Mock + private Function> loggerWithMetricsFactory; + @Mock + private BiFunction loggerWithMetricsFactoryHelper; + @Mock + private ProcessingLogger logger; + @Mock + private ProcessingLogger loggerWithMetrics; + + private final Map customMetricsTags = Collections.singletonMap("tag1", "value1"); + private MeteredProcessingLoggerFactory factory; + private MetricCollectors metricCollectors; + + @Before + public void setup() { + metricCollectors = new MetricCollectors(); + when(innerFactory.getLogger(anyString())).thenReturn(innerLogger); + when(loggerFactory.apply(config, innerLogger)).thenReturn(logger); + when(loggerWithMetricsFactory.apply(any())).thenReturn(loggerWithMetricsFactoryHelper); + when(loggerWithMetricsFactoryHelper.apply(any(), any())).thenReturn(loggerWithMetrics); + factory = new MeteredProcessingLoggerFactory(config, innerFactory, metricCollectors.getMetrics(), loggerFactory, loggerWithMetricsFactory, customMetricsTags); + } + + @Test + public void shouldCreateLoggerWithoutPassingInTags() { + // Given: + final ProcessingLogger testLogger = factory.getLogger("foo.bar"); + final Sensor sensor = metricCollectors.getMetrics().getSensor("foo.bar"); + final Map metricsTags = new HashMap<>(customMetricsTags); + metricsTags.put("logger-id", "foo.bar"); + + // When: + sensor.record(); + sensor.record(); + + // Then: + assertThat(testLogger, is(this.loggerWithMetrics)); + verify(innerFactory).getLogger("foo.bar"); + verify(loggerFactory).apply(config, innerLogger); + verify(loggerWithMetricsFactory).apply(metricCollectors.getMetrics()); + verify(loggerWithMetricsFactoryHelper).apply(logger, sensor); + + // verify the metric was created correctly + assertThat(getMetricValue(metricsTags), equalTo(2.0)); + } + + @Test + public void shouldCreateLoggerWithPassingInAdditionalMetricsTags() { + // Given: + final Map metricsTags = new HashMap<>(customMetricsTags); + metricsTags.put("query-id", "some-id"); + final ProcessingLogger testLogger = factory.getLogger("boo.far", metricsTags); + + // When: + final Sensor sensor = metricCollectors.getMetrics().getSensor("boo.far"); + sensor.record(); + metricsTags.put("logger-id", "boo.far"); + + // Then: + assertThat(testLogger, is(this.loggerWithMetrics)); + verify(innerFactory).getLogger("boo.far"); + verify(loggerFactory).apply(config, innerLogger); + verify(loggerWithMetricsFactory).apply(metricCollectors.getMetrics()); + verify(loggerWithMetricsFactoryHelper).apply(logger, sensor); + + // verify the metric was created correctly + assertThat(getMetricValue(metricsTags), equalTo(1.0)); + } + + @Test + public void shouldReturnExistingLogger() { + // When: + factory.getLogger("boo.far", Collections.singletonMap("tag-value", "some-id-1")); + factory.getLogger("boo.far", Collections.singletonMap("tag-value", "some-id-2")); + factory.getLogger("boo.far", Collections.singletonMap("tag-value", "some-id-3")); + final Sensor sensor = metricCollectors.getMetrics().getSensor("boo.far"); + + // Then: + verify(innerFactory, times(1)).getLogger("boo.far"); + verify(loggerFactory, times(1)).apply(config, innerLogger); + verify(loggerWithMetricsFactory, times(1)).apply(metricCollectors.getMetrics()); + verify(loggerWithMetricsFactoryHelper, times(1)).apply(logger, sensor); + } + + @Test + public void shouldGetLoggers() { + // Given: + factory.getLogger("foo.bar"); + factory.getLogger("foo.bar", Collections.singletonMap("tag-value", "some-id-2")); + factory.getLogger("boo.far", Collections.singletonMap("tag-value", "some-id-2")); + + // When: + final Collection loggers = factory.getLoggers(); + + // Then: + assertThat(loggers.size(), equalTo(2)); + } + + @Test + public void shouldHandleNullMetrics() { + // Given: + final ProcessingLoggerFactory nullMetricsFactory = new MeteredProcessingLoggerFactory(config, innerFactory, null, loggerFactory, loggerWithMetricsFactory, customMetricsTags); + + // When: + final ProcessingLogger logger1 = nullMetricsFactory.getLogger("boo.far"); + final ProcessingLogger logger2 = nullMetricsFactory.getLogger("boo.far", Collections.singletonMap("tag1", "some-id-2")); + + // Then: + assertThat(logger1, is(loggerWithMetrics)); + assertThat(logger2, is(loggerWithMetrics)); + verify(loggerWithMetricsFactory).apply(null); + // no sensor created because metrics object is null + verify(loggerWithMetricsFactoryHelper).apply(logger, null); + } + + private double getMetricValue( final Map metricsTags) { + final Metrics metrics = metricCollectors.getMetrics(); + return Double.parseDouble( + metrics.metric( + metrics.metricName( + MeteredProcessingLoggerFactory.PROCESSING_LOG_ERROR_METRIC_NAME, + MeteredProcessingLoggerFactory.PROCESSING_LOG_METRICS_GROUP_NAME, + MeteredProcessingLoggerFactory.PROCESSING_LOG_METRIC_DESCRIPTION, + metricsTags) + ).metricValue().toString() + ); + } +} \ No newline at end of file diff --git a/ksqldb-common/src/test/java/io/confluent/ksql/logging/processing/MeteredProcessingLoggerTest.java b/ksqldb-common/src/test/java/io/confluent/ksql/logging/processing/MeteredProcessingLoggerTest.java new file mode 100644 index 000000000000..a48bfdbdbce9 --- /dev/null +++ b/ksqldb-common/src/test/java/io/confluent/ksql/logging/processing/MeteredProcessingLoggerTest.java @@ -0,0 +1,87 @@ +/* + * Copyright 2022 Confluent Inc. + * + * Licensed under the Confluent Community License (the "License"); you may not use + * this file except in compliance with the License. You may obtain a copy of the + * License at + * + * http://www.confluent.io/confluent-community-license + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, WITHOUT + * WARRANTIES OF ANY KIND, either express or implied. See the License for the + * specific language governing permissions and limitations under the License. + */ + +package io.confluent.ksql.logging.processing; + +import org.apache.kafka.common.metrics.Metrics; +import org.apache.kafka.common.metrics.Sensor; +import org.junit.Before; +import org.junit.Test; +import org.junit.runner.RunWith; +import org.mockito.Mock; +import org.mockito.junit.MockitoJUnitRunner; + +import static org.mockito.Mockito.never; +import static org.mockito.Mockito.times; +import static org.mockito.Mockito.verify; +import static org.mockito.Mockito.when; + +@RunWith(MockitoJUnitRunner.class) +public class MeteredProcessingLoggerTest { + private final static String sensorName = "sensorName"; + @Mock + private ProcessingLogger processingLogger; + @Mock + private ProcessingLogger.ErrorMessage errMessage; + @Mock + private Sensor sensor; + @Mock + private Metrics metrics; + + private MeteredProcessingLogger meteredProcessingLogger; + + @Before + public void setup() { + when(sensor.name()).thenReturn(sensorName); + meteredProcessingLogger = new MeteredProcessingLogger(processingLogger, metrics, sensor); + } + + @Test + public void shouldRecordMetric() { + // When: + meteredProcessingLogger.error(errMessage); + meteredProcessingLogger.error(errMessage); + + // Then: + verify(processingLogger, times(2)).error(errMessage); + verify(sensor, times(2)).record(); + } + + @Test + public void shouldRemoveSensorOnClose() { + // When: + meteredProcessingLogger.close(); + + // Then: + verify(metrics).removeSensor(sensorName); + verify(processingLogger).close(); + } + + @Test + public void shouldHandleNullMetricsAndSensor() { + // Given: + meteredProcessingLogger = new MeteredProcessingLogger(processingLogger, null, null); + + // When: + meteredProcessingLogger.error(errMessage); + meteredProcessingLogger.close(); + + // Then: + verify(processingLogger, times(1)).error(errMessage); + verify(sensor, never()).record(); + verify(metrics, never()).removeSensor(sensorName); + verify(processingLogger).close(); + } +} \ No newline at end of file diff --git a/ksqldb-common/src/test/java/io/confluent/ksql/logging/processing/ProcessingLoggerFactoryImplTest.java b/ksqldb-common/src/test/java/io/confluent/ksql/logging/processing/ProcessingLoggerFactoryImplTest.java deleted file mode 100644 index 60943d4b0912..000000000000 --- a/ksqldb-common/src/test/java/io/confluent/ksql/logging/processing/ProcessingLoggerFactoryImplTest.java +++ /dev/null @@ -1,80 +0,0 @@ -/* - * Copyright 2018 Confluent Inc. - * - * Licensed under the Confluent Community License (the "License"); you may not use - * this file except in compliance with the License. You may obtain a copy of the - * License at - * - * http://www.confluent.io/confluent-community-license - * - * Unless required by applicable law or agreed to in writing, software - * distributed under the License is distributed on an "AS IS" BASIS, WITHOUT - * WARRANTIES OF ANY KIND, either express or implied. See the License for the - * specific language governing permissions and limitations under the License. - */ - -package io.confluent.ksql.logging.processing; - -import static org.hamcrest.MatcherAssert.assertThat; -import static org.hamcrest.Matchers.equalTo; -import static org.hamcrest.Matchers.is; -import static org.mockito.ArgumentMatchers.anyString; -import static org.mockito.Mockito.verify; -import static org.mockito.Mockito.when; - -import com.google.common.collect.ImmutableList; -import io.confluent.common.logging.StructuredLogger; -import io.confluent.common.logging.StructuredLoggerFactory; -import java.util.Collection; -import java.util.function.BiFunction; -import org.junit.Before; -import org.junit.Test; -import org.junit.runner.RunWith; -import org.mockito.Mock; -import org.mockito.junit.MockitoJUnitRunner; - -@RunWith(MockitoJUnitRunner.class) -public class ProcessingLoggerFactoryImplTest { - @Mock - private StructuredLoggerFactory innerFactory; - @Mock - private StructuredLogger innerLogger; - @Mock - private ProcessingLogConfig config; - @Mock - private BiFunction loggerFactory; - @Mock - private ProcessingLogger logger; - - private final Collection loggers = ImmutableList.of("logger1", "logger2"); - - private ProcessingLoggerFactoryImpl factory; - - @Before - public void setup() { - when(innerFactory.getLogger(anyString())).thenReturn(innerLogger); - when(innerFactory.getLoggers()).thenReturn(loggers); - when(loggerFactory.apply(config, innerLogger)).thenReturn(logger); - factory = new ProcessingLoggerFactoryImpl(config, innerFactory, loggerFactory); - } - - @Test - public void shouldCreateLogger() { - // When: - final ProcessingLogger logger = factory.getLogger("foo.bar"); - - // Then: - assertThat(logger, is(this.logger)); - verify(innerFactory).getLogger("foo.bar"); - verify(loggerFactory).apply(config, innerLogger); - } - - @Test - public void shouldGetLoggers() { - // When: - final Collection loggers = factory.getLoggers(); - - // Then: - assertThat(loggers, equalTo(this.loggers)); - } -} \ No newline at end of file diff --git a/ksqldb-engine/src/main/java/io/confluent/ksql/engine/EngineContext.java b/ksqldb-engine/src/main/java/io/confluent/ksql/engine/EngineContext.java index 189a006db10c..82b42010bd57 100644 --- a/ksqldb-engine/src/main/java/io/confluent/ksql/engine/EngineContext.java +++ b/ksqldb-engine/src/main/java/io/confluent/ksql/engine/EngineContext.java @@ -132,7 +132,7 @@ synchronized EngineContext createSandbox(final ServiceContext serviceContext) { this.runtimeAssignor.rebuildAssignment(queryRegistry.getPersistentQueries().values()); return new EngineContext( SandboxedServiceContext.create(serviceContext), - processingLogContext, + ProcessingLogContext.create(), metaStore.copy(), queryIdGenerator.createSandbox(), new DefaultKsqlParser(), diff --git a/ksqldb-engine/src/main/java/io/confluent/ksql/engine/generic/GenericExpressionResolver.java b/ksqldb-engine/src/main/java/io/confluent/ksql/engine/generic/GenericExpressionResolver.java index 81e1a2efc864..a887e4300107 100644 --- a/ksqldb-engine/src/main/java/io/confluent/ksql/engine/generic/GenericExpressionResolver.java +++ b/ksqldb-engine/src/main/java/io/confluent/ksql/engine/generic/GenericExpressionResolver.java @@ -56,8 +56,16 @@ public class GenericExpressionResolver { private static final LogicalSchema NO_COLUMNS = LogicalSchema.builder().build(); private static final Supplier IGNORED_MSG = () -> ""; - private static final ProcessingLogger THROWING_LOGGER = errorMessage -> { - throw new KsqlException(((RecordProcessingError) errorMessage).getMessage()); + private static final ProcessingLogger THROWING_LOGGER = new ProcessingLogger() { + @Override + public void error(final ErrorMessage errorMessage) { + throw new KsqlException(((RecordProcessingError) errorMessage).getMessage()); + } + + @Override + public void close() { + // no-op + } }; private final SqlType fieldType; diff --git a/ksqldb-engine/src/main/java/io/confluent/ksql/function/udaf/max/MaxAggFunctionFactory.java b/ksqldb-engine/src/main/java/io/confluent/ksql/function/udaf/max/MaxAggFunctionFactory.java index 7fa8b1dcd121..0729e98ff608 100644 --- a/ksqldb-engine/src/main/java/io/confluent/ksql/function/udaf/max/MaxAggFunctionFactory.java +++ b/ksqldb-engine/src/main/java/io/confluent/ksql/function/udaf/max/MaxAggFunctionFactory.java @@ -52,6 +52,8 @@ public KsqlAggregateFunction createAggregateFunction( case DATE: case TIME: case TIMESTAMP: + case STRING: + case BYTES: return new MaxKudaf(FUNCTION_NAME, initArgs.udafIndex(), argSchema); default: throw new KsqlException( @@ -65,8 +67,8 @@ public KsqlAggregateFunction createAggregateFunction( } @Override - @SuppressFBWarnings(value = "EI_EXPOSE_REP", justification = "NUMERICAL_TIME is ImmutableList") + @SuppressFBWarnings(value = "EI_EXPOSE_REP", justification = "COMPARABLE_ARGS is ImmutableList") public List> supportedArgs() { - return NUMERICAL_TIME; + return COMPARABLE_ARGS; } } diff --git a/ksqldb-engine/src/main/java/io/confluent/ksql/function/udaf/min/MinAggFunctionFactory.java b/ksqldb-engine/src/main/java/io/confluent/ksql/function/udaf/min/MinAggFunctionFactory.java index 0445cde96e27..2b157f7d379f 100644 --- a/ksqldb-engine/src/main/java/io/confluent/ksql/function/udaf/min/MinAggFunctionFactory.java +++ b/ksqldb-engine/src/main/java/io/confluent/ksql/function/udaf/min/MinAggFunctionFactory.java @@ -51,6 +51,8 @@ public KsqlAggregateFunction createAggregateFunction( case DATE: case TIME: case TIMESTAMP: + case STRING: + case BYTES: return new MinKudaf(FUNCTION_NAME, initArgs.udafIndex(), argSchema); default: throw new KsqlException( @@ -64,8 +66,8 @@ public KsqlAggregateFunction createAggregateFunction( } @Override - @SuppressFBWarnings(value = "EI_EXPOSE_REP", justification = "NUMERICAL_TIME is ImmutableList") + @SuppressFBWarnings(value = "EI_EXPOSE_REP", justification = "COMPARABLE_ARGS is ImmutableList") public List> supportedArgs() { - return NUMERICAL_TIME; + return COMPARABLE_ARGS; } } diff --git a/ksqldb-engine/src/main/java/io/confluent/ksql/function/udaf/sum/DecimalSumKudaf.java b/ksqldb-engine/src/main/java/io/confluent/ksql/function/udaf/sum/DecimalSumKudaf.java index e6ab4033bfc1..09e08420b40a 100644 --- a/ksqldb-engine/src/main/java/io/confluent/ksql/function/udaf/sum/DecimalSumKudaf.java +++ b/ksqldb-engine/src/main/java/io/confluent/ksql/function/udaf/sum/DecimalSumKudaf.java @@ -18,6 +18,7 @@ import io.confluent.ksql.GenericKey; import io.confluent.ksql.execution.function.TableAggregationFunction; import io.confluent.ksql.function.BaseAggregateFunction; +import io.confluent.ksql.function.KsqlFunctionException; import io.confluent.ksql.function.ParameterInfo; import io.confluent.ksql.function.types.ParamTypes; import io.confluent.ksql.schema.ksql.types.SqlDecimal; @@ -32,6 +33,10 @@ public class DecimalSumKudaf implements TableAggregationFunction { private final MathContext context; + private final BigDecimal maxValue; + private final int precision; + private final int scale; + private final int digits; DecimalSumKudaf( final String functionName, @@ -48,6 +53,10 @@ public class DecimalSumKudaf "Computes the sum of decimal values for a key, resulting in a decimal with the same " + "precision and scale."); context = new MathContext(outputSchema.getPrecision()); + precision = outputSchema.getPrecision(); + scale = outputSchema.getScale(); + digits = outputSchema.getPrecision() - outputSchema.getScale(); + maxValue = BigDecimal.valueOf(Math.pow(10, digits)); } @Override @@ -56,7 +65,19 @@ public BigDecimal aggregate(final BigDecimal currentValue, final BigDecimal aggr return aggregateValue; } - return aggregateValue.add(currentValue, context); + final BigDecimal value = aggregateValue.add(currentValue, context); + + if (maxValue.compareTo(value.abs()) < 1) { + throw new KsqlFunctionException( + String.format("Numeric field overflow: A field with precision %d and scale %d " + + "must round to an absolute value less than 10^%d. Got %s", + precision, + scale, + digits, + value.toPlainString())); + } + + return value; } @Override diff --git a/ksqldb-engine/src/main/java/io/confluent/ksql/query/KsqlFunctionClassifier.java b/ksqldb-engine/src/main/java/io/confluent/ksql/query/KsqlFunctionClassifier.java new file mode 100644 index 000000000000..03ecfbcc945c --- /dev/null +++ b/ksqldb-engine/src/main/java/io/confluent/ksql/query/KsqlFunctionClassifier.java @@ -0,0 +1,58 @@ +/* + * Copyright 2022 Confluent Inc. + * + * Licensed under the Confluent Community License (the "License"; you may not use + * this file except in compliance with the License. You may obtain a copy of the + * License at + * + * http://www.confluent.io/confluent-community-license + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, WITHOUT + * WARRANTIES OF ANY KIND, either express or implied. See the License for the + * specific language governing permissions and limitations under the License. + */ + +package io.confluent.ksql.query; + +import io.confluent.ksql.function.KsqlFunctionException; +import io.confluent.ksql.query.QueryError.Type; +import java.util.Objects; +import org.apache.commons.lang3.exception.ExceptionUtils; +import org.apache.kafka.streams.errors.StreamsException; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +/** + * {@code KsqlFunctionClassifier} classifies ksql function exceptions as user error + */ +public class KsqlFunctionClassifier implements QueryErrorClassifier { + + private static final Logger LOG = LoggerFactory.getLogger(KsqlFunctionClassifier.class); + + private final String queryId; + + public KsqlFunctionClassifier(final String queryId) { + this.queryId = Objects.requireNonNull(queryId, "queryId"); + } + + @Override + public Type classify(final Throwable e) { + Type type = Type.UNKNOWN; + if (e instanceof KsqlFunctionException + || (e instanceof StreamsException + && ExceptionUtils.getRootCause(e) instanceof KsqlFunctionException)) { + type = Type.USER; + } + + if (type == Type.USER) { + LOG.info( + "Classified error as USER error based on invalid user input. Query ID: {} Exception: {}", + queryId, + e); + } + + return type; + } + +} diff --git a/ksqldb-engine/src/main/java/io/confluent/ksql/query/MissingSubjectClassifier.java b/ksqldb-engine/src/main/java/io/confluent/ksql/query/MissingSubjectClassifier.java new file mode 100644 index 000000000000..a155f8e3f31c --- /dev/null +++ b/ksqldb-engine/src/main/java/io/confluent/ksql/query/MissingSubjectClassifier.java @@ -0,0 +1,49 @@ +/* + * Copyright 2022 Confluent Inc. + * + * Licensed under the Confluent Community License (the "License"; you may not use + * this file except in compliance with the License. You may obtain a copy of the + * License at + * + * http://www.confluent.io/confluent-community-license + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, WITHOUT + * WARRANTIES OF ANY KIND, either express or implied. See the License for the + * specific language governing permissions and limitations under the License. + */ + +package io.confluent.ksql.query; + +import io.confluent.ksql.query.QueryError.Type; +import io.confluent.ksql.schema.registry.SchemaRegistryUtil; +import java.util.Objects; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +/** + * {@code MissingSubjectClassifier} classifies missing SR subjects exceptions as user error + */ +public class MissingSubjectClassifier implements QueryErrorClassifier { + private static final Logger LOG = LoggerFactory.getLogger(MissingSubjectClassifier.class); + + private final String queryId; + + public MissingSubjectClassifier(final String queryId) { + this.queryId = Objects.requireNonNull(queryId, "queryId"); + } + + @Override + public Type classify(final Throwable e) { + final Type type = SchemaRegistryUtil.isSubjectNotFoundErrorCode(e) ? Type.USER : Type.UNKNOWN; + + if (type == Type.USER) { + LOG.info( + "Classified error as USER error based on missing SR subject. Query ID: {} Exception: {}", + queryId, + e); + } + + return type; + } +} diff --git a/ksqldb-engine/src/main/java/io/confluent/ksql/query/QueryBuilder.java b/ksqldb-engine/src/main/java/io/confluent/ksql/query/QueryBuilder.java index 53b14a222f10..d7139083f301 100644 --- a/ksqldb-engine/src/main/java/io/confluent/ksql/query/QueryBuilder.java +++ b/ksqldb-engine/src/main/java/io/confluent/ksql/query/QueryBuilder.java @@ -1,5 +1,5 @@ /* - * Copyright 2019 Confluent Inc. + * Copyright 2022 Confluent Inc. * * Licensed under the Confluent Community License (the "License"); you may not use * this file except in compliance with the License. You may obtain a copy of the @@ -61,6 +61,7 @@ import io.confluent.ksql.util.KsqlConfig; import io.confluent.ksql.util.KsqlConstants; import io.confluent.ksql.util.KsqlException; +import io.confluent.ksql.util.MetricsTagsUtil; import io.confluent.ksql.util.PersistentQueryMetadata; import io.confluent.ksql.util.PersistentQueryMetadataImpl; import io.confluent.ksql.util.PushQueryMetadata.ResultType; @@ -74,6 +75,7 @@ import io.confluent.ksql.util.TransientQueryMetadata; import io.vertx.core.impl.ConcurrentHashSet; import java.util.Arrays; +import java.util.Collections; import java.util.HashMap; import java.util.LinkedList; import java.util.List; @@ -441,23 +443,14 @@ PersistentQueryMetadata buildPersistentQueryInSharedRuntime( ); final RuntimeBuildContext runtimeBuildContext = buildContext( - applicationId, - queryId, - namedTopologyBuilder + applicationId, + queryId, + namedTopologyBuilder ); final Object result = buildQueryImplementation(physicalPlan, runtimeBuildContext); final NamedTopology topology = namedTopologyBuilder.build(); - final Optional - materializationProviderBuilder = getMaterializationInfo(result).map(info -> - materializationProviderBuilderFactory.materializationProviderBuilder( - info, - querySchema, - keyFormat, - queryOverrides, - applicationId, - queryId.toString() - )); + final Optional materializationInfo = getMaterializationInfo(result); final Optional scalablePushRegistry = applyScalablePushProcessor( querySchema.logicalSchema(), @@ -483,13 +476,13 @@ PersistentQueryMetadata buildPersistentQueryInSharedRuntime( runtimeBuildContext.getSchemas(), config.getOverrides(), queryId, - materializationProviderBuilder, + materializationInfo, + materializationProviderBuilderFactory, physicalPlan, getUncaughtExceptionProcessingLogger(queryId), sinkDataSource, listener, - queryOverrides, - scalablePushRegistry, + scalablePushRegistry, (streamsRuntime) -> getNamedTopology( streamsRuntime, queryId, @@ -497,6 +490,7 @@ PersistentQueryMetadata buildPersistentQueryInSharedRuntime( queryOverrides, physicalPlan ), + keyFormat, metricCollectors.getMetrics(), ksqlConfig.getStringAsMap(KsqlConfig.KSQL_CUSTOM_METRICS_TAGS) ); @@ -541,12 +535,18 @@ public static Map buildStreamsProperties( newStreamsProperties.put(StreamsConfig.APPLICATION_ID_CONFIG, applicationId); // get logger - final ProcessingLogger logger; + final String id; if (queryId.isPresent()) { - logger = processingLogContext.getLoggerFactory().getLogger(queryId.get().toString()); + id = queryId.get().toString(); } else { - logger = processingLogContext.getLoggerFactory().getLogger(applicationId); + id = applicationId; } + + final ProcessingLogger logger = + processingLogContext + .getLoggerFactory() + .getLogger(id, MetricsTagsUtil.getMetricsTagsWithQueryId(id, Collections.emptyMap())); + newStreamsProperties.put( ProductionExceptionHandlerUtil.KSQL_PRODUCTION_ERROR_LOGGER, logger); @@ -649,18 +649,24 @@ private QueryErrorClassifier getConfiguredQueryErrorClassifier( final String applicationId ) { final QueryErrorClassifier userErrorClassifiers = new MissingTopicClassifier(applicationId) - .and(new AuthorizationClassifier(applicationId)); + .and(new AuthorizationClassifier(applicationId)) + .and(new KsqlFunctionClassifier(applicationId)) + .and(new MissingSubjectClassifier(applicationId)) + .and(new SchemaAuthorizationClassifier(applicationId)); return buildConfiguredClassifiers(ksqlConfig, applicationId) .map(userErrorClassifiers::and) .orElse(userErrorClassifiers); } - private ProcessingLogger getUncaughtExceptionProcessingLogger(final QueryId queryId) { + private ProcessingLogger getUncaughtExceptionProcessingLogger( + final QueryId queryId + ) { final QueryContext.Stacker stacker = new QueryContext.Stacker() .push(KSQL_THREAD_EXCEPTION_UNCAUGHT_LOGGER); return processingLogContext.getLoggerFactory().getLogger( - QueryLoggerUtil.queryLoggerName(queryId, stacker.getQueryContext())); + QueryLoggerUtil.queryLoggerName(queryId, stacker.getQueryContext()), + MetricsTagsUtil.getMetricsTagsWithQueryId(queryId.toString(), Collections.emptyMap())); } private static TransientQueryQueue buildTransientQueryQueue( @@ -702,9 +708,10 @@ private static Object buildQueryImplementation( } private RuntimeBuildContext buildContext( - final String applicationId, - final QueryId queryId, - final StreamsBuilder streamsBuilder) { + final String applicationId, + final QueryId queryId, + final StreamsBuilder streamsBuilder + ) { return RuntimeBuildContext.of( streamsBuilder, config.getConfig(true), diff --git a/ksqldb-engine/src/main/java/io/confluent/ksql/query/SchemaAuthorizationClassifier.java b/ksqldb-engine/src/main/java/io/confluent/ksql/query/SchemaAuthorizationClassifier.java new file mode 100644 index 000000000000..eb15b8b5e2ad --- /dev/null +++ b/ksqldb-engine/src/main/java/io/confluent/ksql/query/SchemaAuthorizationClassifier.java @@ -0,0 +1,51 @@ +/* + * Copyright 2022 Confluent Inc. + * + * Licensed under the Confluent Community License (the "License"; you may not use + * this file except in compliance with the License. You may obtain a copy of the + * License at + * + * http://www.confluent.io/confluent-community-license + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, WITHOUT + * WARRANTIES OF ANY KIND, either express or implied. See the License for the + * specific language governing permissions and limitations under the License. + */ + +package io.confluent.ksql.query; + +import io.confluent.ksql.query.QueryError.Type; +import io.confluent.ksql.schema.registry.SchemaRegistryUtil; +import java.util.Objects; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +/** + * {@code SchemaAuthorizationClassifier} classifies authorization SR subjects exceptions + * as user error + */ +public class SchemaAuthorizationClassifier implements QueryErrorClassifier { + private static final Logger LOG = LoggerFactory.getLogger(SchemaAuthorizationClassifier.class); + + private final String queryId; + + public SchemaAuthorizationClassifier(final String queryId) { + this.queryId = Objects.requireNonNull(queryId, "queryId"); + } + + @Override + public Type classify(final Throwable e) { + final Type type = SchemaRegistryUtil.isAuthErrorCode(e) ? Type.USER : Type.UNKNOWN; + + if (type == Type.USER) { + LOG.info( + "Classified error as USER error based on missing SR subject access rights. " + + "Query ID: {} Exception: {}", + queryId, + e); + } + + return type; + } +} diff --git a/ksqldb-engine/src/main/java/io/confluent/ksql/schema/ksql/inference/DefaultSchemaInjector.java b/ksqldb-engine/src/main/java/io/confluent/ksql/schema/ksql/inference/DefaultSchemaInjector.java index 972b37aeb49f..8c90bea456f4 100644 --- a/ksqldb-engine/src/main/java/io/confluent/ksql/schema/ksql/inference/DefaultSchemaInjector.java +++ b/ksqldb-engine/src/main/java/io/confluent/ksql/schema/ksql/inference/DefaultSchemaInjector.java @@ -514,11 +514,13 @@ private static void throwOnMultiSchemaDefinitions( } } + // CHECKSTYLE_RULES.OFF: CyclomaticComplexity private static CreateSource addSchemaFields( final ConfiguredStatement preparedStatement, final Optional keySchema, final Optional valueSchema ) { + // CHECKSTYLE_RULES.ON: CyclomaticComplexity final TableElements elements = buildElements(preparedStatement, keySchema, valueSchema); final CreateSource statement = preparedStatement.getStatement(); @@ -541,13 +543,20 @@ private static CreateSource addSchemaFields( throwOnMultiSchemaDefinitions(valueSchema.get().rawSchema, valueFormat, false); } - // Only populate key and value schema names when schema ids are explicitly provided - if (properties.getKeySchemaId().isPresent() && keySchema.isPresent()) { + // Only populate key and value schema names when schema ids or full schema names are + // explicitly provided + + if (properties.getKeySchemaFullName().isPresent()) { + keySchemaName = properties.getKeySchemaFullName(); + } else if (properties.getKeySchemaId().isPresent() && keySchema.isPresent()) { keySchemaName = Optional.ofNullable(keySchema.get().rawSchema.name()); } else { keySchemaName = Optional.empty(); } - if (properties.getValueSchemaId().isPresent() && valueSchema.isPresent()) { + + if (properties.getValueSchemaFullName().isPresent()) { + valueSchemaName = properties.getValueSchemaFullName(); + } else if (properties.getValueSchemaId().isPresent() && valueSchema.isPresent()) { valueSchemaName = Optional.ofNullable(valueSchema.get().rawSchema.name()); } else { valueSchemaName = Optional.empty(); diff --git a/ksqldb-engine/src/main/java/io/confluent/ksql/util/BinPackedPersistentQueryMetadataImpl.java b/ksqldb-engine/src/main/java/io/confluent/ksql/util/BinPackedPersistentQueryMetadataImpl.java index cfabc91b646b..8cf77965bf5b 100644 --- a/ksqldb-engine/src/main/java/io/confluent/ksql/util/BinPackedPersistentQueryMetadataImpl.java +++ b/ksqldb-engine/src/main/java/io/confluent/ksql/util/BinPackedPersistentQueryMetadataImpl.java @@ -24,10 +24,10 @@ import edu.umd.cs.findbugs.annotations.SuppressFBWarnings; import io.confluent.ksql.execution.context.QueryContext; import io.confluent.ksql.execution.ddl.commands.KsqlTopic; +import io.confluent.ksql.execution.materialization.MaterializationInfo; import io.confluent.ksql.execution.plan.ExecutionStep; import io.confluent.ksql.execution.scalablepush.ScalablePushRegistry; import io.confluent.ksql.execution.streams.materialization.Materialization; -import io.confluent.ksql.execution.streams.materialization.MaterializationProvider; import io.confluent.ksql.logging.processing.ProcessingLogger; import io.confluent.ksql.metastore.model.DataSource; import io.confluent.ksql.name.SourceName; @@ -38,6 +38,7 @@ import io.confluent.ksql.schema.ksql.LogicalSchema; import io.confluent.ksql.schema.ksql.PhysicalSchema; import io.confluent.ksql.schema.query.QuerySchemas; +import io.confluent.ksql.serde.KeyFormat; import io.confluent.ksql.util.QueryMetadataImpl.TimeBoundedQueue; import java.util.Collection; import java.util.List; @@ -69,7 +70,9 @@ public class BinPackedPersistentQueryMetadataImpl implements PersistentQueryMeta private final String statementString; private final String executionPlan; private final String applicationId; - private final NamedTopology topology; + private final Optional materializationInfo; + private final KeyFormat keyFormat; + private NamedTopology topology; private final SharedKafkaStreamsRuntime sharedKafkaStreamsRuntime; private final QuerySchemas schemas; private final ImmutableMap overriddenProperties; @@ -85,9 +88,8 @@ public class BinPackedPersistentQueryMetadataImpl implements PersistentQueryMeta private final Optional metrics; private final Optional restartSensor; - private final Optional - materializationProviderBuilder; - private final Optional materializationProvider; + private final MaterializationProviderBuilderFactory + materializationProviderBuilderFactory; private final Optional scalablePushRegistry; public boolean everStarted = false; private boolean corruptionCommandTopic = false; @@ -107,15 +109,15 @@ public BinPackedPersistentQueryMetadataImpl( final QuerySchemas schemas, final Map overriddenProperties, final QueryId queryId, - final Optional - materializationProviderBuilder, + final Optional materializationInfo, + final MaterializationProviderBuilderFactory materializationProviderBuilderFactory, final ExecutionStep physicalPlan, final ProcessingLogger processingLogger, final Optional sinkDataSource, final Listener listener, - final Map streamsProperties, final Optional scalablePushRegistry, final Function namedTopologyBuilder, + final KeyFormat keyFormat, final Metrics metrics, final Map metricsTags) { // CHECKSTYLE_RULES.ON: ParameterNumberCheck @@ -136,20 +138,17 @@ public BinPackedPersistentQueryMetadataImpl( this.processingLogger = requireNonNull(processingLogger, "processingLogger"); this.physicalPlan = requireNonNull(physicalPlan, "physicalPlan"); this.resultSchema = requireNonNull(schema, "schema"); - this.materializationProviderBuilder = - requireNonNull(materializationProviderBuilder, "materializationProviderBuilder"); + this.materializationProviderBuilderFactory = requireNonNull( + materializationProviderBuilderFactory, "materializationProviderBuilderFactory"); + this.materializationInfo = requireNonNull(materializationInfo, "materializationInfo"); this.listener = new QueryListenerWrapper(listener, scalablePushRegistry); this.namedTopologyBuilder = requireNonNull(namedTopologyBuilder, "namedTopologyBuilder"); this.queryErrors = sharedKafkaStreamsRuntime.getNewQueryErrorQueue(); - this.materializationProvider = materializationProviderBuilder - .flatMap(builder -> builder.apply( - this.sharedKafkaStreamsRuntime.getKafkaStreams(), - topology - )); this.scalablePushRegistry = requireNonNull(scalablePushRegistry, "scalablePushRegistry"); this.metrics = Optional.of(metrics); this.restartSensor = Optional.of( QueryMetricsUtil.createQueryRestartMetricSensor(queryId.toString(), metricsTags, metrics)); + this.keyFormat = requireNonNull(keyFormat, "keyFormat"); } // for creating sandbox instances @@ -172,14 +171,15 @@ public BinPackedPersistentQueryMetadataImpl( this.processingLogger = original.processingLogger; this.physicalPlan = original.getPhysicalPlan(); this.resultSchema = original.resultSchema; - this.materializationProviderBuilder = original.materializationProviderBuilder; + this.materializationProviderBuilderFactory = original.materializationProviderBuilderFactory; + this.materializationInfo = original.materializationInfo; this.listener = requireNonNull(listener, "listener"); this.queryErrors = sharedKafkaStreamsRuntime.getNewQueryErrorQueue(); - this.materializationProvider = original.materializationProvider; this.scalablePushRegistry = original.scalablePushRegistry; this.namedTopologyBuilder = original.namedTopologyBuilder; this.metrics = Optional.empty(); this.restartSensor = Optional.empty(); + this.keyFormat = original.keyFormat; } @Override @@ -231,7 +231,19 @@ public ProcessingLogger getProcessingLogger() { public Optional getMaterialization( final QueryId queryId, final QueryContext.Stacker contextStacker) { - return materializationProvider.map(builder -> builder.build(queryId, contextStacker)); + return this.materializationInfo.map(info -> + materializationProviderBuilderFactory.materializationProviderBuilder( + info, + resultSchema, + keyFormat, + getStreamsProperties(), + applicationId, + this.queryId.toString() + ) + ).flatMap(builder -> builder.apply( + sharedKafkaStreamsRuntime.getKafkaStreams(), + topology) + ).map(builder -> builder.build(queryId, contextStacker)); } @Override @@ -313,6 +325,10 @@ public NamedTopology getTopologyCopy(final SharedKafkaStreamsRuntime builder) { return namedTopologyBuilder.apply(builder); } + public void updateTopology(final NamedTopology topology) { + this.topology = topology; + } + @Override public Map> getAllLocalStorePartitionLags() { return sharedKafkaStreamsRuntime.getAllLocalStorePartitionLagsForQuery(queryId); diff --git a/ksqldb-engine/src/main/java/io/confluent/ksql/util/QueryMetadataImpl.java b/ksqldb-engine/src/main/java/io/confluent/ksql/util/QueryMetadataImpl.java index 111d63be9e43..3f545ecbab53 100644 --- a/ksqldb-engine/src/main/java/io/confluent/ksql/util/QueryMetadataImpl.java +++ b/ksqldb-engine/src/main/java/io/confluent/ksql/util/QueryMetadataImpl.java @@ -489,17 +489,13 @@ public void backOff(final String threadName) { } sensor.ifPresent(Sensor::record); - if (numRetries.containsKey(threadName)) { - numRetries.put(threadName, numRetries.get(threadName) + 1); - } else { - numRetries.put(threadName, 1); - } + final int retries = numRetries.merge(threadName, 1, Integer::sum); LOG.info( "Restarting query {} thread {} (attempt #{})", queryId, threadName, - numRetries.get(threadName)); + retries); // Math.max() prevents overflow if now is Long.MAX_VALUE (found just in tests) this.expiryTimeMs = Math.max(now, now + waitingTimeMs); diff --git a/ksqldb-engine/src/main/java/io/confluent/ksql/util/QueryMetricsUtil.java b/ksqldb-engine/src/main/java/io/confluent/ksql/util/QueryMetricsUtil.java index c7a761e4ab51..b221419bc067 100644 --- a/ksqldb-engine/src/main/java/io/confluent/ksql/util/QueryMetricsUtil.java +++ b/ksqldb-engine/src/main/java/io/confluent/ksql/util/QueryMetricsUtil.java @@ -34,7 +34,7 @@ public static Sensor createQueryRestartMetricSensor( final Metrics metrics ) { final Map customMetricsTagsForQuery = - MetricsTagsUtil.getCustomMetricsTagsForQuery(queryId, metricsTags); + MetricsTagsUtil.getMetricsTagsWithQueryId(queryId, metricsTags); final MetricName restartMetricName = metrics.metricName( QueryMetadataImpl.QUERY_RESTART_METRIC_NAME, QueryMetadataImpl.QUERY_RESTART_METRIC_GROUP_NAME, diff --git a/ksqldb-engine/src/main/java/io/confluent/ksql/util/SharedKafkaStreamsRuntimeImpl.java b/ksqldb-engine/src/main/java/io/confluent/ksql/util/SharedKafkaStreamsRuntimeImpl.java index 438b29cdb562..f934787bd36f 100644 --- a/ksqldb-engine/src/main/java/io/confluent/ksql/util/SharedKafkaStreamsRuntimeImpl.java +++ b/ksqldb-engine/src/main/java/io/confluent/ksql/util/SharedKafkaStreamsRuntimeImpl.java @@ -33,6 +33,7 @@ import org.apache.kafka.common.metrics.Metrics; import org.apache.kafka.streams.KafkaStreams; import org.apache.kafka.streams.KafkaStreams.StateListener; +import org.apache.kafka.streams.StreamsConfig; import org.apache.kafka.streams.errors.StreamsException; import org.apache.kafka.streams.errors.StreamsUncaughtExceptionHandler; import org.apache.kafka.streams.processor.TaskId; @@ -130,6 +131,7 @@ public StreamsUncaughtExceptionHandler.StreamThreadExceptionResponse uncaughtHan if (queryInError.getSensor().isPresent()) { queryInError.getSensor().get().record(); } + System.out.println("here"); log.error(String.format( "Unhandled query exception caught in streams thread %s for query %s. (%s)", Thread.currentThread().getName(), @@ -144,6 +146,7 @@ public StreamsUncaughtExceptionHandler.StreamThreadExceptionResponse uncaughtHan query.getSensor().get().record(); } } + System.out.println("also here"); log.error(String.format( "Unhandled runtime exception caught in streams thread %s. (%s)", Thread.currentThread().getName(), @@ -258,6 +261,9 @@ public void start(final QueryId queryId) { @Override public void overrideStreamsProperties(final Map newProperties) { + newProperties.put(StreamsConfig.APPLICATION_SERVER_CONFIG, + streamsProperties.get(StreamsConfig.APPLICATION_SERVER_CONFIG)); + //The application server should not be overwritten streamsProperties = ImmutableMap.copyOf(newProperties); } @@ -273,7 +279,8 @@ public void restartStreamsRuntime() { for (final NamedTopology topology : liveTopologies) { final BinPackedPersistentQueryMetadataImpl query = collocatedQueries .get(new QueryId(topology.name())); - kafkaStreamsNamedTopologyWrapper.addNamedTopology(query.getTopologyCopy(this)); + query.updateTopology(query.getTopologyCopy(this)); + kafkaStreamsNamedTopologyWrapper.addNamedTopology(query.getTopology()); } setupAndStartKafkaStreams(kafkaStreamsNamedTopologyWrapper); } diff --git a/ksqldb-engine/src/test/java/io/confluent/ksql/function/udaf/max/BytesMaxKudafTest.java b/ksqldb-engine/src/test/java/io/confluent/ksql/function/udaf/max/BytesMaxKudafTest.java new file mode 100644 index 000000000000..499c2549c8e1 --- /dev/null +++ b/ksqldb-engine/src/test/java/io/confluent/ksql/function/udaf/max/BytesMaxKudafTest.java @@ -0,0 +1,130 @@ +/* + * Copyright 2022 Confluent Inc. + * + * Licensed under the Confluent Community License (the "License"); you may not use + * this file except in compliance with the License. You may obtain a copy of the + * License at + * + * http://www.confluent.io/confluent-community-license + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, WITHOUT + * WARRANTIES OF ANY KIND, either express or implied. See the License for the + * specific language governing permissions and limitations under the License. + */ + +package io.confluent.ksql.function.udaf.max; + +import static org.hamcrest.CoreMatchers.equalTo; +import static org.hamcrest.CoreMatchers.instanceOf; +import static org.hamcrest.MatcherAssert.assertThat; + +import io.confluent.ksql.GenericKey; +import io.confluent.ksql.function.AggregateFunctionInitArguments; +import io.confluent.ksql.function.KsqlAggregateFunction; +import io.confluent.ksql.function.udf.string.FromBytes; +import io.confluent.ksql.function.udf.string.ToBytes; +import io.confluent.ksql.schema.ksql.SqlArgument; +import io.confluent.ksql.schema.ksql.types.SqlTypes; +import io.confluent.ksql.util.BytesUtils.Encoding; +import java.nio.ByteBuffer; +import java.util.Collections; +import org.apache.kafka.streams.kstream.Merger; +import org.junit.Before; +import org.junit.Test; + +public class BytesMaxKudafTest { + + private ToBytes toBytesUDF; + private FromBytes fromBytesUDF; + + @Before + public void setUp() { + toBytesUDF = new ToBytes(); + fromBytesUDF = new FromBytes(); + } + + @Test + public void shouldFindCorrectMax() { + final MaxKudaf bytesMaxKudaf = getMaxComparableKudaf(); + final String[] values = new String[]{"C", "F", "B", "E", "A", "D", "B"}; + String currentMax = "A"; + for (final String val : values) { + currentMax = fromBytesUDF.fromBytes( + bytesMaxKudaf.aggregate( + toBytesUDF.toBytes(val, Encoding.ASCII.toString()), + toBytesUDF.toBytes(currentMax, Encoding.ASCII.toString())), + Encoding.ASCII.toString()); + } + assertThat("F", equalTo(currentMax)); + } + + @Test + public void shouldHandleNull() { + final MaxKudaf bytesMaxKudaf = getMaxComparableKudaf(); + final String[] values = new String[]{"C", "F", "B", "E", "A", "D", "B"}; + String currentMax = null; + + // null before any aggregation + currentMax = fromBytesUDF.fromBytes( + bytesMaxKudaf.aggregate( + null, + toBytesUDF.toBytes(currentMax, Encoding.ASCII.toString())), + Encoding.ASCII.toString()); + assertThat(null, equalTo(currentMax)); + + // now send each value to aggregation and verify + for (final String val : values) { + currentMax = fromBytesUDF.fromBytes( + bytesMaxKudaf.aggregate( + toBytesUDF.toBytes(val, Encoding.ASCII.toString()), + toBytesUDF.toBytes(currentMax, Encoding.ASCII.toString())), + Encoding.ASCII.toString()); + } + assertThat("F", equalTo(currentMax)); + + // null should not impact result + currentMax = fromBytesUDF.fromBytes( + bytesMaxKudaf.aggregate( + null, + toBytesUDF.toBytes(currentMax, Encoding.ASCII.toString())), + Encoding.ASCII.toString()); + assertThat("F", equalTo(currentMax)); + } + + @Test + public void shouldFindCorrectMaxForMerge() { + final MaxKudaf bytesMaxKudaf = getMaxComparableKudaf(); + final Merger merger = bytesMaxKudaf.getMerger(); + final String mergeResult1 = fromBytesUDF.fromBytes( + merger.apply( + null, + toBytesUDF.toBytes("B", Encoding.ASCII.toString()), + toBytesUDF.toBytes("D", Encoding.ASCII.toString())), + Encoding.ASCII.toString()); + assertThat(mergeResult1, equalTo("D")); + final String mergeResult2 = fromBytesUDF.fromBytes( + merger.apply( + null, + toBytesUDF.toBytes("P", Encoding.ASCII.toString()), + toBytesUDF.toBytes("F", Encoding.ASCII.toString())), + Encoding.ASCII.toString()); + assertThat(mergeResult2, equalTo("P")); + final String mergeResult3 = fromBytesUDF.fromBytes( + merger.apply( + null, + toBytesUDF.toBytes("A", Encoding.ASCII.toString()), + toBytesUDF.toBytes("K", Encoding.ASCII.toString())), + Encoding.ASCII.toString()); + assertThat(mergeResult3, equalTo("K")); + } + + private MaxKudaf getMaxComparableKudaf() { + final KsqlAggregateFunction aggregateFunction = new MaxAggFunctionFactory() + .createAggregateFunction(Collections.singletonList(SqlArgument.of(SqlTypes.BYTES)), + AggregateFunctionInitArguments.EMPTY_ARGS); + assertThat(aggregateFunction, instanceOf(MaxKudaf.class)); + return (MaxKudaf) aggregateFunction; + } + +} diff --git a/ksqldb-engine/src/test/java/io/confluent/ksql/function/udaf/max/StringMaxKudafTest.java b/ksqldb-engine/src/test/java/io/confluent/ksql/function/udaf/max/StringMaxKudafTest.java new file mode 100644 index 000000000000..d07a0b37c613 --- /dev/null +++ b/ksqldb-engine/src/test/java/io/confluent/ksql/function/udaf/max/StringMaxKudafTest.java @@ -0,0 +1,85 @@ +/* + * Copyright 2022 Confluent Inc. + * + * Licensed under the Confluent Community License (the "License"); you may not use + * this file except in compliance with the License. You may obtain a copy of the + * License at + * + * http://www.confluent.io/confluent-community-license + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, WITHOUT + * WARRANTIES OF ANY KIND, either express or implied. See the License for the + * specific language governing permissions and limitations under the License. + */ + +package io.confluent.ksql.function.udaf.max; + +import static org.hamcrest.CoreMatchers.equalTo; +import static org.hamcrest.CoreMatchers.instanceOf; +import static org.hamcrest.MatcherAssert.assertThat; + +import io.confluent.ksql.GenericKey; +import io.confluent.ksql.function.AggregateFunctionInitArguments; +import io.confluent.ksql.function.KsqlAggregateFunction; +import io.confluent.ksql.schema.ksql.SqlArgument; +import io.confluent.ksql.schema.ksql.types.SqlTypes; +import java.util.Collections; +import org.apache.kafka.streams.kstream.Merger; +import org.junit.Test; + +public class StringMaxKudafTest { + + @Test + public void shouldFindCorrectMax() { + final MaxKudaf stringMaxKudaf = getMaxComparableKudaf(); + final String[] values = new String[]{"C", "F", "B", "E", "A", "D", "B"}; + String currentMax = "A"; + for (final String val : values) { + currentMax = stringMaxKudaf.aggregate(val, currentMax); + } + assertThat("F", equalTo(currentMax)); + } + + @Test + public void shouldHandleNull() { + final MaxKudaf stringMaxKudaf = getMaxComparableKudaf(); + final String[] values = new String[]{"C", "F", "B", "E", "A", "D", "B"}; + String currentMax = null; + + // null before any aggregation + currentMax = stringMaxKudaf.aggregate(null, currentMax); + assertThat(null, equalTo(currentMax)); + + // now send each value to aggregation and verify + for (final String val : values) { + currentMax = stringMaxKudaf.aggregate(val, currentMax); + } + assertThat("F", equalTo(currentMax)); + + // null should not impact result + currentMax = stringMaxKudaf.aggregate(null, currentMax); + assertThat("F", equalTo(currentMax)); + } + + @Test + public void shouldFindCorrectMaxForMerge() { + final MaxKudaf stringMaxKudaf = getMaxComparableKudaf(); + final Merger merger = stringMaxKudaf.getMerger(); + final String mergeResult1 = merger.apply(null, "B", "D"); + assertThat(mergeResult1, equalTo("D")); + final String mergeResult2 = merger.apply(null, "P", "F"); + assertThat(mergeResult2, equalTo("P")); + final String mergeResult3 = merger.apply(null, "A", "K"); + assertThat(mergeResult3, equalTo("K")); + } + + private MaxKudaf getMaxComparableKudaf() { + final KsqlAggregateFunction aggregateFunction = new MaxAggFunctionFactory() + .createAggregateFunction(Collections.singletonList(SqlArgument.of(SqlTypes.STRING)), + AggregateFunctionInitArguments.EMPTY_ARGS); + assertThat(aggregateFunction, instanceOf(MaxKudaf.class)); + return (MaxKudaf) aggregateFunction; + } + +} diff --git a/ksqldb-engine/src/test/java/io/confluent/ksql/function/udaf/min/BytesMinKudafTest.java b/ksqldb-engine/src/test/java/io/confluent/ksql/function/udaf/min/BytesMinKudafTest.java new file mode 100644 index 000000000000..ab1b2573ceb3 --- /dev/null +++ b/ksqldb-engine/src/test/java/io/confluent/ksql/function/udaf/min/BytesMinKudafTest.java @@ -0,0 +1,130 @@ +/* + * Copyright 2022 Confluent Inc. + * + * Licensed under the Confluent Community License (the "License"); you may not use + * this file except in compliance with the License. You may obtain a copy of the + * License at + * + * http://www.confluent.io/confluent-community-license + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, WITHOUT + * WARRANTIES OF ANY KIND, either express or implied. See the License for the + * specific language governing permissions and limitations under the License. + */ + +package io.confluent.ksql.function.udaf.min; + +import static org.hamcrest.CoreMatchers.equalTo; +import static org.hamcrest.CoreMatchers.instanceOf; +import static org.hamcrest.MatcherAssert.assertThat; + +import io.confluent.ksql.GenericKey; +import io.confluent.ksql.function.AggregateFunctionInitArguments; +import io.confluent.ksql.function.KsqlAggregateFunction; +import io.confluent.ksql.function.udf.string.FromBytes; +import io.confluent.ksql.function.udf.string.ToBytes; +import io.confluent.ksql.schema.ksql.SqlArgument; +import io.confluent.ksql.schema.ksql.types.SqlTypes; +import io.confluent.ksql.util.BytesUtils.Encoding; +import java.nio.ByteBuffer; +import java.util.Collections; +import org.apache.kafka.streams.kstream.Merger; +import org.junit.Before; +import org.junit.Test; + +public class BytesMinKudafTest { + + private ToBytes toBytesUDF; + private FromBytes fromBytesUDF; + + @Before + public void setUp() { + toBytesUDF = new ToBytes(); + fromBytesUDF = new FromBytes(); + } + + @Test + public void shouldFindCorrectMin() { + final MinKudaf bytesMinKudaf = getBytesMinKudaf(); + final String[] values = new String[]{"C", "F", "B", "E", "A", "D", "B"}; + String currentMin = "D"; + for (final String val : values) { + currentMin = fromBytesUDF.fromBytes( + bytesMinKudaf.aggregate( + toBytesUDF.toBytes(val, Encoding.ASCII.toString()), + toBytesUDF.toBytes(currentMin, Encoding.ASCII.toString())), + Encoding.ASCII.toString()); + } + assertThat("A", equalTo(currentMin)); + } + + @Test + public void shouldHandleNull() { + final MinKudaf bytesMinKudaf = getBytesMinKudaf(); + final String[] values = new String[]{"C", "F", "B", "E", "A", "D", "B"}; + String currentMin = null; + + // null before any aggregation + currentMin = fromBytesUDF.fromBytes( + bytesMinKudaf.aggregate( + null, + toBytesUDF.toBytes(currentMin, Encoding.ASCII.toString())), + Encoding.ASCII.toString()); + assertThat(null, equalTo(currentMin)); + + // now send each value to aggregation and verify + for (final String val : values) { + currentMin = fromBytesUDF.fromBytes( + bytesMinKudaf.aggregate( + toBytesUDF.toBytes(val, Encoding.ASCII.toString()), + toBytesUDF.toBytes(currentMin, Encoding.ASCII.toString())), + Encoding.ASCII.toString()); + } + assertThat("A", equalTo(currentMin)); + + // null should not impact result + currentMin = fromBytesUDF.fromBytes( + bytesMinKudaf.aggregate( + null, + toBytesUDF.toBytes(currentMin, Encoding.ASCII.toString())), + Encoding.ASCII.toString()); + assertThat("A", equalTo(currentMin)); + } + + @Test + public void shouldFindCorrectMinForMerge() { + final MinKudaf bytesMinKudaf = getBytesMinKudaf(); + final Merger merger = bytesMinKudaf.getMerger(); + final String mergeResult1 = fromBytesUDF.fromBytes( + merger.apply( + null, + toBytesUDF.toBytes("B", Encoding.ASCII.toString()), + toBytesUDF.toBytes("D", Encoding.ASCII.toString())), + Encoding.ASCII.toString()); + assertThat(mergeResult1, equalTo("B")); + final String mergeResult2 = fromBytesUDF.fromBytes( + merger.apply( + null, + toBytesUDF.toBytes("P", Encoding.ASCII.toString()), + toBytesUDF.toBytes("F", Encoding.ASCII.toString())), + Encoding.ASCII.toString()); + assertThat(mergeResult2, equalTo("F")); + final String mergeResult3 = fromBytesUDF.fromBytes( + merger.apply( + null, + toBytesUDF.toBytes("A", Encoding.ASCII.toString()), + toBytesUDF.toBytes("K", Encoding.ASCII.toString())), + Encoding.ASCII.toString()); + assertThat(mergeResult3, equalTo("A")); + } + + private MinKudaf getBytesMinKudaf() { + final KsqlAggregateFunction aggregateFunction = new MinAggFunctionFactory() + .createAggregateFunction(Collections.singletonList(SqlArgument.of(SqlTypes.BYTES)), + AggregateFunctionInitArguments.EMPTY_ARGS); + assertThat(aggregateFunction, instanceOf(MinKudaf.class)); + return (MinKudaf) aggregateFunction; + } + +} diff --git a/ksqldb-engine/src/test/java/io/confluent/ksql/function/udaf/min/StringMinKudafTest.java b/ksqldb-engine/src/test/java/io/confluent/ksql/function/udaf/min/StringMinKudafTest.java new file mode 100644 index 000000000000..a2709adc9b1f --- /dev/null +++ b/ksqldb-engine/src/test/java/io/confluent/ksql/function/udaf/min/StringMinKudafTest.java @@ -0,0 +1,85 @@ +/* + * Copyright 2022 Confluent Inc. + * + * Licensed under the Confluent Community License (the "License"); you may not use + * this file except in compliance with the License. You may obtain a copy of the + * License at + * + * http://www.confluent.io/confluent-community-license + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, WITHOUT + * WARRANTIES OF ANY KIND, either express or implied. See the License for the + * specific language governing permissions and limitations under the License. + */ + +package io.confluent.ksql.function.udaf.min; + +import static org.hamcrest.CoreMatchers.equalTo; +import static org.hamcrest.CoreMatchers.instanceOf; +import static org.hamcrest.MatcherAssert.assertThat; + +import io.confluent.ksql.GenericKey; +import io.confluent.ksql.function.AggregateFunctionInitArguments; +import io.confluent.ksql.function.KsqlAggregateFunction; +import io.confluent.ksql.schema.ksql.SqlArgument; +import io.confluent.ksql.schema.ksql.types.SqlTypes; +import java.util.Collections; +import org.apache.kafka.streams.kstream.Merger; +import org.junit.Test; + +public class StringMinKudafTest { + + @Test + public void shouldFindCorrectMin() { + final MinKudaf stringMinKudaf = getStringMinKudaf(); + final String[] values = new String[]{"C", "F", "B", "E", "A", "D", "B"}; + String currentMin = "D"; + for (final String val : values) { + currentMin = stringMinKudaf.aggregate(val, currentMin); + } + assertThat("A", equalTo(currentMin)); + } + + @Test + public void shouldHandleNull() { + final MinKudaf stringMinKudaf = getStringMinKudaf(); + final String[] values = new String[]{"C", "F", "B", "E", "A", "D", "B"}; + String currentMin = null; + + // null before any aggregation + currentMin = stringMinKudaf.aggregate(null, currentMin); + assertThat(null, equalTo(currentMin)); + + // now send each value to aggregation and verify + for (final String val : values) { + currentMin = stringMinKudaf.aggregate(val, currentMin); + } + assertThat("A", equalTo(currentMin)); + + // null should not impact result + currentMin = stringMinKudaf.aggregate(null, currentMin); + assertThat("A", equalTo(currentMin)); + } + + @Test + public void shouldFindCorrectMinForMerge() { + final MinKudaf stringMinKudaf = getStringMinKudaf(); + final Merger merger = stringMinKudaf.getMerger(); + final String mergeResult1 = merger.apply(null, "B", "D"); + assertThat(mergeResult1, equalTo("B")); + final String mergeResult2 = merger.apply(null, "P", "F"); + assertThat(mergeResult2, equalTo("F")); + final String mergeResult3 = merger.apply(null, "A", "K"); + assertThat(mergeResult3, equalTo("A")); + } + + private MinKudaf getStringMinKudaf() { + final KsqlAggregateFunction aggregateFunction = new MinAggFunctionFactory() + .createAggregateFunction(Collections.singletonList(SqlArgument.of(SqlTypes.STRING)), + AggregateFunctionInitArguments.EMPTY_ARGS); + assertThat(aggregateFunction, instanceOf(MinKudaf.class)); + return (MinKudaf) aggregateFunction; + } + +} diff --git a/ksqldb-engine/src/test/java/io/confluent/ksql/function/udf/lambda/ReduceTest.java b/ksqldb-engine/src/test/java/io/confluent/ksql/function/udf/lambda/ReduceTest.java index 7ce062cc9e6c..cbc9c73e0e6b 100644 --- a/ksqldb-engine/src/test/java/io/confluent/ksql/function/udf/lambda/ReduceTest.java +++ b/ksqldb-engine/src/test/java/io/confluent/ksql/function/udf/lambda/ReduceTest.java @@ -23,8 +23,7 @@ import com.google.common.collect.ImmutableList; import edu.umd.cs.findbugs.annotations.SuppressFBWarnings; import io.confluent.ksql.execution.codegen.helpers.TriFunction; -import io.confluent.ksql.function.GenericsUtil; -import io.confluent.ksql.util.KsqlException; + import java.util.Arrays; import java.util.Collections; import java.util.HashMap; diff --git a/ksqldb-engine/src/test/java/io/confluent/ksql/query/KsqlFunctionClassifierTest.java b/ksqldb-engine/src/test/java/io/confluent/ksql/query/KsqlFunctionClassifierTest.java new file mode 100644 index 000000000000..d4a788ec84e2 --- /dev/null +++ b/ksqldb-engine/src/test/java/io/confluent/ksql/query/KsqlFunctionClassifierTest.java @@ -0,0 +1,91 @@ +/* + * Copyright 2022 Confluent Inc. + * + * Licensed under the Confluent Community License (the "License"; you may not use + * this file except in compliance with the License. You may obtain a copy of the + * License at + * + * http://www.confluent.io/confluent-community-license + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, WITHOUT + * WARRANTIES OF ANY KIND, either express or implied. See the License for the + * specific language governing permissions and limitations under the License. + */ + +package io.confluent.ksql.query; + +import static org.hamcrest.MatcherAssert.assertThat; +import static org.hamcrest.Matchers.is; + +import io.confluent.ksql.function.KsqlFunctionException; +import io.confluent.ksql.query.QueryError.Type; +import org.apache.kafka.streams.errors.StreamsException; +import org.junit.Test; +import org.junit.runner.RunWith; +import org.mockito.junit.MockitoJUnitRunner; + +@RunWith(MockitoJUnitRunner.class) +public class KsqlFunctionClassifierTest { + + @Test + public void shouldClassifyWrappedKsqlFunctionExceptionAsUserError() { + // Given: + final Exception e = new StreamsException(new KsqlFunctionException("foo")); + + // When: + final Type type = new KsqlFunctionClassifier("").classify(e); + + // Then: + assertThat(type, is(Type.USER)); + } + + @Test + public void shouldClassifyKsqlFunctionExceptionAsUserError() { + // Given: + final Exception e = new KsqlFunctionException("foo"); + + // When: + final Type type = new KsqlFunctionClassifier("").classify(e); + + // Then: + assertThat(type, is(Type.USER)); + } + + @Test + public void shouldClassifyWrappedStreamsExceptionWithoutKsqlFunctionExceptionAsUnknownError() { + // Given: + final Exception e = new StreamsException(new ArithmeticException()); + + // When: + final Type type = new KsqlFunctionClassifier("").classify(e); + + // Then: + assertThat(type, is(Type.UNKNOWN)); + } + + @Test + public void shouldClassifyStreamsExceptionAsUnknownError() { + // Given: + final Exception e = new StreamsException("foo"); + + // When: + final Type type = new KsqlFunctionClassifier("").classify(e); + + // Then: + assertThat(type, is(Type.UNKNOWN)); + } + + @Test + public void shouldClassifyGenericExceptionAsUnknownError() { + // Given: + final Exception e = new Exception("foo"); + + // When: + final Type type = new KsqlFunctionClassifier("").classify(e); + + // Then: + assertThat(type, is(Type.UNKNOWN)); + } + +} \ No newline at end of file diff --git a/ksqldb-engine/src/test/java/io/confluent/ksql/query/MissingSubjectClassifierTest.java b/ksqldb-engine/src/test/java/io/confluent/ksql/query/MissingSubjectClassifierTest.java new file mode 100644 index 000000000000..2ac11df4a2fa --- /dev/null +++ b/ksqldb-engine/src/test/java/io/confluent/ksql/query/MissingSubjectClassifierTest.java @@ -0,0 +1,61 @@ +/* + * Copyright 2022 Confluent Inc. + * + * Licensed under the Confluent Community License (the "License"; you may not use + * this file except in compliance with the License. You may obtain a copy of the + * License at + * + * http://www.confluent.io/confluent-community-license + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, WITHOUT + * WARRANTIES OF ANY KIND, either express or implied. See the License for the + * specific language governing permissions and limitations under the License. + */ + +package io.confluent.ksql.query; + +import io.confluent.kafka.schemaregistry.client.rest.exceptions.RestClientException; +import org.apache.kafka.streams.errors.MissingSourceTopicException; +import org.junit.Test; + +import static org.hamcrest.MatcherAssert.assertThat; +import static org.hamcrest.Matchers.is; + +public class MissingSubjectClassifierTest { + @Test + public void shouldClassifyMissingSubjectAsUserError() { + // Given: + final Exception e = new RestClientException("foo", 404, 40401); + + // When: + final QueryError.Type type = new MissingSubjectClassifier("").classify(e); + + // Then: + assertThat(type, is(QueryError.Type.USER)); + } + + @Test + public void shouldClassifyNoMissingSubjectAsUnknownErrorCode() { + // Given: + final Exception e = new RestClientException("foo", 401, 40101); + + // When: + final QueryError.Type type = new MissingSubjectClassifier("").classify(e); + + // Then: + assertThat(type, is(QueryError.Type.UNKNOWN)); + } + + @Test + public void shouldClassifyOtherExceptionAsUnknownException() { + // Given: + final Exception e = new Exception("foo"); + + // When: + final QueryError.Type type = new MissingSubjectClassifier("").classify(e); + + // Then: + assertThat(type, is(QueryError.Type.UNKNOWN)); + } +} diff --git a/ksqldb-engine/src/test/java/io/confluent/ksql/query/QueryBuilderTest.java b/ksqldb-engine/src/test/java/io/confluent/ksql/query/QueryBuilderTest.java index eabd21339a79..4e1eb17d5cea 100644 --- a/ksqldb-engine/src/test/java/io/confluent/ksql/query/QueryBuilderTest.java +++ b/ksqldb-engine/src/test/java/io/confluent/ksql/query/QueryBuilderTest.java @@ -4,10 +4,7 @@ import static org.hamcrest.Matchers.contains; import static org.hamcrest.Matchers.equalTo; import static org.hamcrest.Matchers.is; -import static org.mockito.ArgumentMatchers.any; -import static org.mockito.ArgumentMatchers.anyString; -import static org.mockito.ArgumentMatchers.eq; -import static org.mockito.ArgumentMatchers.same; +import static org.mockito.ArgumentMatchers.*; import static org.mockito.Mockito.mock; import static org.mockito.Mockito.verify; import static org.mockito.Mockito.when; @@ -224,7 +221,7 @@ public void setup() { .thenReturn(Optional.of(ksMaterialization)); when(ksqlMaterializationFactory.create(any(), any(), any(), any())).thenReturn(materialization); when(processingLogContext.getLoggerFactory()).thenReturn(processingLoggerFactory); - when(processingLoggerFactory.getLogger(any())).thenReturn(processingLogger); + when(processingLoggerFactory.getLogger(any(), anyMap())).thenReturn(processingLogger); when(ksqlConfig.getKsqlStreamConfigProps(anyString())).thenReturn(Collections.emptyMap()); when(ksqlConfig.getString(KsqlConfig.KSQL_CUSTOM_METRICS_TAGS)).thenReturn(""); when(ksqlConfig.getString(KsqlConfig.KSQL_PERSISTENT_QUERY_NAME_PREFIX_CONFIG)) @@ -296,8 +293,9 @@ public void shouldBuildCreateAsPersistentQueryCorrectly() { final ProcessingLogger uncaughtProcessingLogger = mock(ProcessingLogger.class); when(processingLoggerFactory.getLogger( QueryLoggerUtil.queryLoggerName(QUERY_ID, new QueryContext.Stacker() - .push("ksql.logger.thread.exception.uncaught").getQueryContext()) - )).thenReturn(uncaughtProcessingLogger); + .push("ksql.logger.thread.exception.uncaught").getQueryContext()), + Collections.singletonMap("query-id", QUERY_ID.toString())) + ).thenReturn(uncaughtProcessingLogger); // When: final PersistentQueryMetadata queryMetadata = buildPersistentQuery( @@ -331,8 +329,9 @@ public void shouldBuildInsertPersistentQueryCorrectly() { final ProcessingLogger uncaughtProcessingLogger = mock(ProcessingLogger.class); when(processingLoggerFactory.getLogger( QueryLoggerUtil.queryLoggerName(QUERY_ID, new QueryContext.Stacker() - .push("ksql.logger.thread.exception.uncaught").getQueryContext()) - )).thenReturn(uncaughtProcessingLogger); + .push("ksql.logger.thread.exception.uncaught").getQueryContext()), + Collections.singletonMap("query-id", QUERY_ID.toString())) + ).thenReturn(uncaughtProcessingLogger); // When: final PersistentQueryMetadata queryMetadata = buildPersistentQuery( @@ -678,7 +677,7 @@ public void shouldMakePersistentQueriesWithDifferentSources() { @Test public void shouldConfigureProducerErrorHandler() { final ProcessingLogger logger = mock(ProcessingLogger.class); - when(processingLoggerFactory.getLogger(QUERY_ID.toString())).thenReturn(logger); + when(processingLoggerFactory.getLogger(QUERY_ID.toString(), Collections.singletonMap("query-id", QUERY_ID.toString()))).thenReturn(logger); // When: buildPersistentQuery( diff --git a/ksqldb-engine/src/test/java/io/confluent/ksql/query/SchemaAuthorizationClassifierTest.java b/ksqldb-engine/src/test/java/io/confluent/ksql/query/SchemaAuthorizationClassifierTest.java new file mode 100644 index 000000000000..3acc0fc8664e --- /dev/null +++ b/ksqldb-engine/src/test/java/io/confluent/ksql/query/SchemaAuthorizationClassifierTest.java @@ -0,0 +1,75 @@ +/* + * Copyright 2022 Confluent Inc. + * + * Licensed under the Confluent Community License (the "License"; you may not use + * this file except in compliance with the License. You may obtain a copy of the + * License at + * + * http://www.confluent.io/confluent-community-license + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, WITHOUT + * WARRANTIES OF ANY KIND, either express or implied. See the License for the + * specific language governing permissions and limitations under the License. + */ + +package io.confluent.ksql.query; + +import io.confluent.kafka.schemaregistry.client.rest.exceptions.RestClientException; +import org.apache.kafka.common.errors.AuthorizationException; +import org.apache.kafka.common.errors.TopicAuthorizationException; +import org.apache.kafka.streams.errors.StreamsException; +import org.junit.Test; + +import static org.hamcrest.MatcherAssert.assertThat; +import static org.hamcrest.Matchers.is; + +public class SchemaAuthorizationClassifierTest { + @Test + public void shouldClassifySRAuthorizationErrorCodeAsUserError() { + // Given: + final Exception e = new RestClientException("foo", 403, 40301); + + // When: + final QueryError.Type type = new SchemaAuthorizationClassifier("").classify(e); + + // Then: + assertThat(type, is(QueryError.Type.USER)); + } + + @Test + public void shouldClassifySRAuthenticationErrorCodeAsUserError() { + // Given: + final Exception e = new RestClientException("foo", 401, 403101); + + // When: + final QueryError.Type type = new SchemaAuthorizationClassifier("").classify(e); + + // Then: + assertThat(type, is(QueryError.Type.USER)); + } + + @Test + public void shouldClassifyNoAuthErrorSubjectAsUnknownErrorCode() { + // Given: + final Exception e = new RestClientException("foo", 404, 40401); + + // When: + final QueryError.Type type = new SchemaAuthorizationClassifier("").classify(e); + + // Then: + assertThat(type, is(QueryError.Type.UNKNOWN)); + } + + @Test + public void shouldClassifyOtherExceptionAsUnknownException() { + // Given: + final Exception e = new Exception("foo"); + + // When: + final QueryError.Type type = new SchemaAuthorizationClassifier("").classify(e); + + // Then: + assertThat(type, is(QueryError.Type.UNKNOWN)); + } +} diff --git a/ksqldb-engine/src/test/java/io/confluent/ksql/util/BinPackedPersistentQueryMetadataImplTest.java b/ksqldb-engine/src/test/java/io/confluent/ksql/util/BinPackedPersistentQueryMetadataImplTest.java index f9b15aa60b45..9d41a50ee7d2 100644 --- a/ksqldb-engine/src/test/java/io/confluent/ksql/util/BinPackedPersistentQueryMetadataImplTest.java +++ b/ksqldb-engine/src/test/java/io/confluent/ksql/util/BinPackedPersistentQueryMetadataImplTest.java @@ -15,19 +15,27 @@ package io.confluent.ksql.util; +import static org.mockito.ArgumentMatchers.any; +import static org.mockito.ArgumentMatchers.eq; import static org.mockito.Mockito.inOrder; +import static org.mockito.Mockito.mock; import static org.mockito.Mockito.times; import static org.mockito.Mockito.verify; +import static org.mockito.Mockito.when; import com.google.common.collect.ImmutableSet; import edu.umd.cs.findbugs.annotations.SuppressFBWarnings; +import io.confluent.ksql.execution.context.QueryContext; +import io.confluent.ksql.execution.materialization.MaterializationInfo; import io.confluent.ksql.execution.plan.ExecutionStep; import io.confluent.ksql.logging.processing.ProcessingLogger; import io.confluent.ksql.metastore.model.DataSource; import io.confluent.ksql.execution.scalablepush.ScalablePushRegistry; +import io.confluent.ksql.query.MaterializationProviderBuilderFactory; import io.confluent.ksql.query.QueryId; import io.confluent.ksql.schema.ksql.PhysicalSchema; import io.confluent.ksql.schema.query.QuerySchemas; +import io.confluent.ksql.serde.KeyFormat; import io.confluent.ksql.util.QueryMetadata.Listener; import java.util.Collections; @@ -35,6 +43,8 @@ import java.util.Optional; import org.apache.kafka.common.metrics.Metrics; +import org.apache.kafka.streams.KafkaStreams; +import org.apache.kafka.streams.processor.internals.namedtopology.KafkaStreamsNamedTopologyWrapper; import org.apache.kafka.streams.processor.internals.namedtopology.NamedTopology; import org.junit.Before; import org.junit.Test; @@ -72,6 +82,21 @@ public class BinPackedPersistentQueryMetadataImplTest { private Map streamsProperties; @Mock private Optional scalablePushRegistry; + @Mock + private MaterializationProviderBuilderFactory + materializationProviderBuilderFactory; + @Mock + private KafkaStreamsNamedTopologyWrapper kafkaStreamsNamedTopologyWrapper; + @Mock + private KafkaStreamsNamedTopologyWrapper kafkaStreamsNamedTopologyWrapper2; + @Mock + private QueryContext.Stacker stacker; + @Mock + private KeyFormat keyFormat; + @Mock + private MaterializationInfo materializationInfo; + @Mock + private MaterializationProviderBuilderFactory.MaterializationProviderBuilder materializationProviderBuilder; private Metrics metrics; private Map metricsTags = Collections.singletonMap("tag1", "value1"); @@ -91,18 +116,35 @@ public void setUp() { schemas, overrides, QUERY_ID, - Optional.empty(), + Optional.of(materializationInfo), + materializationProviderBuilderFactory, physicalPlan, processingLogger, Optional.of(sinkDataSource), listener, - streamsProperties, scalablePushRegistry, (runtime) -> topology, + keyFormat, metrics, metricsTags); query.initialize(); + when(materializationProviderBuilderFactory.materializationProviderBuilder( + any(), any(), any(), any(), any(), any())) + .thenReturn(materializationProviderBuilder); + } + + @Test + public void shouldGetStreamsFreshForMaterialization() { + when(sharedKafkaStreamsRuntimeImpl.getKafkaStreams()) + .thenReturn(kafkaStreamsNamedTopologyWrapper) + .thenReturn(kafkaStreamsNamedTopologyWrapper2); + when(materializationProviderBuilder.apply(any(KafkaStreams.class), any())).thenReturn(Optional.empty()); + query.getMaterialization(query.getQueryId(), stacker); + query.getMaterialization(query.getQueryId(), stacker); + + verify(materializationProviderBuilder).apply(eq(kafkaStreamsNamedTopologyWrapper), any()); + verify(materializationProviderBuilder).apply(eq(kafkaStreamsNamedTopologyWrapper2), any()); } @Test diff --git a/ksqldb-engine/src/test/java/io/confluent/ksql/util/SharedKafkaStreamsRuntimeImplTest.java b/ksqldb-engine/src/test/java/io/confluent/ksql/util/SharedKafkaStreamsRuntimeImplTest.java index 93a41a76b723..239473ee3e83 100644 --- a/ksqldb-engine/src/test/java/io/confluent/ksql/util/SharedKafkaStreamsRuntimeImplTest.java +++ b/ksqldb-engine/src/test/java/io/confluent/ksql/util/SharedKafkaStreamsRuntimeImplTest.java @@ -93,6 +93,7 @@ public void setUp() throws Exception { metricCollectors = new MetricCollectors(); when(kafkaStreamsBuilder.buildNamedTopologyWrapper(any())).thenReturn(kafkaStreamsNamedTopologyWrapper).thenReturn(kafkaStreamsNamedTopologyWrapper2); streamProps.put(StreamsConfig.APPLICATION_ID_CONFIG, "runtime"); + streamProps.put(StreamsConfig.APPLICATION_SERVER_CONFIG, "old"); sharedKafkaStreamsRuntimeImpl = new SharedKafkaStreamsRuntimeImpl( kafkaStreamsBuilder, queryErrorClassifier, @@ -120,14 +121,14 @@ public void setUp() throws Exception { public void overrideStreamsPropertiesShouldReplaceProperties() { // Given: final Map newProps = new HashMap<>(); - newProps.put("Test", "Test"); + newProps.put(StreamsConfig.APPLICATION_SERVER_CONFIG, "notused"); // When: sharedKafkaStreamsRuntimeImpl.overrideStreamsProperties(newProps); // Then: final Map properties = sharedKafkaStreamsRuntimeImpl.streamsProperties; - assertThat(properties.get("Test"), equalTo("Test")); + assertThat(properties.get(StreamsConfig.APPLICATION_SERVER_CONFIG), equalTo("old")); assertThat(properties.size(), equalTo(1)); } @@ -240,7 +241,7 @@ public void shouldRestart() { //Then: verify(kafkaStreamsNamedTopologyWrapper).close(); - verify(kafkaStreamsNamedTopologyWrapper2).addNamedTopology(namedTopology); + verify(kafkaStreamsNamedTopologyWrapper2).addNamedTopology(any()); verify(kafkaStreamsNamedTopologyWrapper2).start(); verify(kafkaStreamsNamedTopologyWrapper2).setUncaughtExceptionHandler((StreamsUncaughtExceptionHandler) any()); } diff --git a/ksqldb-execution/src/main/java/io/confluent/ksql/execution/runtime/RuntimeBuildContext.java b/ksqldb-execution/src/main/java/io/confluent/ksql/execution/runtime/RuntimeBuildContext.java index 88cecde4987c..2431064400e7 100644 --- a/ksqldb-execution/src/main/java/io/confluent/ksql/execution/runtime/RuntimeBuildContext.java +++ b/ksqldb-execution/src/main/java/io/confluent/ksql/execution/runtime/RuntimeBuildContext.java @@ -40,6 +40,8 @@ import io.confluent.ksql.serde.tracked.TrackedCallback; import io.confluent.ksql.services.ServiceContext; import io.confluent.ksql.util.KsqlConfig; +import io.confluent.ksql.util.MetricsTagsUtil; +import java.util.Collections; import java.util.Optional; import org.apache.kafka.common.serialization.Serde; import org.apache.kafka.streams.StreamsBuilder; @@ -86,8 +88,8 @@ public static RuntimeBuildContext of( functionRegistry, applicationId, queryId, - new GenericKeySerDe(), - new GenericRowSerDe() + new GenericKeySerDe(queryId.toString()), + new GenericRowSerDe(queryId.toString()) ); } @@ -117,7 +119,9 @@ public static RuntimeBuildContext of( public ProcessingLogger getProcessingLogger(final QueryContext queryContext) { return processingLogContext .getLoggerFactory() - .getLogger(QueryLoggerUtil.queryLoggerName(queryId, queryContext)); + .getLogger( + QueryLoggerUtil.queryLoggerName(queryId, queryContext), + MetricsTagsUtil.getMetricsTagsWithQueryId(queryId.toString(), Collections.emptyMap())); } public ServiceContext getServiceContext() { diff --git a/ksqldb-execution/src/test/java/io/confluent/ksql/execution/runtime/RuntimeBuildContextTest.java b/ksqldb-execution/src/test/java/io/confluent/ksql/execution/runtime/RuntimeBuildContextTest.java index 0817f5a8a3fc..c36749f876b4 100644 --- a/ksqldb-execution/src/test/java/io/confluent/ksql/execution/runtime/RuntimeBuildContextTest.java +++ b/ksqldb-execution/src/test/java/io/confluent/ksql/execution/runtime/RuntimeBuildContextTest.java @@ -16,8 +16,7 @@ package io.confluent.ksql.execution.runtime; import static org.hamcrest.MatcherAssert.assertThat; -import static org.hamcrest.Matchers.hasSize; -import static org.hamcrest.Matchers.is; +import static org.hamcrest.Matchers.*; import static org.junit.Assert.assertThrows; import static org.mockito.ArgumentMatchers.any; import static org.mockito.Mockito.verify; @@ -32,6 +31,8 @@ import io.confluent.ksql.execution.context.QueryLoggerUtil; import io.confluent.ksql.function.FunctionRegistry; import io.confluent.ksql.logging.processing.ProcessingLogContext; +import io.confluent.ksql.logging.processing.ProcessingLogger; +import io.confluent.ksql.logging.processing.ProcessingLoggerFactory; import io.confluent.ksql.model.WindowType; import io.confluent.ksql.name.ColumnName; import io.confluent.ksql.query.QueryId; @@ -53,9 +54,11 @@ import io.confluent.ksql.services.ServiceContext; import io.confluent.ksql.util.KsqlConfig; import java.time.Duration; +import java.util.Collections; import java.util.Map; import java.util.Optional; import java.util.function.Supplier; + import org.apache.kafka.common.serialization.Serde; import org.apache.kafka.streams.StreamsBuilder; import org.apache.kafka.streams.kstream.Windowed; @@ -99,6 +102,10 @@ public class RuntimeBuildContextTest { @Mock private ProcessingLogContext processingLogContext; @Mock + private ProcessingLoggerFactory processingLogFactory; + @Mock + private ProcessingLogger processingLogger; + @Mock private FunctionRegistry functionRegistry; @Mock private Serde keySerde; @@ -115,7 +122,6 @@ public class RuntimeBuildContextTest { private QueryContext queryContext; private RuntimeBuildContext runtimeBuildContext; - @Before public void setUp() { when(serviceContext.getSchemaRegistryClientFactory()).thenReturn(srClientFactory); @@ -131,6 +137,8 @@ public void setUp() { when(valueSerdeFactory.create(any(), any(), any(), any(), any(), any(), any())) .thenReturn(valueSerde); + when(processingLogContext.getLoggerFactory()).thenReturn(processingLogFactory); + runtimeBuildContext = new RuntimeBuildContext( streamsBuilder, ksqlConfig, @@ -318,4 +326,16 @@ public void shouldFailWhenTackingSerdeOnSchemaMismatch() { queryContext )); } + + @Test + public void shouldReturnProcessingLogger() { + // When: + when(processingLogFactory.getLogger( + QueryLoggerUtil.queryLoggerName(QUERY_ID, queryContext), Collections.singletonMap("query-id", QUERY_ID.toString()))) + .thenReturn(processingLogger); + final ProcessingLogger logger = runtimeBuildContext.getProcessingLogger(queryContext); + + // Then: + assertThat(processingLogger, equalTo(logger)); + } } \ No newline at end of file diff --git a/ksqldb-functional-tests/hs_err_pid11216.log b/ksqldb-functional-tests/hs_err_pid11216.log deleted file mode 100644 index 71dec2e9b1af..000000000000 --- a/ksqldb-functional-tests/hs_err_pid11216.log +++ /dev/null @@ -1,518 +0,0 @@ -# -# A fatal error has been detected by the Java Runtime Environment: -# -# SIGSEGV (0xb) at pc=0x000000010a90114f, pid=11216, tid=0x0000000000003503 -# -# JRE version: OpenJDK Runtime Environment (8.0_282-b08) (build 1.8.0_282-b08) -# Java VM: OpenJDK 64-Bit Server VM (25.282-b08 mixed mode bsd-amd64 compressed oops) -# Problematic frame: -# V [libjvm.dylib+0x50114f] -# -# Failed to write core dump. Core dumps have been disabled. To enable core dumping, try "ulimit -c unlimited" before starting Java again -# -# If you would like to submit a bug report, please visit: -# https://github.com/AdoptOpenJDK/openjdk-support/issues -# - ---------------- T H R E A D --------------- - -Current thread (0x00007f91e901c800): VMThread [stack: 0x00007000057d8000,0x00007000058d8000] [id=13571] - -siginfo: si_signo: 11 (SIGSEGV), si_code: 1 (SEGV_MAPERR), si_addr: 0x0000000000000008 - -Registers: -RAX=0x0000000000000000, RBX=0x00007000058d7130, RCX=0x0000000000000000, RDX=0x0000000000000000 -RSP=0x00007000058d6fd0, RBP=0x00007000058d70f0, RSI=0x0000000000000028, RDI=0x00007000058d7130 -R8 =0x000000000000001a, R9 =0x000000008fffffff, R10=0x000000008a000000, R11=0xffffffffffa4b380 -R12=0x00007000058d7130, R13=0x00007000058d77b8, R14=0x0000000000000001, R15=0x000000000000001a -RIP=0x000000010a90114f, EFLAGS=0x0000000000010202, ERR=0x0000000000000004 - TRAPNO=0x000000000000000e - -Top of Stack: (sp=0x00007000058d6fd0) -0x00007000058d6fd0: 00007f91e9855910 0000002000000008 -0x00007000058d6fe0: 0105c177c18103c1 00007f91e9855bb0 -0x00007000058d6ff0: 0000000000000004 00007f91e98559b0 -0x00007000058d7000: 0000000400000004 0000000000000000 -0x00007000058d7010: 00007f9100000007 00007f91e9855950 -0x00007000058d7020: 00007000058d7040 000000010a90136f -0x00007000058d7030: 00007000058d7080 91cf91d5ade50023 -0x00007000058d7040: 00007000058d7080 00007000058d7080 -0x00007000058d7050: 00007000058d7070 000000010a901598 -0x00007000058d7060: 0000000000000000 000000004fffffff -0x00007000058d7070: 00007000058d7500 000000010a66ab05 -0x00007000058d7080: 000000010aad3728 00007f91f00543f0 -0x00007000058d7090: 0000000000000002 000000000000000e -0x00007000058d70a0: 00007000058d70b0 00007f9100000001 -0x00007000058d70b0: 4fffffff4fffffff 0000002000000008 -0x00007000058d70c0: 0105c177c18103c1 00007f91e9855ac0 -0x00007000058d70d0: 00007000058d70f0 91cf91d5ade50023 -0x00007000058d70e0: 00007000058d7130 0000000000000001 -0x00007000058d70f0: 00007000058d7120 000000010a901727 -0x00007000058d7100: 00007000058d7570 00007000058d7130 -0x00007000058d7110: 0000000000000001 000000000000001a -0x00007000058d7120: 00007000058d75b0 000000010a66aa1f -0x00007000058d7130: 000000010aad3728 0000000000000000 -0x00007000058d7140: 0000000000000000 000000010000000c -0x00007000058d7150: 00007000058d7570 0000000000000000 -0x00007000058d7160: 4fffffff4fffffff 000000010a66ab05 -0x00007000058d7170: 000000010aad3728 00007f91f0064570 -0x00007000058d7180: 000000000000001f 000000000000000e -0x00007000058d7190: 00007000058d71a0 00007f9100000002 -0x00007000058d71a0: 4fffffff4fffffff 000000010828ba00 -0x00007000058d71b0: 00007000058d71f0 00007fff6f07aa3d -0x00007000058d71c0: 00007000058d71e0 000000010a90136f - -Instructions: (pc=0x000000010a90114f) -0x000000010a90112f: 48 81 ec 10 01 00 00 48 89 fb 48 8b 05 a8 c2 14 -0x000000010a90113f: 00 48 8b 00 48 89 45 e8 48 8b 4f 08 48 63 47 10 -0x000000010a90114f: 40 38 74 01 08 74 38 40 0f be d6 48 8d 35 c2 a4 -0x000000010a90115f: 10 00 4c 8d b5 e0 fe ff ff 31 c0 4c 89 f7 e8 ae - -Register to memory mapping: - -RAX=0x0000000000000000 is an unknown value -RBX=0x00007000058d7130 is an unknown value -RCX=0x0000000000000000 is an unknown value -RDX=0x0000000000000000 is an unknown value -RSP=0x00007000058d6fd0 is an unknown value -RBP=0x00007000058d70f0 is an unknown value -RSI=0x0000000000000028 is an unknown value -RDI=0x00007000058d7130 is an unknown value -R8 =0x000000000000001a is an unknown value -R9 =0x000000008fffffff is an unknown value -R10=0x000000008a000000 is an unknown value -R11=0xffffffffffa4b380 is an unknown value -R12=0x00007000058d7130 is an unknown value -R13=0x00007000058d77b8 is an unknown value -R14=0x0000000000000001 is an unknown value -R15=0x000000000000001a is an unknown value - - -Stack: [0x00007000057d8000,0x00007000058d8000], sp=0x00007000058d6fd0, free space=1019k -Native frames: (J=compiled Java code, j=interpreted, Vv=VM code, C=native code) -V [libjvm.dylib+0x50114f] -V [libjvm.dylib+0x501727] -V [libjvm.dylib+0x26aa1f] -V [libjvm.dylib+0x2690e9] -V [libjvm.dylib+0x268d36] -V [libjvm.dylib+0x2688ff] -V [libjvm.dylib+0x26b348] -V [libjvm.dylib+0x48392b] -V [libjvm.dylib+0x4841ce] -V [libjvm.dylib+0x484d86] -V [libjvm.dylib+0x484a5e] -V [libjvm.dylib+0x2991b1] -V [libjvm.dylib+0x44bcb9] -V [libjvm.dylib+0x399297] -V [libjvm.dylib+0x5a8222] -V [libjvm.dylib+0x5a7b70] -V [libjvm.dylib+0x5a7580] -V [libjvm.dylib+0x5a73b5] -V [libjvm.dylib+0x489627] -C [libsystem_pthread.dylib+0x6109] _pthread_start+0x94 -C [libsystem_pthread.dylib+0x1b8b] thread_start+0xf - -VM_Operation (0x0000700005de9a98): GetOrSetLocal, mode: safepoint, requested by thread 0x00007f91b8809800 - - ---------------- P R O C E S S --------------- - -Java Threads: ( => current thread ) - 0x00007f91c918a800 JavaThread "Service Thread" daemon [_thread_blocked, id=26627, stack(0x0000700006c17000,0x0000700006d17000)] - 0x00007f91b880c000 JavaThread "C1 CompilerThread11" daemon [_thread_blocked, id=26371, stack(0x0000700006b14000,0x0000700006c14000)] - 0x00007f91e8b34800 JavaThread "C1 CompilerThread10" daemon [_thread_blocked, id=25859, stack(0x0000700006a11000,0x0000700006b11000)] - 0x00007f91e8b33800 JavaThread "C1 CompilerThread9" daemon [_thread_blocked, id=40963, stack(0x000070000690e000,0x0000700006a0e000)] - 0x00007f91b880b000 JavaThread "C1 CompilerThread8" daemon [_thread_blocked, id=25347, stack(0x000070000680b000,0x000070000690b000)] - 0x00007f91c917f800 JavaThread "C2 CompilerThread7" daemon [_thread_blocked, id=24835, stack(0x0000700006708000,0x0000700006808000)] - 0x00007f91c917e800 JavaThread "C2 CompilerThread6" daemon [_thread_blocked, id=24323, stack(0x0000700006605000,0x0000700006705000)] - 0x00007f91a8808800 JavaThread "C2 CompilerThread5" daemon [_thread_blocked, id=24067, stack(0x0000700006502000,0x0000700006602000)] - 0x00007f91c9175800 JavaThread "C2 CompilerThread4" daemon [_thread_blocked, id=41475, stack(0x00007000063ff000,0x00007000064ff000)] - 0x00007f91c916c800 JavaThread "C2 CompilerThread3" daemon [_thread_blocked, id=41987, stack(0x00007000062fc000,0x00007000063fc000)] - 0x00007f91e981c800 JavaThread "C2 CompilerThread2" daemon [_thread_blocked, id=23555, stack(0x00007000061f9000,0x00007000062f9000)] - 0x00007f91e8b2a800 JavaThread "C2 CompilerThread1" daemon [_thread_blocked, id=23299, stack(0x00007000060f6000,0x00007000061f6000)] - 0x00007f91ca03f800 JavaThread "C2 CompilerThread0" daemon [_thread_blocked, id=22787, stack(0x0000700005ff3000,0x00007000060f3000)] - 0x00007f91e9034000 JavaThread "JDWP Command Reader" daemon [_thread_in_native, id=42755, stack(0x0000700005ef0000,0x0000700005ff0000)] - 0x00007f91b880a000 JavaThread "JDWP Event Helper Thread" daemon [_thread_blocked, id=43011, stack(0x0000700005ded000,0x0000700005eed000)] - 0x00007f91b8809800 JavaThread "JDWP Transport Listener: dt_socket" daemon [_thread_blocked, id=22023, stack(0x0000700005cea000,0x0000700005dea000)] - 0x00007f91c9017000 JavaThread "Signal Dispatcher" daemon [_thread_blocked, id=16643, stack(0x0000700005be7000,0x0000700005ce7000)] - 0x00007f91ca036800 JavaThread "Finalizer" daemon [_thread_blocked, id=14339, stack(0x00007000059de000,0x0000700005ade000)] - 0x00007f91ea00b800 JavaThread "Reference Handler" daemon [_thread_blocked, id=13827, stack(0x00007000058db000,0x00007000059db000)] - 0x00007f91c9009000 JavaThread "main" [_thread_blocked, id=5635, stack(0x00007000049ae000,0x0000700004aae000)] - -Other Threads: -=>0x00007f91e901c800 VMThread [stack: 0x00007000057d8000,0x00007000058d8000] [id=13571] - 0x00007f91b880d000 WatcherThread [stack: 0x0000700006d1a000,0x0000700006e1a000] [id=26883] - -VM state:at safepoint (normal execution) - -VM Mutex/Monitor currently owned by a thread: ([mutex/lock_event]) -[0x00007f91c8405470] Threads_lock - owner thread: 0x00007f91e901c800 - -heap address: 0x00000005c0000000, size: 8192 MB, Compressed Oops mode: Zero based, Oop shift amount: 3 -Narrow klass base: 0x0000000000000000, Narrow klass shift: 3 -Compressed class space size: 1073741824 Address: 0x00000007c0000000 - -Heap: - PSYoungGen total 153088K, used 44751K [0x0000000715580000, 0x0000000720000000, 0x00000007c0000000) - eden space 131584K, 34% used [0x0000000715580000,0x0000000718133e10,0x000000071d600000) - from space 21504K, 0% used [0x000000071eb00000,0x000000071eb00000,0x0000000720000000) - to space 21504K, 0% used [0x000000071d600000,0x000000071d600000,0x000000071eb00000) - ParOldGen total 349696K, used 0K [0x00000005c0000000, 0x00000005d5580000, 0x0000000715580000) - object space 349696K, 0% used [0x00000005c0000000,0x00000005c0000000,0x00000005d5580000) - Metaspace used 6270K, capacity 6508K, committed 6784K, reserved 1056768K - class space used 734K, capacity 828K, committed 896K, reserved 1048576K - -Card table byte_map: [0x000000010b19e000,0x000000010c19f000] byte_map_base: 0x000000010839e000 - -Marking Bits: (ParMarkBitMap*) 0x000000010ab10e38 - Begin Bits: [0x000000011f5d0000, 0x00000001275d0000) - End Bits: [0x00000001275d0000, 0x000000012f5d0000) - -Polling page: 0x00000001082f6000 - -CodeCache: size=245760Kb used=4010Kb max_used=4010Kb free=241749Kb - bounds [0x00000001105d0000, 0x00000001109c0000, 0x000000011f5d0000] - total_blobs=740 nmethods=455 adapters=204 - compilation: enabled - -Compilation events (10 events): -Event: 0.511 Thread 0x00007f91e8b34800 nmethod 401 0x00000001109ac7d0 code [0x00000001109ac9e0, 0x00000001109ad798] -Event: 0.511 Thread 0x00007f91e8b33800 nmethod 406 0x00000001109add90 code [0x00000001109adf60, 0x00000001109ae678] -Event: 0.511 Thread 0x00007f91b880c000 nmethod 405 0x00000001109ae9d0 code [0x00000001109aebe0, 0x00000001109af7f8] -Event: 0.511 Thread 0x00007f91e8b34800 407 1 java.lang.StringCoding$StringDecoder::requestedCharsetName (5 bytes) -Event: 0.511 Thread 0x00007f91e8b34800 nmethod 407 0x00000001109a25d0 code [0x00000001109a2720, 0x00000001109a2830] -Event: 0.511 Thread 0x00007f91b880b000 nmethod 404 0x00000001109afe90 code [0x00000001109b0240, 0x00000001109b3908] -Event: 0.512 Thread 0x00007f91e8b33800 408 3 org.groovy.debug.hotswap.ResetAgent::removeTimestampField (63 bytes) -Event: 0.513 Thread 0x00007f91e8b33800 nmethod 408 0x00000001109b4c10 code [0x00000001109b4de0, 0x00000001109b5378] -Event: 0.515 Thread 0x00007f91b880c000 409 3 java.nio.ByteBuffer::arrayOffset (35 bytes) -Event: 0.515 Thread 0x00007f91b880c000 nmethod 409 0x00000001109b55d0 code [0x00000001109b5780, 0x00000001109b5ab8] - -GC Heap History (0 events): -No events - -Deoptimization events (0 events): -No events - -Classes redefined (2 events): -Event: 367.757 Thread 0x00007f91e901c800 redefined class name=io.confluent.ksql.test.QueryAnonymizerTest, count=1 -Event: 408.317 Thread 0x00007f91e901c800 redefined class name=io.confluent.ksql.test.QueryAnonymizerTest, count=2 - -Internal exceptions (9 events): -Event: 0.107 Thread 0x00007f91c9009000 Exception (0x0000000715587cc0) thrown at [/Users/jenkins/workspace/build-scripts/jobs/jdk8u/jdk8u-mac-x64-hotsp -Event: 0.107 Thread 0x00007f91c9009000 Exception (0x0000000715587fa8) thrown at [/Users/jenkins/workspace/build-scripts/jobs/jdk8u/jdk8u-mac-x64-hotspot/workspace/build -Event: 0.375 Thread 0x00007f91c9009000 Exception (0x0000000716cbc5d0) thrown at [/Users/jenkins/workspace/build-scripts/jobs/jdk8u/jdk8u-mac-x64-hotspot/workspace/build/src/hotspot/src/share/vm/prims/jni.cpp, line 711] -Event: 0.404 Thread 0x00007f91c9009000 Exception (0x0000000716e2da60) thrown at [/Users/jenkins/workspace/build-scripts/jobs/jdk8u/jdk8u-mac-x64-hotspot/workspace/build/src/hotspot/src/share/vm/prims/jvm.cpp, line 1523] -Event: 0.404 Thread 0x00007f91c9009000 Exception (0x0000000716e2de58) thrown at [/Users/jenkins/workspace/build-scripts/jobs/jdk8u/jdk8u-mac-x64-hotspot/workspace/build/src/hotspot/src/share/vm/prims/jvm.cpp, line 1523] -Event: 0.404 Thread 0x00007f91c9009000 Exception (0x0000000716e31630) thrown at [/Users/jenkins/workspace/build-scripts/jobs/jdk8u/jdk8u-mac-x64-hotspot/workspace/build/src/hotspot/src/share/vm/prims/jvm.cpp, line 1523] -Event: 0.404 Thread 0x00007f91c9009000 Exception (0x0000000716e31a28) thrown at [/Users/jenkins/workspace/build-scripts/jobs/jdk8u/jdk8u-mac-x64-hotspot/workspace/build/src/hotspot/src/share/vm/prims/jvm.cpp, line 1523] -Event: 3.000 Thread 0x00007f91c9009000 Exception (0x0000000717f7bb68) thrown at [/Users/jenkins/workspace/build-scripts/jobs/jdk8u/jdk8u-mac-x64-hotspot/workspace/build/src/hotspot/src/share/vm/prims/methodHandles.cpp, line 1169] -Event: 3.005 Thread 0x00007f91c9009000 Exception (0x0000000717f8ac38) thrown at [/Users/jenkins/workspace/build-scripts/jobs/jdk8u/jdk8u-mac-x64-hotspot/workspace/build/src/hotspot/src/share/vm/prims/methodHandles.cpp, line 1169] - -Events (10 events): -Event: 1732.551 Executing VM operation: ChangeBreakpoints done -Event: 1732.552 Executing VM operation: ChangeBreakpoints -Event: 1732.552 Executing VM operation: ChangeBreakpoints done -Event: 1732.552 Executing VM operation: ChangeBreakpoints -Event: 1732.552 Executing VM operation: ChangeBreakpoints done -Event: 1732.553 Executing VM operation: ChangeBreakpoints -Event: 1732.554 Executing VM operation: ChangeBreakpoints done -Event: 1732.554 Executing VM operation: ChangeBreakpoints -Event: 1732.554 Executing VM operation: ChangeBreakpoints done -Event: 2066.219 Executing VM operation: GetOrSetLocal - - -Dynamic libraries: -0x00007fff34094000 /System/Library/Frameworks/Cocoa.framework/Versions/A/Cocoa -0x00007fff4189e000 /System/Library/Frameworks/Security.framework/Versions/A/Security -0x00007fff32ffa000 /System/Library/Frameworks/ApplicationServices.framework/Versions/A/ApplicationServices -0x00007fff6e3e2000 /usr/lib/libz.1.dylib -0x00007fff6bea6000 /usr/lib/libSystem.B.dylib -0x00007fff34f36000 /System/Library/Frameworks/CoreFoundation.framework/Versions/A/CoreFoundation -0x00007fff375f4000 /System/Library/Frameworks/Foundation.framework/Versions/C/Foundation -0x00007fff6dd06000 /usr/lib/libobjc.A.dylib -0x00007fff321ea000 /System/Library/Frameworks/AppKit.framework/Versions/C/AppKit -0x00007fff34a18000 /System/Library/Frameworks/CoreData.framework/Versions/A/CoreData -0x00007fff65ea1000 /System/Library/PrivateFrameworks/UIFoundation.framework/Versions/A/UIFoundation -0x00007fff60f35000 /System/Library/PrivateFrameworks/RemoteViewServices.framework/Versions/A/RemoteViewServices -0x00007fff68a16000 /System/Library/PrivateFrameworks/XCTTargetBootstrap.framework/Versions/A/XCTTargetBootstrap -0x00007fff34e03000 /System/Library/Frameworks/CoreDisplay.framework/Versions/A/CoreDisplay -0x00007fff3a4ae000 /System/Library/Frameworks/Metal.framework/Versions/A/Metal -0x00007fff4ef33000 /System/Library/PrivateFrameworks/DesktopServicesPriv.framework/Versions/A/DesktopServicesPriv -0x00007fff6cfe1000 /usr/lib/liblangid.dylib -0x00007fff4e1c8000 /System/Library/PrivateFrameworks/CoreSVG.framework/Versions/A/CoreSVG -0x00007fff64091000 /System/Library/PrivateFrameworks/SkyLight.framework/Versions/A/SkyLight -0x00007fff353b8000 /System/Library/Frameworks/CoreGraphics.framework/Versions/A/CoreGraphics -0x00007fff30bbe000 /System/Library/Frameworks/Accelerate.framework/Versions/A/Accelerate -0x00007fff6e2d3000 /usr/lib/libxml2.2.dylib -0x00007fff59ebf000 /System/Library/PrivateFrameworks/IconServices.framework/Versions/A/IconServices -0x00007fff37dd3000 /System/Library/Frameworks/IOSurface.framework/Versions/A/IOSurface -0x00007fff6b9d0000 /usr/lib/libDiagnosticMessagesClient.dylib -0x00007fff4ed8d000 /System/Library/PrivateFrameworks/DFRFoundation.framework/Versions/A/DFRFoundation -0x00007fff6cd70000 /usr/lib/libicucore.A.dylib -0x00007fff3332c000 /System/Library/Frameworks/AudioToolbox.framework/Versions/A/AudioToolbox -0x00007fff3340f000 /System/Library/Frameworks/AudioUnit.framework/Versions/A/AudioUnit -0x00007fff6c0a8000 /usr/lib/libauto.dylib -0x00007fff4ee2f000 /System/Library/PrivateFrameworks/DataDetectorsCore.framework/Versions/A/DataDetectorsCore -0x00007fff33bb5000 /System/Library/Frameworks/Carbon.framework/Versions/A/Frameworks/HIToolbox.framework/Versions/A/HIToolbox -0x00007fff40a98000 /System/Library/Frameworks/QuartzCore.framework/Versions/A/QuartzCore -0x00007fff33ef4000 /System/Library/Frameworks/Carbon.framework/Versions/A/Frameworks/SpeechRecognition.framework/Versions/A/SpeechRecognition -0x00007fff4e8a1000 /System/Library/PrivateFrameworks/CoreUI.framework/Versions/A/CoreUI -0x00007fff34483000 /System/Library/Frameworks/CoreAudio.framework/Versions/A/CoreAudio -0x00007fff372b3000 /System/Library/Frameworks/DiskArbitration.framework/Versions/A/DiskArbitration -0x00007fff5bf0f000 /System/Library/PrivateFrameworks/MultitouchSupport.framework/Versions/A/MultitouchSupport -0x00007fff6cc37000 /usr/lib/libenergytrace.dylib -0x00007fff37d2d000 /System/Library/Frameworks/IOKit.framework/Versions/A/IOKit -0x00007fff3631f000 /System/Library/Frameworks/CoreServices.framework/Versions/A/CoreServices -0x00007fff5e0c8000 /System/Library/PrivateFrameworks/PerformanceAnalysis.framework/Versions/A/PerformanceAnalysis -0x00007fff3facb000 /System/Library/Frameworks/OpenGL.framework/Versions/A/OpenGL -0x00007fff340a2000 /System/Library/Frameworks/ColorSync.framework/Versions/A/ColorSync -0x00007fff35a3b000 /System/Library/Frameworks/CoreImage.framework/Versions/A/CoreImage -0x00007fff36d92000 /System/Library/Frameworks/CoreText.framework/Versions/A/CoreText -0x00007fff37e63000 /System/Library/Frameworks/ImageIO.framework/Versions/A/ImageIO -0x00007fff6c18c000 /usr/lib/libc++.1.dylib -0x00007fff6c208000 /usr/lib/libcompression.dylib -0x00007fff6bd14000 /usr/lib/libMobileGestalt.dylib -0x00007fff65c0b000 /System/Library/PrivateFrameworks/TextureIO.framework/Versions/A/TextureIO -0x00007fff6c00b000 /usr/lib/libate.dylib -0x00007fff5a0c8000 /System/Library/PrivateFrameworks/InternationalSupport.framework/Versions/A/InternationalSupport -0x00007fff6eca3000 /usr/lib/system/libcache.dylib -0x00007fff6eca9000 /usr/lib/system/libcommonCrypto.dylib -0x00007fff6ecb5000 /usr/lib/system/libcompiler_rt.dylib -0x00007fff6ecbd000 /usr/lib/system/libcopyfile.dylib -0x00007fff6ecc7000 /usr/lib/system/libcorecrypto.dylib -0x00007fff6ee66000 /usr/lib/system/libdispatch.dylib -0x00007fff6eea7000 /usr/lib/system/libdyld.dylib -0x00007fff6eede000 /usr/lib/system/libkeymgr.dylib -0x00007fff6eeec000 /usr/lib/system/liblaunch.dylib -0x00007fff6eeed000 /usr/lib/system/libmacho.dylib -0x00007fff6eef3000 /usr/lib/system/libquarantine.dylib -0x00007fff6eef6000 /usr/lib/system/libremovefile.dylib -0x00007fff6eef8000 /usr/lib/system/libsystem_asl.dylib -0x00007fff6ef10000 /usr/lib/system/libsystem_blocks.dylib -0x00007fff6ef11000 /usr/lib/system/libsystem_c.dylib -0x00007fff6ef99000 /usr/lib/system/libsystem_configuration.dylib -0x00007fff6ef9d000 /usr/lib/system/libsystem_coreservices.dylib -0x00007fff6efa1000 /usr/lib/system/libsystem_darwin.dylib -0x00007fff6efaa000 /usr/lib/system/libsystem_dnssd.dylib -0x00007fff6efb2000 /usr/lib/system/libsystem_featureflags.dylib -0x00007fff6efb4000 /usr/lib/system/libsystem_info.dylib -0x00007fff6f02f000 /usr/lib/system/libsystem_m.dylib -0x00007fff6f077000 /usr/lib/system/libsystem_malloc.dylib -0x00007fff6f09f000 /usr/lib/system/libsystem_networkextension.dylib -0x00007fff6f0ad000 /usr/lib/system/libsystem_notify.dylib -0x00007fff6f0cb000 /usr/lib/system/libsystem_sandbox.dylib -0x00007fff6f0d0000 /usr/lib/system/libsystem_secinit.dylib -0x00007fff6f002000 /usr/lib/system/libsystem_kernel.dylib -0x00007fff6f0b7000 /usr/lib/system/libsystem_platform.dylib -0x00007fff6f0c0000 /usr/lib/system/libsystem_pthread.dylib -0x00007fff6f0d3000 /usr/lib/system/libsystem_symptoms.dylib -0x00007fff6f0db000 /usr/lib/system/libsystem_trace.dylib -0x00007fff6f0f3000 /usr/lib/system/libunwind.dylib -0x00007fff6f0f9000 /usr/lib/system/libxpc.dylib -0x00007fff6c1df000 /usr/lib/libc++abi.dylib -0x00007fff6cfe3000 /usr/lib/liblzma.5.dylib -0x00007fff6cc5e000 /usr/lib/libfakelink.dylib -0x00007fff6bf98000 /usr/lib/libarchive.2.dylib -0x00007fff4294f000 /System/Library/Frameworks/SystemConfiguration.framework/Versions/A/SystemConfiguration -0x00007fff6b903000 /usr/lib/libCRFSuite.dylib -0x00007fff337a6000 /System/Library/Frameworks/CFNetwork.framework/Versions/A/CFNetwork -0x00007fff6c16e000 /usr/lib/libbsm.0.dylib -0x00007fff6eedf000 /usr/lib/system/libkxld.dylib -0x00007fff48709000 /System/Library/PrivateFrameworks/AppleFSCompression.framework/Versions/A/AppleFSCompression -0x00007fff6c4f9000 /usr/lib/libcoretls.dylib -0x00007fff6c510000 /usr/lib/libcoretls_cfhelpers.dylib -0x00007fff6dd4c000 /usr/lib/libpam.2.dylib -0x00007fff6de81000 /usr/lib/libsqlite3.dylib -0x00007fff6e2c0000 /usr/lib/libxar.1.dylib -0x00007fff6c17f000 /usr/lib/libbz2.1.0.dylib -0x00007fff6cc7f000 /usr/lib/libiconv.2.dylib -0x00007fff6c1f5000 /usr/lib/libcharset.1.dylib -0x00007fff6d7e9000 /usr/lib/libnetwork.dylib -0x00007fff6dd53000 /usr/lib/libpcap.A.dylib -0x00007fff6bf4d000 /usr/lib/libapple_nghttp2.dylib -0x00007fff366d6000 /System/Library/Frameworks/CoreServices.framework/Versions/A/Frameworks/FSEvents.framework/Versions/A/FSEvents -0x00007fff363a6000 /System/Library/Frameworks/CoreServices.framework/Versions/A/Frameworks/CarbonCore.framework/Versions/A/CarbonCore -0x00007fff3691a000 /System/Library/Frameworks/CoreServices.framework/Versions/A/Frameworks/Metadata.framework/Versions/A/Metadata -0x00007fff369b3000 /System/Library/Frameworks/CoreServices.framework/Versions/A/Frameworks/OSServices.framework/Versions/A/OSServices -0x00007fff369e1000 /System/Library/Frameworks/CoreServices.framework/Versions/A/Frameworks/SearchKit.framework/Versions/A/SearchKit -0x00007fff36320000 /System/Library/Frameworks/CoreServices.framework/Versions/A/Frameworks/AE.framework/Versions/A/AE -0x00007fff366df000 /System/Library/Frameworks/CoreServices.framework/Versions/A/Frameworks/LaunchServices.framework/Versions/A/LaunchServices -0x00007fff36688000 /System/Library/Frameworks/CoreServices.framework/Versions/A/Frameworks/DictionaryServices.framework/Versions/A/DictionaryServices -0x00007fff36a49000 /System/Library/Frameworks/CoreServices.framework/Versions/A/Frameworks/SharedFileList.framework/Versions/A/SharedFileList -0x00007fff3b8ea000 /System/Library/Frameworks/NetFS.framework/Versions/A/NetFS -0x00007fff5c43f000 /System/Library/PrivateFrameworks/NetAuth.framework/Versions/A/NetAuth -0x00007fff68de3000 /System/Library/PrivateFrameworks/login.framework/Versions/A/Frameworks/loginsupport.framework/Versions/A/loginsupport -0x00007fff656d6000 /System/Library/PrivateFrameworks/TCC.framework/Versions/A/TCC -0x00007fff4d33b000 /System/Library/PrivateFrameworks/CoreNLP.framework/Versions/A/CoreNLP -0x00007fff5b93c000 /System/Library/PrivateFrameworks/MetadataUtilities.framework/Versions/A/MetadataUtilities -0x00007fff6d0bb000 /usr/lib/libmecabra.dylib -0x00007fff6d013000 /usr/lib/libmecab.dylib -0x00007fff6cc6f000 /usr/lib/libgermantok.dylib -0x00007fff6bf34000 /usr/lib/libThaiTokenizer.dylib -0x00007fff6b93a000 /usr/lib/libChineseTokenizer.dylib -0x00007fff30bd6000 /System/Library/Frameworks/Accelerate.framework/Versions/A/Frameworks/vImage.framework/Versions/A/vImage -0x00007fff32040000 /System/Library/Frameworks/Accelerate.framework/Versions/A/Frameworks/vecLib.framework/Versions/A/vecLib -0x00007fff31f7d000 /System/Library/Frameworks/Accelerate.framework/Versions/A/Frameworks/vecLib.framework/Versions/A/libvMisc.dylib -0x00007fff31da5000 /System/Library/Frameworks/Accelerate.framework/Versions/A/Frameworks/vecLib.framework/Versions/A/libvDSP.dylib -0x00007fff3122d000 /System/Library/Frameworks/Accelerate.framework/Versions/A/Frameworks/vecLib.framework/Versions/A/libBLAS.dylib -0x00007fff31969000 /System/Library/Frameworks/Accelerate.framework/Versions/A/Frameworks/vecLib.framework/Versions/A/libLAPACK.dylib -0x00007fff31d05000 /System/Library/Frameworks/Accelerate.framework/Versions/A/Frameworks/vecLib.framework/Versions/A/libLinearAlgebra.dylib -0x00007fff31d92000 /System/Library/Frameworks/Accelerate.framework/Versions/A/Frameworks/vecLib.framework/Versions/A/libSparseBLAS.dylib -0x00007fff31d1b000 /System/Library/Frameworks/Accelerate.framework/Versions/A/Frameworks/vecLib.framework/Versions/A/libQuadrature.dylib -0x00007fff31495000 /System/Library/Frameworks/Accelerate.framework/Versions/A/Frameworks/vecLib.framework/Versions/A/libBNNS.dylib -0x00007fff31d21000 /System/Library/Frameworks/Accelerate.framework/Versions/A/Frameworks/vecLib.framework/Versions/A/libSparse.dylib -0x00007fff5a4b0000 /System/Library/PrivateFrameworks/LanguageModeling.framework/Versions/A/LanguageModeling -0x00007fff4cceb000 /System/Library/PrivateFrameworks/CoreEmoji.framework/Versions/A/CoreEmoji -0x00007fff5a5ce000 /System/Library/PrivateFrameworks/LinguisticData.framework/Versions/A/LinguisticData -0x00007fff5a57f000 /System/Library/PrivateFrameworks/Lexicon.framework/Versions/A/Lexicon -0x00007fff6c1f6000 /usr/lib/libcmph.dylib -0x00007fff3e4d9000 /System/Library/Frameworks/OpenDirectory.framework/Versions/A/Frameworks/CFOpenDirectory.framework/Versions/A/CFOpenDirectory -0x00007fff3e4f6000 /System/Library/Frameworks/OpenDirectory.framework/Versions/A/OpenDirectory -0x00007fff46939000 /System/Library/PrivateFrameworks/APFS.framework/Versions/A/APFS -0x00007fff41be8000 /System/Library/Frameworks/SecurityFoundation.framework/Versions/A/SecurityFoundation -0x00007fff6e2bc000 /usr/lib/libutil.dylib -0x00007fff4e220000 /System/Library/PrivateFrameworks/CoreServicesStore.framework/Versions/A/CoreServicesStore -0x00007fff41c9f000 /System/Library/Frameworks/ServiceManagement.framework/Versions/A/ServiceManagement -0x00007fff49ed9000 /System/Library/PrivateFrameworks/BackgroundTaskManagement.framework/Versions/A/BackgroundTaskManagement -0x00007fff6e3b9000 /usr/lib/libxslt.1.dylib -0x00007fff48d7c000 /System/Library/PrivateFrameworks/AppleSystemInfo.framework/Versions/A/AppleSystemInfo -0x00007fff3807f000 /System/Library/Frameworks/ImageIO.framework/Versions/A/Resources/libJPEG.dylib -0x00007fff3833c000 /System/Library/Frameworks/ImageIO.framework/Versions/A/Resources/libTIFF.dylib -0x00007fff3831f000 /System/Library/Frameworks/ImageIO.framework/Versions/A/Resources/libPng.dylib -0x00007fff37fc1000 /System/Library/Frameworks/ImageIO.framework/Versions/A/Resources/libGIF.dylib -0x00007fff37fc5000 /System/Library/Frameworks/ImageIO.framework/Versions/A/Resources/libJP2.dylib -0x00007fff3833a000 /System/Library/Frameworks/ImageIO.framework/Versions/A/Resources/libRadiance.dylib -0x00007fff6cc38000 /usr/lib/libexpat.1.dylib -0x00007fff48864000 /System/Library/PrivateFrameworks/AppleJPEG.framework/Versions/A/AppleJPEG -0x00007fff51520000 /System/Library/PrivateFrameworks/FontServices.framework/libFontParser.dylib -0x00007fff67de5000 /System/Library/PrivateFrameworks/WatchdogClient.framework/Versions/A/WatchdogClient -0x00007fff59b0b000 /System/Library/PrivateFrameworks/IOAccelerator.framework/Versions/A/IOAccelerator -0x00007fff3a849000 /System/Library/Frameworks/MetalPerformanceShaders.framework/Versions/A/MetalPerformanceShaders -0x00007fff5629b000 /System/Library/PrivateFrameworks/GPUWrangler.framework/Versions/A/GPUWrangler -0x00007fff59b20000 /System/Library/PrivateFrameworks/IOPresentment.framework/Versions/A/IOPresentment -0x00007fff4eda0000 /System/Library/PrivateFrameworks/DSExternalDisplay.framework/Versions/A/DSExternalDisplay -0x00007fff3ee6a000 /System/Library/Frameworks/OpenGL.framework/Versions/A/Libraries/libCoreFSCache.dylib -0x00007fff3a595000 /System/Library/Frameworks/MetalPerformanceShaders.framework/Frameworks/MPSCore.framework/Versions/A/MPSCore -0x00007fff3a5d3000 /System/Library/Frameworks/MetalPerformanceShaders.framework/Frameworks/MPSImage.framework/Versions/A/MPSImage -0x00007fff3a69a000 /System/Library/Frameworks/MetalPerformanceShaders.framework/Frameworks/MPSNeuralNetwork.framework/Versions/A/MPSNeuralNetwork -0x00007fff3a65e000 /System/Library/Frameworks/MetalPerformanceShaders.framework/Frameworks/MPSMatrix.framework/Versions/A/MPSMatrix -0x00007fff3a7f9000 /System/Library/Frameworks/MetalPerformanceShaders.framework/Frameworks/MPSRayIntersector.framework/Versions/A/MPSRayIntersector -0x00007fff3a684000 /System/Library/Frameworks/MetalPerformanceShaders.framework/Frameworks/MPSNDArray.framework/Versions/A/MPSNDArray -0x00007fff5b989000 /System/Library/PrivateFrameworks/MetalTools.framework/Versions/A/MetalTools -0x00007fff47b13000 /System/Library/PrivateFrameworks/AggregateDictionary.framework/Versions/A/AggregateDictionary -0x00007fff4c733000 /System/Library/PrivateFrameworks/CoreAnalytics.framework/Versions/A/CoreAnalytics -0x00007fff48c9b000 /System/Library/PrivateFrameworks/AppleSauce.framework/Versions/A/AppleSauce -0x00007fff6bbf6000 /usr/lib/libIOReport.dylib -0x00007fff36f4a000 /System/Library/Frameworks/CoreVideo.framework/Versions/A/CoreVideo -0x00007fff57714000 /System/Library/PrivateFrameworks/GraphVisualizer.framework/Versions/A/GraphVisualizer -0x00007fff50a66000 /System/Library/PrivateFrameworks/FaceCore.framework/Versions/A/FaceCore -0x00007fff3e480000 /System/Library/Frameworks/OpenCL.framework/Versions/A/OpenCL -0x00007fff6ba18000 /usr/lib/libFosl_dynamic.dylib -0x00007fff5ce60000 /System/Library/PrivateFrameworks/OTSVG.framework/Versions/A/OTSVG -0x00007fff330ff000 /System/Library/Frameworks/ApplicationServices.framework/Versions/A/Frameworks/ATS.framework/Versions/A/Resources/libFontRegistry.dylib -0x00007fff516ef000 /System/Library/PrivateFrameworks/FontServices.framework/libhvf.dylib -0x00007fff3ee75000 /System/Library/Frameworks/OpenGL.framework/Versions/A/Libraries/libGFXShared.dylib -0x00007fff3f053000 /System/Library/Frameworks/OpenGL.framework/Versions/A/Libraries/libGLU.dylib -0x00007fff3ee7e000 /System/Library/Frameworks/OpenGL.framework/Versions/A/Libraries/libGL.dylib -0x00007fff3ee89000 /System/Library/Frameworks/OpenGL.framework/Versions/A/Libraries/libGLImage.dylib -0x00007fff3ee67000 /System/Library/Frameworks/OpenGL.framework/Versions/A/Libraries/libCVMSPluginSupport.dylib -0x00007fff3ee70000 /System/Library/Frameworks/OpenGL.framework/Versions/A/Libraries/libCoreVMClient.dylib -0x00007fff6d68a000 /usr/lib/libncurses.5.4.dylib -0x00007fff32ffb000 /System/Library/Frameworks/ApplicationServices.framework/Versions/A/Frameworks/ATS.framework/Versions/A/ATS -0x00007fff331c8000 /System/Library/Frameworks/ApplicationServices.framework/Versions/A/Frameworks/ColorSyncLegacy.framework/Versions/A/ColorSyncLegacy -0x00007fff33266000 /System/Library/Frameworks/ApplicationServices.framework/Versions/A/Frameworks/HIServices.framework/Versions/A/HIServices -0x00007fff332be000 /System/Library/Frameworks/ApplicationServices.framework/Versions/A/Frameworks/LangAnalysis.framework/Versions/A/LangAnalysis -0x00007fff332cd000 /System/Library/Frameworks/ApplicationServices.framework/Versions/A/Frameworks/PrintCore.framework/Versions/A/PrintCore -0x00007fff33313000 /System/Library/Frameworks/ApplicationServices.framework/Versions/A/Frameworks/QD.framework/Versions/A/QD -0x00007fff3331e000 /System/Library/Frameworks/ApplicationServices.framework/Versions/A/Frameworks/SpeechSynthesis.framework/Versions/A/SpeechSynthesis -0x00007fff33198000 /System/Library/Frameworks/ApplicationServices.framework/Versions/A/Frameworks/ATSUI.framework/Versions/A/ATSUI -0x00007fff6cace000 /usr/lib/libcups.2.dylib -0x00007fff398e4000 /System/Library/Frameworks/Kerberos.framework/Versions/A/Kerberos -0x00007fff37a26000 /System/Library/Frameworks/GSS.framework/Versions/A/GSS -0x00007fff6de0d000 /usr/lib/libresolv.9.dylib -0x00007fff578c1000 /System/Library/PrivateFrameworks/Heimdal.framework/Versions/A/Heimdal -0x00007fff398f7000 /System/Library/Frameworks/Kerberos.framework/Versions/A/Libraries/libHeimdalProxy.dylib -0x00007fff6cc75000 /usr/lib/libheimdal-asn1.dylib -0x00007fff4bbcb000 /System/Library/PrivateFrameworks/CommonAuth.framework/Versions/A/CommonAuth -0x00007fff48e19000 /System/Library/PrivateFrameworks/AssertionServices.framework/Versions/A/AssertionServices -0x00007fff499bc000 /System/Library/PrivateFrameworks/AudioToolboxCore.framework/Versions/A/AudioToolboxCore -0x00007fff68a93000 /System/Library/PrivateFrameworks/caulk.framework/Versions/A/caulk -0x00007fff49f85000 /System/Library/PrivateFrameworks/BaseBoard.framework/Versions/A/BaseBoard -0x00007fff610c3000 /System/Library/PrivateFrameworks/RunningBoardServices.framework/Versions/A/RunningBoardServices -0x00007fff5e0d4000 /System/Library/PrivateFrameworks/PersistentConnection.framework/Versions/A/PersistentConnection -0x00007fff60abc000 /System/Library/PrivateFrameworks/ProtocolBuffer.framework/Versions/A/ProtocolBuffer -0x00007fff4bbef000 /System/Library/PrivateFrameworks/CommonUtilities.framework/Versions/A/CommonUtilities -0x00007fff4a113000 /System/Library/PrivateFrameworks/Bom.framework/Versions/A/Bom -0x00007fff6b8c9000 /usr/lib/libAudioToolboxUtility.dylib -0x00007fff49ee3000 /System/Library/PrivateFrameworks/Backup.framework/Versions/A/Backup -0x00007fff4ecc1000 /System/Library/PrivateFrameworks/CrashReporterSupport.framework/Versions/A/CrashReporterSupport -0x00007fff62b55000 /System/Library/PrivateFrameworks/Sharing.framework/Versions/A/Sharing -0x00007fff48427000 /System/Library/PrivateFrameworks/Apple80211.framework/Versions/A/Apple80211 -0x00007fff49c00000 /System/Library/PrivateFrameworks/AuthKit.framework/Versions/A/AuthKit -0x00007fff4e9ce000 /System/Library/PrivateFrameworks/CoreUtils.framework/Versions/A/CoreUtils -0x00007fff36f8f000 /System/Library/Frameworks/CoreWLAN.framework/Versions/A/CoreWLAN -0x00007fff37bb3000 /System/Library/Frameworks/IOBluetooth.framework/Versions/A/IOBluetooth -0x00007fff5bc8e000 /System/Library/PrivateFrameworks/MobileKeyBag.framework/Versions/A/MobileKeyBag -0x00007fff4d7d3000 /System/Library/PrivateFrameworks/CorePhoneNumbers.framework/Versions/A/CorePhoneNumbers -0x00007fff48817000 /System/Library/PrivateFrameworks/AppleIDAuthSupport.framework/Versions/A/AppleIDAuthSupport -0x00007fff3b8f7000 /System/Library/Frameworks/Network.framework/Versions/A/Network -0x00007fff5a35c000 /System/Library/PrivateFrameworks/KeychainCircle.framework/Versions/A/KeychainCircle -0x00007fff349df000 /System/Library/Frameworks/CoreBluetooth.framework/Versions/A/CoreBluetooth -0x00007fff64bd4000 /System/Library/PrivateFrameworks/SpeechRecognitionCore.framework/Versions/A/SpeechRecognitionCore -0x000000010a400000 /Library/Java/JavaVirtualMachines/adoptopenjdk-8.jdk/Contents/Home/jre/lib/server/libjvm.dylib -0x00000001082b2000 /Library/Java/JavaVirtualMachines/adoptopenjdk-8.jdk/Contents/Home/jre/lib/libverify.dylib -0x00000001082c3000 /Library/Java/JavaVirtualMachines/adoptopenjdk-8.jdk/Contents/Home/jre/lib/libjava.dylib -0x00000001082f8000 /Library/Java/JavaVirtualMachines/adoptopenjdk-8.jdk/Contents/Home/jre/lib/libjdwp.dylib -0x0000000108334000 /Library/Java/JavaVirtualMachines/adoptopenjdk-8.jdk/Contents/Home/jre/lib/libnpt.dylib -0x000000010833d000 /Library/Java/JavaVirtualMachines/adoptopenjdk-8.jdk/Contents/Home/jre/lib/libinstrument.dylib -0x00000001083a4000 /Library/Java/JavaVirtualMachines/adoptopenjdk-8.jdk/Contents/Home/jre/lib/libzip.dylib -0x00000001083c1000 /Library/Java/JavaVirtualMachines/adoptopenjdk-8.jdk/Contents/Home/jre/lib/libdt_socket.dylib -0x000000012fb15000 /Library/Java/JavaVirtualMachines/adoptopenjdk-8.jdk/Contents/Home/jre/lib/libnio.dylib -0x000000012fb27000 /Library/Java/JavaVirtualMachines/adoptopenjdk-8.jdk/Contents/Home/jre/lib/libnet.dylib - -VM Arguments: -jvm_args: -agentlib:jdwp=transport=dt_socket,address=127.0.0.1:55276,suspend=y,server=n -ea -Dlog4j.configuration=file:/Users/tdur/Confluent/ksql/ksqldb-functional-tests/../ksqldb-test-util/src/main/resources/log4j.properties -verbose:gc -Xloggc:/Users/tdur/Confluent/ksql/ksqldb-functional-tests/target/gc.log -Djava.awt.headless=true -Didea.test.cyclic.buffer.size=1048576 -javaagent:/Users/tdur/Library/Caches/JetBrains/IntelliJIdea2020.3/groovyHotSwap/gragent.jar -javaagent:/Users/tdur/Library/Caches/JetBrains/IntelliJIdea2020.3/captureAgent/debugger-agent.jar -Dfile.encoding=UTF-8 -java_command: com.intellij.rt.junit.JUnitStarter -ideVersion5 -junit4 io.confluent.ksql.test.QueryAnonymizerTest,shouldAnonymizeIntegrationTestsCorrectly -java_class_path (initial): /Applications/IntelliJ IDEA.app/Contents/lib/idea_rt.jar:/Applications/IntelliJ IDEA.app/Contents/plugins/junit/lib/junit5-rt.jar:/Applications/IntelliJ IDEA.app/Contents/plugins/junit/lib/junit-rt.jar:/Library/Java/JavaVirtualMachines/adoptopenjdk-8.jdk/Contents/Home/jre/lib/charsets.jar:/Library/Java/JavaVirtualMachines/adoptopenjdk-8.jdk/Contents/Home/jre/lib/ext/cldrdata.jar:/Library/Java/JavaVirtualMachines/adoptopenjdk-8.jdk/Contents/Home/jre/lib/ext/dnsns.jar:/Library/Java/JavaVirtualMachines/adoptopenjdk-8.jdk/Contents/Home/jre/lib/ext/jaccess.jar:/Library/Java/JavaVirtualMachines/adoptopenjdk-8.jdk/Contents/Home/jre/lib/ext/localedata.jar:/Library/Java/JavaVirtualMachines/adoptopenjdk-8.jdk/Contents/Home/jre/lib/ext/nashorn.jar:/Library/Java/JavaVirtualMachines/adoptopenjdk-8.jdk/Contents/Home/jre/lib/ext/sunec.jar:/Library/Java/JavaVirtualMachines/adoptopenjdk-8.jdk/Contents/Home/jre/lib/ext/sunjce_provider.jar:/Library/Java/JavaVirtualMachines/adoptopenjdk-8.jdk/Contents/Home/jre/lib/ext/sunpkcs11.jar:/Library/Java/JavaVirtualMachines/adoptopenjdk-8.jdk/Contents/Home/jre/lib/ext/zipfs.jar:/Library/Java/JavaVirtualMachines/adoptopenjdk-8.jdk/Contents/Home/jre/lib/jce.jar:/Library/Java/JavaVirtualMachines/adoptopenjdk-8.jdk/Contents/Home/jre/lib/jfr.jar:/Library/Java/JavaVirtualMachines/adoptopenjdk-8.jdk/Contents/Home/jre/lib/jsse.jar:/Library/Java/JavaVirtualMachines/adoptopenjdk-8.jdk/Contents/Home/jre/lib/management-agent.jar:/Library/Java/JavaVirtualMachines/adoptopenjdk-8.jdk/Contents/Home/jre/lib/resources.jar:/Library/Java/JavaVirtualMachines/adoptopenjdk-8.jdk/Contents/Home/jre/lib/rt.jar:/Library/Java/JavaVirtualMachines/adoptopenjdk-8.jdk/Contents/Home/lib/dt.jar:/Library/Java/JavaVirtualMachines/adoptopenjdk-8.jdk/Contents/Home/lib/jconsole.jar:/Library/Java/JavaVirtualMachines/adoptopenjdk-8.jdk/Contents/Home/lib/sa-jdi.jar:/Library/Java/JavaVirtualMachines/adoptopenjdk-8.jdk/Contents/Home/lib/tools.jar:/Users/tdur/ -Launcher Type: SUN_STANDARD - -Environment Variables: -JAVA_HOME=/Library/Java/JavaVirtualMachines/adoptopenjdk-8.jdk/Contents/Home -CLASSPATH=.:/usr/local/lib/antlr-4.9-complete.jar: -PATH=/Users/tdur/.pyenv/versions/3.9.1/bin:/usr/local/Cellar/pyenv/1.2.22/libexec:/usr/local/Cellar/pyenv/1.2.22/plugins/python-build/bin:/Users/tdur/.goenv/shims:/Users/tdur/.goenv/bin:/Users/tdur/LocalCP/confluent-5.5.4/bin:/Users/tdur/.jenv/shims:/usr/local/Caskroom/google-cloud-sdk/latest/google-cloud-sdk/bin:/Users/tdur/.pyenv/shims:/Users/tdur/.nvm/versions/node/v12.20.1/bin:/usr/local/bin:/usr/bin:/bin:/usr/sbin:/sbin:/Users/tdur/:/usr/local/go/bin:/Users/tdur/git/go/src/github.com/confluentinc/cc-dotfiles/bin/cloud-ga:/Users/tdur/git/go/src/github.com/confluentinc/cc-dotfiles/bin/connect-ga:/Users/tdur/git/go/src/github.com/confluentinc/cc-dotfiles/bin/auth-ga:/Users/tdur/git/go/src/github.com/confluentinc/cc-dotfiles/bin/kafka-ga:/Users/tdur/git/go/1.12.7/bin -SHELL=/bin/zsh - -Signal Handlers: -SIGSEGV: [libjvm.dylib+0x597a91], sa_mask[0]=11111111011111110111111111111111, sa_flags=SA_ONSTACK|SA_RESTART|SA_SIGINFO -SIGBUS: [libjvm.dylib+0x597a91], sa_mask[0]=11111111011111110111111111111111, sa_flags=SA_RESTART|SA_SIGINFO -SIGFPE: [libjvm.dylib+0x48ba83], sa_mask[0]=11111111011111110111111111111111, sa_flags=SA_RESTART|SA_SIGINFO -SIGPIPE: [libjvm.dylib+0x48ba83], sa_mask[0]=11111111011111110111111111111111, sa_flags=SA_RESTART|SA_SIGINFO -SIGXFSZ: [libjvm.dylib+0x48ba83], sa_mask[0]=11111111011111110111111111111111, sa_flags=SA_RESTART|SA_SIGINFO -SIGILL: [libjvm.dylib+0x48ba83], sa_mask[0]=11111111011111110111111111111111, sa_flags=SA_RESTART|SA_SIGINFO -SIGUSR1: SIG_DFL, sa_mask[0]=00000000000000000000000000000000, sa_flags=none -SIGUSR2: [libjvm.dylib+0x48c37a], sa_mask[0]=00100000000000000000000000000000, sa_flags=SA_RESTART|SA_SIGINFO -SIGHUP: [libjvm.dylib+0x48a579], sa_mask[0]=11111111011111110111111111111111, sa_flags=SA_RESTART|SA_SIGINFO -SIGINT: [libjvm.dylib+0x48a579], sa_mask[0]=11111111011111110111111111111111, sa_flags=SA_RESTART|SA_SIGINFO -SIGTERM: [libjvm.dylib+0x48a579], sa_mask[0]=11111111011111110111111111111111, sa_flags=SA_RESTART|SA_SIGINFO -SIGQUIT: [libjvm.dylib+0x48a579], sa_mask[0]=11111111011111110111111111111111, sa_flags=SA_RESTART|SA_SIGINFO - - ---------------- S Y S T E M --------------- - -OS:Bsduname:Darwin 19.6.0 Darwin Kernel Version 19.6.0: Mon Apr 12 20:57:45 PDT 2021; root:xnu-6153.141.28.1~1/RELEASE_X86_64 x86_64 -rlimit: STACK 8192k, CORE 0k, NPROC 5568, NOFILE 10240, AS infinity -load average:2.47 2.80 2.75 - -CPU:total 16 (initial active 16) (8 cores per cpu, 2 threads per core) family 6 model 158 stepping 13, cmov, cx8, fxsr, mmx, sse, sse2, sse3, ssse3, sse4.1, sse4.2, popcnt, avx, avx2, aes, clmul, erms, 3dnowpref, lzcnt, ht, tsc, tscinvbit, bmi1, bmi2, adx - -Memory: 4k page, physical 33554432k(2360384k free) - -/proc/meminfo: - - -vm_info: OpenJDK 64-Bit Server VM (25.282-b08) for bsd-amd64 JRE (1.8.0_282-b08), built on Jan 20 2021 11:47:40 by "jenkins" with gcc 4.2.1 Compatible Apple LLVM 10.0.1 (clang-1001.0.46.4) - -time: Wed May 5 12:42:49 2021 -timezone: BST -elapsed time: 2066.227709 seconds (0d 0h 34m 26s) - diff --git a/ksqldb-functional-tests/src/main/java/io/confluent/ksql/test/model/PostConditionsNode.java b/ksqldb-functional-tests/src/main/java/io/confluent/ksql/test/model/PostConditionsNode.java index 1468baf4c805..5bd816c68dca 100644 --- a/ksqldb-functional-tests/src/main/java/io/confluent/ksql/test/model/PostConditionsNode.java +++ b/ksqldb-functional-tests/src/main/java/io/confluent/ksql/test/model/PostConditionsNode.java @@ -192,8 +192,8 @@ public PostTopicNode( this.keyFormat = requireNonNull(keyFormat, "KeyFormat"); this.valueFormat = requireNonNull(valueFormat, "valueFormat"); this.partitions = requireNonNull(partitions, "partitions"); - this.keySchema = requireNonNull(keySchema, "keySchema"); - this.valueSchema = requireNonNull(valueSchema, "valueSchema"); + this.keySchema = keySchema; + this.valueSchema = valueSchema; if (this.name.isEmpty()) { throw new InvalidFieldException("name", "empty or missing"); @@ -218,6 +218,7 @@ public void describeTo(final Description description) { } // CHECKSTYLE_RULES.OFF: BooleanExpressionComplexity + // CHECKSTYLE_RULES.OFF: CyclomaticComplexity @Override public boolean matches(final Object item) { if (!(item instanceof PostTopicNode)) { @@ -229,11 +230,13 @@ public boolean matches(final Object item) { && Objects.equals(keyFormat, that.keyFormat) && Objects.equals(valueFormat, that.valueFormat) && (!partitions.isPresent() || partitions.equals(that.partitions)) - && (keySchema instanceof NullNode || keySchema.equals(that.keySchema)) - && (valueSchema instanceof NullNode + && (keySchema == null || keySchema instanceof NullNode + || keySchema.equals(that.keySchema)) + && (valueSchema == null || valueSchema instanceof NullNode || valueSchema.equals(that.valueSchema)); } // CHECKSTYLE_RULES.ON: BooleanExpressionComplexity + // CHECKSTYLE_RULES.ON: CyclomaticComplexity }; } diff --git a/ksqldb-functional-tests/src/main/java/io/confluent/ksql/test/model/TopicNode.java b/ksqldb-functional-tests/src/main/java/io/confluent/ksql/test/model/TopicNode.java index 69190fea1e10..d9728ceb5b64 100644 --- a/ksqldb-functional-tests/src/main/java/io/confluent/ksql/test/model/TopicNode.java +++ b/ksqldb-functional-tests/src/main/java/io/confluent/ksql/test/model/TopicNode.java @@ -15,8 +15,6 @@ package io.confluent.ksql.test.model; -import static java.util.Objects.requireNonNull; - import com.fasterxml.jackson.annotation.JsonInclude; import com.fasterxml.jackson.annotation.JsonInclude.Include; import com.fasterxml.jackson.annotation.JsonProperty; @@ -59,9 +57,10 @@ public TopicNode( @JsonProperty("keySerdeFeatures") final SerdeFeatures keySerdeFeatures, @JsonProperty("valueSerdeFeatures") final SerdeFeatures valueSerdeFeatures ) { + this.name = name == null ? "" : name; - this.keySchema = requireNonNull(keySchema, "keySchema"); - this.valueSchema = requireNonNull(valueSchema, "valueSchema"); + this.keySchema = keySchema; + this.valueSchema = valueSchema; this.keyFormat = keyFormat; this.valueFormat = valueFormat; this.numPartitions = numPartitions == null ? 1 : numPartitions; diff --git a/ksqldb-functional-tests/src/main/java/io/confluent/ksql/test/serde/ConnectSerdeSupplier.java b/ksqldb-functional-tests/src/main/java/io/confluent/ksql/test/serde/ConnectSerdeSupplier.java index 4232a70d23d1..b18fb37ce77d 100644 --- a/ksqldb-functional-tests/src/main/java/io/confluent/ksql/test/serde/ConnectSerdeSupplier.java +++ b/ksqldb-functional-tests/src/main/java/io/confluent/ksql/test/serde/ConnectSerdeSupplier.java @@ -19,28 +19,14 @@ import io.confluent.kafka.schemaregistry.ParsedSchema; import io.confluent.kafka.schemaregistry.client.SchemaRegistryClient; import io.confluent.kafka.serializers.AbstractKafkaSchemaSerDeConfig; -import io.confluent.ksql.serde.SerdeUtils; -import io.confluent.ksql.test.TestFrameworkException; -import io.confluent.ksql.util.DecimalUtil; import io.confluent.ksql.util.KsqlConstants; import io.confluent.ksql.util.KsqlException; -import java.math.BigDecimal; -import java.nio.ByteBuffer; -import java.util.Date; -import java.util.HashMap; -import java.util.List; -import java.util.Map; import java.util.Objects; import java.util.function.Function; -import java.util.stream.Collectors; import org.apache.kafka.common.serialization.Deserializer; import org.apache.kafka.common.serialization.Serializer; -import org.apache.kafka.connect.data.Field; import org.apache.kafka.connect.data.Schema; import org.apache.kafka.connect.data.SchemaAndValue; -import org.apache.kafka.connect.data.Struct; -import org.apache.kafka.connect.data.Time; -import org.apache.kafka.connect.data.Timestamp; import org.apache.kafka.connect.storage.Converter; /** @@ -113,99 +99,9 @@ public byte[] serialize(final String topic, final Object spec) { return converter.fromConnectData( topic, connectSchema, - specToConnect(spec, connectSchema) + SpecToConnectConverter.specToConnect(spec, connectSchema) ); } - - // CHECKSTYLE_RULES.OFF: CyclomaticComplexity - private Object specToConnect(final Object spec, final Schema schema) { - // CHECKSTYLE_RULES.ON: CyclomaticComplexity - if (spec == null) { - return null; - } - - switch (schema.type()) { - case INT32: - final Integer intVal = Integer.valueOf(spec.toString()); - if (Time.LOGICAL_NAME.equals(schema.name())) { - return new java.sql.Time(intVal); - } - if (org.apache.kafka.connect.data.Date.LOGICAL_NAME.equals(schema.name())) { - return SerdeUtils.getDateFromEpochDays(intVal); - } - return intVal; - case INT64: - final Long longVal = Long.valueOf(spec.toString()); - if (Timestamp.LOGICAL_NAME.equals(schema.name())) { - return new java.sql.Timestamp(longVal); - } - return longVal; - case FLOAT32: - return Float.valueOf(spec.toString()); - case FLOAT64: - return Double.valueOf(spec.toString()); - case BOOLEAN: - return Boolean.valueOf(spec.toString()); - case STRING: - return spec.toString(); - case ARRAY: - return ((List) spec) - .stream() - .map(el -> specToConnect(el, schema.valueSchema())) - .collect(Collectors.toList()); - case MAP: - return ((Map) spec) - .entrySet() - .stream() - // cannot use Collectors.toMap due to JDK bug: - // https://bugs.openjdk.java.net/browse/JDK-8148463 - .collect( - HashMap::new, - ((map, v) -> map.put( - specToConnect(v.getKey(), schema.keySchema()), - specToConnect(v.getValue(), schema.valueSchema()))), - HashMap::putAll - ); - case STRUCT: - final Map caseInsensitiveFieldMap = schema.fields() - .stream() - .collect(Collectors.toMap( - f -> f.name().toUpperCase(), - Field::name - )); - - final Struct struct = new Struct(schema); - ((Map) spec) - .forEach((key, value) -> { - final String realKey = caseInsensitiveFieldMap.get(key.toString().toUpperCase()); - if (realKey != null) { - struct.put(realKey, specToConnect(value, schema.field(realKey).schema())); - } - }); - return struct; - case BYTES: - if (DecimalUtil.isDecimal(schema)) { - if (spec instanceof BigDecimal) { - return DecimalUtil.ensureFit((BigDecimal) spec, schema); - } - - if (spec instanceof String) { - // Supported for legacy reasons... - return DecimalUtil.cast( - (String) spec, - DecimalUtil.precision(schema), - DecimalUtil.scale(schema)); - } - - throw new TestFrameworkException("DECIMAL type requires JSON number in test data"); - } else { - return spec; - } - default: - throw new RuntimeException( - "This test does not support the data type yet: " + schema.type()); - } - } } private class SpecDeserializer implements Deserializer { @@ -227,78 +123,9 @@ public Object deserialize(final String topic, final byte[] bytes) { } final SchemaAndValue schemaAndValue = converter.toConnectData(topic, bytes); - return connectToSpec(schemaAndValue.value(), schemaAndValue.schema(), false); + return SpecToConnectConverter.connectToSpec(schemaAndValue.value(), + schemaAndValue.schema(), + false); } - - // CHECKSTYLE_RULES.OFF: CyclomaticComplexity - private Object connectToSpec( - final Object data, - final Schema schema, - final boolean toUpper - ) { - // CHECKSTYLE_RULES.ON: CyclomaticComplexity - if (data == null) { - return null; - } - - switch (schema.type()) { - case INT64: - if (Timestamp.LOGICAL_NAME.equals(schema.name())) { - return Timestamp.fromLogical(schema, (Date) data); - } - return data; - case INT32: - if (Time.LOGICAL_NAME.equals(schema.name())) { - return Time.fromLogical(schema, (Date) data); - } - if (org.apache.kafka.connect.data.Date.LOGICAL_NAME.equals(schema.name())) { - return org.apache.kafka.connect.data.Date.fromLogical(schema, (Date) data); - } - return data; - case FLOAT32: - case FLOAT64: - case BOOLEAN: - return data; - case STRING: - return data.toString(); - case ARRAY: - return ((List) data) - .stream() - .map(v -> connectToSpec(v, schema.valueSchema(), toUpper)) - .collect(Collectors.toList()); - case MAP: - final Map map = new HashMap<>(); - ((Map) data) - .forEach((k, v) -> map.put( - k.toString(), - connectToSpec(v, schema.valueSchema(), toUpper))); - return map; - case STRUCT: - final Map recordSpec = new HashMap<>(); - schema.fields() - .forEach(f -> recordSpec.put( - toUpper ? f.name().toUpperCase() : f.name(), - connectToSpec(((Struct) data).get(f.name()), f.schema(), toUpper))); - return recordSpec; - case BYTES: - if (DecimalUtil.isDecimal(schema)) { - if (data instanceof BigDecimal) { - return data; - } - throw new RuntimeException("Unexpected BYTES type " + schema.name()); - } else { - if (data instanceof byte[]) { - return ByteBuffer.wrap((byte[]) data); - } else { - return data; - } - } - default: - throw new RuntimeException("Test cannot handle data of type: " + schema.type()); - } - } - } - - } diff --git a/ksqldb-functional-tests/src/main/java/io/confluent/ksql/test/serde/SpecToConnectConverter.java b/ksqldb-functional-tests/src/main/java/io/confluent/ksql/test/serde/SpecToConnectConverter.java new file mode 100644 index 000000000000..f63924db3ece --- /dev/null +++ b/ksqldb-functional-tests/src/main/java/io/confluent/ksql/test/serde/SpecToConnectConverter.java @@ -0,0 +1,201 @@ +/* + * Copyright 2022 Confluent Inc. + * + * Licensed under the Confluent Community License (the "License"); you may not use + * this file except in compliance with the License. You may obtain a copy of the + * License at + * + * http://www.confluent.io/confluent-community-license + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, WITHOUT + * WARRANTIES OF ANY KIND, either express or implied. See the License for the + * specific language governing permissions and limitations under the License. + */ + +package io.confluent.ksql.test.serde; + +import io.confluent.ksql.serde.SerdeUtils; +import io.confluent.ksql.test.TestFrameworkException; +import io.confluent.ksql.util.BytesUtils; +import io.confluent.ksql.util.BytesUtils.Encoding; +import io.confluent.ksql.util.DecimalUtil; +import java.math.BigDecimal; +import java.nio.ByteBuffer; +import java.util.Date; +import java.util.HashMap; +import java.util.List; +import java.util.Map; +import java.util.stream.Collectors; +import org.apache.kafka.connect.data.Field; +import org.apache.kafka.connect.data.Schema; +import org.apache.kafka.connect.data.Struct; +import org.apache.kafka.connect.data.Time; +import org.apache.kafka.connect.data.Timestamp; + +public final class SpecToConnectConverter { + + private SpecToConnectConverter() { + } + + // CHECKSTYLE_RULES.OFF: CyclomaticComplexity + public static Object specToConnect(final Object spec, final Schema schema) { + // CHECKSTYLE_RULES.ON: CyclomaticComplexity + if (spec == null) { + return null; + } + + switch (schema.type()) { + case INT32: + final Integer intVal = Integer.valueOf(spec.toString()); + if (Time.LOGICAL_NAME.equals(schema.name())) { + return new java.sql.Time(intVal); + } + if (org.apache.kafka.connect.data.Date.LOGICAL_NAME.equals(schema.name())) { + return SerdeUtils.getDateFromEpochDays(intVal); + } + return intVal; + case INT64: + final Long longVal = Long.valueOf(spec.toString()); + if (Timestamp.LOGICAL_NAME.equals(schema.name())) { + return new java.sql.Timestamp(longVal); + } + return longVal; + case FLOAT32: + return Float.valueOf(spec.toString()); + case FLOAT64: + return Double.valueOf(spec.toString()); + case BOOLEAN: + return Boolean.valueOf(spec.toString()); + case STRING: + return spec.toString(); + case ARRAY: + return ((List) spec) + .stream() + .map(el -> specToConnect(el, schema.valueSchema())) + .collect(Collectors.toList()); + case MAP: + return ((Map) spec) + .entrySet() + .stream() + // cannot use Collectors.toMap due to JDK bug: + // https://bugs.openjdk.java.net/browse/JDK-8148463 + .collect( + HashMap::new, + ((map, v) -> map.put( + specToConnect(v.getKey(), schema.keySchema()), + specToConnect(v.getValue(), schema.valueSchema()))), + HashMap::putAll + ); + case STRUCT: + final Map caseInsensitiveFieldMap = schema.fields() + .stream() + .collect(Collectors.toMap( + f -> f.name().toUpperCase(), + Field::name + )); + + final Struct struct = new Struct(schema); + ((Map) spec) + .forEach((key, value) -> { + final String realKey = caseInsensitiveFieldMap.get(key.toString().toUpperCase()); + if (realKey != null) { + struct.put(realKey, specToConnect(value, schema.field(realKey).schema())); + } + }); + return struct; + case BYTES: + if (DecimalUtil.isDecimal(schema)) { + if (spec instanceof BigDecimal) { + return DecimalUtil.ensureFit((BigDecimal) spec, schema); + } + + if (spec instanceof String) { + // Supported for legacy reasons... + return DecimalUtil.cast( + (String) spec, + DecimalUtil.precision(schema), + DecimalUtil.scale(schema)); + } + + throw new TestFrameworkException("DECIMAL type requires JSON number in test data"); + } else if (spec instanceof String) { + // covers PROTOBUF_NOSR + return BytesUtils.decode((String) spec, Encoding.BASE64); + } else { + return spec; + } + default: + throw new RuntimeException( + "This test does not support the data type yet: " + schema.type()); + } + } + + // CHECKSTYLE_RULES.OFF: CyclomaticComplexity + public static Object connectToSpec( + final Object data, + final Schema schema, + final boolean toUpper + ) { + // CHECKSTYLE_RULES.ON: CyclomaticComplexity + if (data == null) { + return null; + } + + switch (schema.type()) { + case INT64: + if (Timestamp.LOGICAL_NAME.equals(schema.name())) { + return Timestamp.fromLogical(schema, (Date) data); + } + return data; + case INT32: + if (Time.LOGICAL_NAME.equals(schema.name())) { + return Time.fromLogical(schema, (Date) data); + } + if (org.apache.kafka.connect.data.Date.LOGICAL_NAME.equals(schema.name())) { + return org.apache.kafka.connect.data.Date.fromLogical(schema, (Date) data); + } + return data; + case FLOAT32: + case FLOAT64: + case BOOLEAN: + return data; + case STRING: + return data.toString(); + case ARRAY: + return ((List) data) + .stream() + .map(v -> connectToSpec(v, schema.valueSchema(), toUpper)) + .collect(Collectors.toList()); + case MAP: + final Map map = new HashMap<>(); + ((Map) data) + .forEach((k, v) -> map.put( + k.toString(), + connectToSpec(v, schema.valueSchema(), toUpper))); + return map; + case STRUCT: + final Map recordSpec = new HashMap<>(); + schema.fields() + .forEach(f -> recordSpec.put( + toUpper ? f.name().toUpperCase() : f.name(), + connectToSpec(((Struct) data).get(f.name()), f.schema(), toUpper))); + return recordSpec; + case BYTES: + if (DecimalUtil.isDecimal(schema)) { + if (data instanceof BigDecimal) { + return data; + } + throw new RuntimeException("Unexpected BYTES type " + schema.name()); + } else { + if (data instanceof byte[]) { + return ByteBuffer.wrap((byte[]) data); + } else { + return data; + } + } + default: + throw new RuntimeException("Test cannot handle data of type: " + schema.type()); + } + } +} diff --git a/ksqldb-functional-tests/src/main/java/io/confluent/ksql/test/serde/protobuf/ValueSpecProtobufNoSRSerdeSupplier.java b/ksqldb-functional-tests/src/main/java/io/confluent/ksql/test/serde/protobuf/ValueSpecProtobufNoSRSerdeSupplier.java new file mode 100644 index 000000000000..0c787c814cf3 --- /dev/null +++ b/ksqldb-functional-tests/src/main/java/io/confluent/ksql/test/serde/protobuf/ValueSpecProtobufNoSRSerdeSupplier.java @@ -0,0 +1,110 @@ +/* + * Copyright 2022 Confluent Inc. + * + * Licensed under the Confluent Community License (the "License"); you may not use + * this file except in compliance with the License. You may obtain a copy of the + * License at + * + * http://www.confluent.io/confluent-community-license + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, WITHOUT + * WARRANTIES OF ANY KIND, either express or implied. See the License for the + * specific language governing permissions and limitations under the License. + */ + +package io.confluent.ksql.test.serde.protobuf; + +import io.confluent.kafka.schemaregistry.client.SchemaRegistryClient; +import io.confluent.ksql.schema.ksql.LogicalSchema; +import io.confluent.ksql.serde.connect.ConnectSchemas; +import io.confluent.ksql.serde.protobuf.ProtobufNoSRConverter; +import io.confluent.ksql.test.serde.SerdeSupplier; +import io.confluent.ksql.test.serde.SpecToConnectConverter; +import java.util.Map; +import org.apache.kafka.common.serialization.Deserializer; +import org.apache.kafka.common.serialization.Serializer; +import org.apache.kafka.connect.data.Schema; +import org.apache.kafka.connect.data.SchemaAndValue; + +public class ValueSpecProtobufNoSRSerdeSupplier implements SerdeSupplier { + + private final Schema keySchema; + private final Schema valueSchema; + private final ProtobufNoSRConverter keyConverter; + private final ProtobufNoSRConverter valueConverter; + + public ValueSpecProtobufNoSRSerdeSupplier(final LogicalSchema schema, + final Map properties) { + this.keySchema = ConnectSchemas.columnsToConnectSchema(schema.key()); + this.valueSchema = ConnectSchemas.columnsToConnectSchema(schema.value()); + this.valueConverter = new ProtobufNoSRConverter(this.valueSchema); + this.keyConverter = new ProtobufNoSRConverter(this.keySchema); + + keyConverter.configure(properties, true); + valueConverter.configure(properties, false); + } + + @Override + public Serializer getSerializer(final SchemaRegistryClient schemaRegistryClient, + final boolean isKey) { + if (isKey) { + return new ValueSpecProtobufNoSRSerializer(keyConverter, keySchema); + } + return new ValueSpecProtobufNoSRSerializer(valueConverter, valueSchema); + } + + @Override + public Deserializer getDeserializer(final SchemaRegistryClient schemaRegistryClient, + final boolean isKey) { + if (isKey) { + return new ValueSpecProtobufNoSRDeserializer(keyConverter); + } + return new ValueSpecProtobufNoSRDeserializer(valueConverter); + } + + private static final class ValueSpecProtobufNoSRSerializer implements Serializer { + private final Schema schema; + private final ProtobufNoSRConverter converter; + + ValueSpecProtobufNoSRSerializer(final ProtobufNoSRConverter converter, + final Schema schema) { + this.converter = converter; + this.schema = schema; + } + + @Override + public void configure(final Map configs, final boolean isKey) { + } + + @Override + public byte[] serialize(final String topic, final Object o) { + final byte[] bytes = converter.fromConnectData(topic, + schema, + SpecToConnectConverter.specToConnect(o, schema)); + return bytes; + } + } + + private static final class ValueSpecProtobufNoSRDeserializer implements Deserializer { + private final ProtobufNoSRConverter converter; + + ValueSpecProtobufNoSRDeserializer(final ProtobufNoSRConverter converter) { + this.converter = converter; + } + + @Override + public void configure(final Map configs, final boolean isKey) { + } + + + @Override + public Object deserialize(final String s, final byte[] bytes) { + final SchemaAndValue schemaAndValue = converter.toConnectData(s, bytes); + return SpecToConnectConverter.connectToSpec(schemaAndValue.value(), + schemaAndValue.schema(), + false); + } + } + +} diff --git a/ksqldb-functional-tests/src/main/java/io/confluent/ksql/test/utils/SerdeUtil.java b/ksqldb-functional-tests/src/main/java/io/confluent/ksql/test/utils/SerdeUtil.java index e676ce1bf485..86db18bd8b31 100644 --- a/ksqldb-functional-tests/src/main/java/io/confluent/ksql/test/utils/SerdeUtil.java +++ b/ksqldb-functional-tests/src/main/java/io/confluent/ksql/test/utils/SerdeUtil.java @@ -38,12 +38,14 @@ import io.confluent.ksql.serde.kafka.KafkaFormat; import io.confluent.ksql.serde.none.NoneFormat; import io.confluent.ksql.serde.protobuf.ProtobufFormat; +import io.confluent.ksql.serde.protobuf.ProtobufNoSRFormat; import io.confluent.ksql.serde.protobuf.ProtobufProperties; import io.confluent.ksql.test.serde.SerdeSupplier; import io.confluent.ksql.test.serde.avro.ValueSpecAvroSerdeSupplier; import io.confluent.ksql.test.serde.json.ValueSpecJsonSerdeSupplier; import io.confluent.ksql.test.serde.kafka.KafkaSerdeSupplier; import io.confluent.ksql.test.serde.none.NoneSerdeSupplier; +import io.confluent.ksql.test.serde.protobuf.ValueSpecProtobufNoSRSerdeSupplier; import io.confluent.ksql.test.serde.protobuf.ValueSpecProtobufSerdeSupplier; import io.confluent.ksql.test.serde.string.StringSerdeSupplier; import io.confluent.ksql.test.tools.exceptions.InvalidFieldException; @@ -77,6 +79,8 @@ public static SerdeSupplier getSerdeSupplier( case ProtobufFormat.NAME: return new ValueSpecProtobufSerdeSupplier( new ProtobufProperties(formatInfo.getProperties())); + case ProtobufNoSRFormat.NAME: + return new ValueSpecProtobufNoSRSerdeSupplier(schema, formatInfo.getProperties()); case JsonFormat.NAME: return new ValueSpecJsonSerdeSupplier(false, properties); case JsonSchemaFormat.NAME: return new ValueSpecJsonSerdeSupplier(true, properties); case DelimitedFormat.NAME: return new StringSerdeSupplier(); @@ -88,7 +92,7 @@ public static SerdeSupplier getSerdeSupplier( } public static Optional buildSchema(final JsonNode schema, final String format) { - if (schema instanceof NullNode) { + if (schema == null || schema instanceof NullNode) { return Optional.empty(); } diff --git a/ksqldb-functional-tests/src/test/resources/historical_plans/bytes_-_PROTOBUF_NOSR_in_out/7.3.0_1651069522837/plan.json b/ksqldb-functional-tests/src/test/resources/historical_plans/bytes_-_PROTOBUF_NOSR_in_out/7.3.0_1651069522837/plan.json new file mode 100644 index 000000000000..dacfe229593e --- /dev/null +++ b/ksqldb-functional-tests/src/test/resources/historical_plans/bytes_-_PROTOBUF_NOSR_in_out/7.3.0_1651069522837/plan.json @@ -0,0 +1,203 @@ +{ + "plan" : [ { + "@type" : "ksqlPlanV1", + "statementText" : "CREATE STREAM TEST (ID STRING KEY, B BYTES) WITH (KAFKA_TOPIC='test', KEY_FORMAT='KAFKA', VALUE_FORMAT='PROTOBUF_NOSR');", + "ddlCommand" : { + "@type" : "createStreamV1", + "sourceName" : "TEST", + "schema" : "`ID` STRING KEY, `B` BYTES", + "timestampColumn" : null, + "topicName" : "test", + "formats" : { + "keyFormat" : { + "format" : "KAFKA", + "properties" : { } + }, + "valueFormat" : { + "format" : "PROTOBUF_NOSR", + "properties" : { } + } + }, + "windowInfo" : null, + "orReplace" : false, + "isSource" : false + }, + "queryPlan" : null + }, { + "@type" : "ksqlPlanV1", + "statementText" : "CREATE STREAM TEST2 AS SELECT *\nFROM TEST TEST\nEMIT CHANGES", + "ddlCommand" : { + "@type" : "createStreamV1", + "sourceName" : "TEST2", + "schema" : "`ID` STRING KEY, `B` BYTES", + "timestampColumn" : null, + "topicName" : "TEST2", + "formats" : { + "keyFormat" : { + "format" : "KAFKA", + "properties" : { } + }, + "valueFormat" : { + "format" : "PROTOBUF_NOSR", + "properties" : { } + } + }, + "windowInfo" : null, + "orReplace" : false, + "isSource" : false + }, + "queryPlan" : { + "sources" : [ "TEST" ], + "sink" : "TEST2", + "physicalPlan" : { + "@type" : "streamSinkV1", + "properties" : { + "queryContext" : "TEST2" + }, + "source" : { + "@type" : "streamSelectV1", + "properties" : { + "queryContext" : "Project" + }, + "source" : { + "@type" : "streamSourceV1", + "properties" : { + "queryContext" : "KsqlTopic/Source" + }, + "topicName" : "test", + "formats" : { + "keyFormat" : { + "format" : "KAFKA", + "properties" : { } + }, + "valueFormat" : { + "format" : "PROTOBUF_NOSR", + "properties" : { } + } + }, + "timestampColumn" : null, + "sourceSchema" : "`ID` STRING KEY, `B` BYTES", + "pseudoColumnVersion" : 1 + }, + "keyColumnNames" : [ "ID" ], + "selectedKeys" : null, + "selectExpressions" : [ "B AS B" ] + }, + "formats" : { + "keyFormat" : { + "format" : "KAFKA", + "properties" : { } + }, + "valueFormat" : { + "format" : "PROTOBUF_NOSR", + "properties" : { } + } + }, + "topicName" : "TEST2", + "timestampColumn" : null + }, + "queryId" : "CSAS_TEST2_0", + "runtimeId" : null + } + } ], + "configs" : { + "ksql.extension.dir" : "ext", + "ksql.streams.cache.max.bytes.buffering" : "0", + "metric.reporters" : "", + "ksql.query.status.running.threshold.seconds" : "300", + "ksql.connect.basic.auth.credentials.reload" : "false", + "ksql.output.topic.name.prefix" : "", + "ksql.query.pull.stream.enabled" : "true", + "ksql.query.push.v2.interpreter.enabled" : "true", + "ksql.queryanonymizer.logs_enabled" : "true", + "ksql.variable.substitution.enable" : "true", + "ksql.streams.shutdown.timeout.ms" : "300000", + "ksql.query.pull.max.allowed.offset.lag" : "9223372036854775807", + "ksql.query.pull.max.qps" : "2147483647", + "ksql.access.validator.enable" : "auto", + "ksql.streams.bootstrap.servers" : "localhost:0", + "ksql.query.pull.metrics.enabled" : "true", + "ksql.metrics.extension" : null, + "ksql.query.push.v2.alos.enabled" : "true", + "ksql.query.push.v2.max.hourly.bandwidth.megabytes" : "2147483647", + "ksql.query.pull.range.scan.enabled" : "true", + "ksql.transient.query.cleanup.service.initial.delay.seconds" : "600", + "ksql.hidden.topics" : "_confluent.*,__confluent.*,_schemas,__consumer_offsets,__transaction_state,connect-configs,connect-offsets,connect-status,connect-statuses", + "ksql.lambdas.enabled" : "true", + "ksql.source.table.materialization.enabled" : "true", + "ksql.query.pull.max.hourly.bandwidth.megabytes" : "2147483647", + "ksql.sink.window.change.log.additional.retention" : "1000000", + "ksql.query.persistent.active.limit" : "2147483647", + "ksql.persistence.wrap.single.values" : null, + "ksql.query.transient.max.bytes.buffering.total" : "-1", + "ksql.connect.basic.auth.credentials.source" : "NONE", + "ksql.schema.registry.url" : "schema_registry.url:0", + "ksql.properties.overrides.denylist" : "", + "ksql.service.id" : "some.ksql.service.id", + "ksql.query.push.v2.max.catchup.consumers" : "5", + "ksql.query.push.v2.enabled" : "false", + "ksql.transient.query.cleanup.service.enable" : "true", + "ksql.query.push.v2.metrics.enabled" : "true", + "ksql.rowpartition.rowoffset.enabled" : "true", + "ksql.streams.commit.interval.ms" : "2000", + "ksql.query.pull.table.scan.enabled" : "true", + "ksql.streams.auto.commit.interval.ms" : "0", + "ksql.streams.topology.optimization" : "all", + "ksql.endpoint.migrate.query" : "true", + "ksql.query.push.v2.registry.installed" : "false", + "ksql.streams.num.stream.threads" : "4", + "ksql.metrics.tags.custom" : "", + "ksql.query.push.v2.catchup.consumer.msg.window" : "50", + "ksql.runtime.feature.shared.enabled" : "false", + "ksql.udf.collect.metrics" : "false", + "ksql.new.query.planner.enabled" : "false", + "ksql.connect.request.headers.plugin" : null, + "ksql.security.extension.class" : null, + "ksql.transient.prefix" : "transient_", + "ksql.headers.columns.enabled" : "true", + "ksql.streams.default.deserialization.exception.handler" : "io.confluent.ksql.errors.LogMetricAndContinueExceptionHandler", + "ksql.connect.request.timeout.ms" : "5000", + "ksql.query.pull.enable.standby.reads" : "false", + "ksql.persistence.default.format.key" : "KAFKA", + "ksql.query.persistent.max.bytes.buffering.total" : "-1", + "ksql.query.error.max.queue.size" : "10", + "ksql.query.cleanup.shutdown.timeout.ms" : "30000", + "ksql.internal.topic.min.insync.replicas" : "1", + "ksql.internal.topic.replicas" : "1", + "ksql.insert.into.values.enabled" : "true", + "ksql.queryanonymizer.cluster_namespace" : null, + "ksql.create.or.replace.enabled" : "true", + "ksql.shared.runtimes.count" : "2", + "ksql.cast.strings.preserve.nulls" : "true", + "ksql.authorization.cache.max.entries" : "10000", + "ksql.pull.queries.enable" : "true", + "ksql.transient.query.cleanup.service.period.seconds" : "600", + "ksql.suppress.enabled" : "false", + "ksql.readonly.topics" : "_confluent.*,__confluent.*,_schemas,__consumer_offsets,__transaction_state,connect-configs,connect-offsets,connect-status,connect-statuses", + "ksql.connect.basic.auth.credentials.file" : "", + "ksql.authorization.cache.expiry.time.secs" : "30", + "ksql.query.retry.backoff.initial.ms" : "15000", + "ksql.query.pull.max.concurrent.requests" : "2147483647", + "ksql.streams.auto.offset.reset" : "earliest", + "ksql.connect.url" : "http://localhost:8083", + "ksql.query.push.v2.new.latest.delay.ms" : "5000", + "ksql.query.push.v2.latest.reset.age.ms" : "30000", + "ksql.streams.default.production.exception.handler" : "io.confluent.ksql.errors.ProductionExceptionHandlerUtil$LogAndFailProductionExceptionHandler", + "ksql.query.pull.interpreter.enabled" : "true", + "ksql.query.pull.limit.clause.enabled" : "true", + "ksql.query.pull.router.thread.pool.size" : "50", + "ksql.query.push.v2.continuation.tokens.enabled" : "false", + "ksql.query.retry.backoff.max.ms" : "900000", + "ksql.timestamp.throw.on.invalid" : "false", + "ksql.persistence.default.format.value" : null, + "ksql.udfs.enabled" : "true", + "ksql.udf.enable.security.manager" : "true", + "ksql.connect.worker.config" : "", + "ksql.nested.error.set.null" : "true", + "ksql.query.pull.thread.pool.size" : "50", + "ksql.persistent.prefix" : "query_", + "ksql.metastore.backup.location" : "", + "ksql.error.classifier.regex" : "", + "ksql.suppress.buffer.size.bytes" : "-1" + } +} \ No newline at end of file diff --git a/ksqldb-functional-tests/src/test/resources/historical_plans/bytes_-_PROTOBUF_NOSR_in_out/7.3.0_1651069522837/spec.json b/ksqldb-functional-tests/src/test/resources/historical_plans/bytes_-_PROTOBUF_NOSR_in_out/7.3.0_1651069522837/spec.json new file mode 100644 index 000000000000..f4d7082f5562 --- /dev/null +++ b/ksqldb-functional-tests/src/test/resources/historical_plans/bytes_-_PROTOBUF_NOSR_in_out/7.3.0_1651069522837/spec.json @@ -0,0 +1,98 @@ +{ + "version" : "7.3.0", + "timestamp" : 1651069522837, + "path" : "query-validation-tests/bytes.json", + "schemas" : { + "CSAS_TEST2_0.TEST2" : { + "schema" : "`ID` STRING KEY, `B` BYTES", + "keyFormat" : { + "format" : "KAFKA" + }, + "valueFormat" : { + "format" : "PROTOBUF_NOSR" + } + }, + "CSAS_TEST2_0.KsqlTopic.Source" : { + "schema" : "`ID` STRING KEY, `B` BYTES", + "keyFormat" : { + "format" : "KAFKA" + }, + "valueFormat" : { + "format" : "PROTOBUF_NOSR" + } + } + }, + "testCase" : { + "name" : "PROTOBUF_NOSR in/out", + "inputs" : [ { + "topic" : "test", + "key" : null, + "value" : { + "b" : "dmFyaWF0aW9ucw==" + } + } ], + "outputs" : [ { + "topic" : "TEST2", + "key" : null, + "value" : { + "B" : "dmFyaWF0aW9ucw==" + } + } ], + "topics" : [ { + "name" : "TEST2", + "replicas" : 1, + "numPartitions" : 4 + }, { + "name" : "test", + "replicas" : 1, + "numPartitions" : 4 + } ], + "statements" : [ "CREATE STREAM TEST (ID STRING KEY, b BYTES) WITH (kafka_topic='test', value_format='PROTOBUF_NOSR');", "CREATE STREAM TEST2 AS SELECT * FROM TEST;" ], + "post" : { + "sources" : [ { + "name" : "TEST", + "type" : "STREAM", + "schema" : "`ID` STRING KEY, `B` BYTES", + "keyFormat" : { + "format" : "KAFKA" + }, + "valueFormat" : "PROTOBUF_NOSR", + "keyFeatures" : [ ], + "valueFeatures" : [ ], + "isSource" : false + }, { + "name" : "TEST2", + "type" : "STREAM", + "schema" : "`ID` STRING KEY, `B` BYTES", + "keyFormat" : { + "format" : "KAFKA" + }, + "valueFormat" : "PROTOBUF_NOSR", + "keyFeatures" : [ ], + "valueFeatures" : [ ], + "isSource" : false + } ], + "topics" : { + "topics" : [ { + "name" : "test", + "keyFormat" : { + "format" : "KAFKA" + }, + "valueFormat" : { + "format" : "PROTOBUF_NOSR" + }, + "partitions" : 4 + }, { + "name" : "TEST2", + "keyFormat" : { + "format" : "KAFKA" + }, + "valueFormat" : { + "format" : "PROTOBUF_NOSR" + }, + "partitions" : 4 + } ] + } + } + } +} \ No newline at end of file diff --git a/ksqldb-functional-tests/src/test/resources/historical_plans/bytes_-_PROTOBUF_NOSR_in_out/7.3.0_1651069522837/topology b/ksqldb-functional-tests/src/test/resources/historical_plans/bytes_-_PROTOBUF_NOSR_in_out/7.3.0_1651069522837/topology new file mode 100644 index 000000000000..e080cd675668 --- /dev/null +++ b/ksqldb-functional-tests/src/test/resources/historical_plans/bytes_-_PROTOBUF_NOSR_in_out/7.3.0_1651069522837/topology @@ -0,0 +1,13 @@ +Topologies: + Sub-topology: 0 + Source: KSTREAM-SOURCE-0000000000 (topics: [test]) + --> KSTREAM-TRANSFORMVALUES-0000000001 + Processor: KSTREAM-TRANSFORMVALUES-0000000001 (stores: []) + --> Project + <-- KSTREAM-SOURCE-0000000000 + Processor: Project (stores: []) + --> KSTREAM-SINK-0000000003 + <-- KSTREAM-TRANSFORMVALUES-0000000001 + Sink: KSTREAM-SINK-0000000003 (topic: TEST2) + <-- Project + diff --git a/ksqldb-functional-tests/src/test/resources/historical_plans/collect-list_-_collect_list_bool_map_-_PROTOBUF_NOSR/7.3.0_1651069545402/plan.json b/ksqldb-functional-tests/src/test/resources/historical_plans/collect-list_-_collect_list_bool_map_-_PROTOBUF_NOSR/7.3.0_1651069545402/plan.json new file mode 100644 index 000000000000..ba6991ee5ab0 --- /dev/null +++ b/ksqldb-functional-tests/src/test/resources/historical_plans/collect-list_-_collect_list_bool_map_-_PROTOBUF_NOSR/7.3.0_1651069545402/plan.json @@ -0,0 +1,255 @@ +{ + "plan" : [ { + "@type" : "ksqlPlanV1", + "statementText" : "CREATE STREAM TEST (ID BIGINT KEY, NAME STRING, VALUE MAP) WITH (KAFKA_TOPIC='test_topic', KEY_FORMAT='KAFKA', VALUE_FORMAT='PROTOBUF_NOSR');", + "ddlCommand" : { + "@type" : "createStreamV1", + "sourceName" : "TEST", + "schema" : "`ID` BIGINT KEY, `NAME` STRING, `VALUE` MAP", + "timestampColumn" : null, + "topicName" : "test_topic", + "formats" : { + "keyFormat" : { + "format" : "KAFKA", + "properties" : { } + }, + "valueFormat" : { + "format" : "PROTOBUF_NOSR", + "properties" : { } + } + }, + "windowInfo" : null, + "orReplace" : false, + "isSource" : false + }, + "queryPlan" : null + }, { + "@type" : "ksqlPlanV1", + "statementText" : "CREATE TABLE S2 AS SELECT\n TEST.ID ID,\n COLLECT_LIST(TEST.VALUE['key1']) COLLECTED\nFROM TEST TEST\nGROUP BY TEST.ID\nEMIT CHANGES", + "ddlCommand" : { + "@type" : "createTableV1", + "sourceName" : "S2", + "schema" : "`ID` BIGINT KEY, `COLLECTED` ARRAY", + "timestampColumn" : null, + "topicName" : "S2", + "formats" : { + "keyFormat" : { + "format" : "KAFKA", + "properties" : { } + }, + "valueFormat" : { + "format" : "PROTOBUF_NOSR", + "properties" : { } + } + }, + "windowInfo" : null, + "orReplace" : false, + "isSource" : false + }, + "queryPlan" : { + "sources" : [ "TEST" ], + "sink" : "S2", + "physicalPlan" : { + "@type" : "tableSinkV1", + "properties" : { + "queryContext" : "S2" + }, + "source" : { + "@type" : "tableSelectV1", + "properties" : { + "queryContext" : "Aggregate/Project" + }, + "source" : { + "@type" : "streamAggregateV1", + "properties" : { + "queryContext" : "Aggregate/Aggregate" + }, + "source" : { + "@type" : "streamGroupByKeyV1", + "properties" : { + "queryContext" : "Aggregate/GroupBy" + }, + "source" : { + "@type" : "streamSelectV1", + "properties" : { + "queryContext" : "Aggregate/Prepare" + }, + "source" : { + "@type" : "streamSourceV1", + "properties" : { + "queryContext" : "KsqlTopic/Source" + }, + "topicName" : "test_topic", + "formats" : { + "keyFormat" : { + "format" : "KAFKA", + "properties" : { } + }, + "valueFormat" : { + "format" : "PROTOBUF_NOSR", + "properties" : { } + } + }, + "timestampColumn" : null, + "sourceSchema" : "`ID` BIGINT KEY, `NAME` STRING, `VALUE` MAP", + "pseudoColumnVersion" : 1 + }, + "keyColumnNames" : [ "ID" ], + "selectedKeys" : null, + "selectExpressions" : [ "ID AS ID", "VALUE AS VALUE", "VALUE['key1'] AS KSQL_INTERNAL_COL_2" ] + }, + "internalFormats" : { + "keyFormat" : { + "format" : "KAFKA", + "properties" : { } + }, + "valueFormat" : { + "format" : "PROTOBUF_NOSR", + "properties" : { } + } + } + }, + "internalFormats" : { + "keyFormat" : { + "format" : "KAFKA", + "properties" : { } + }, + "valueFormat" : { + "format" : "PROTOBUF_NOSR", + "properties" : { } + } + }, + "nonAggregateColumns" : [ "ID", "VALUE" ], + "aggregationFunctions" : [ "COLLECT_LIST(KSQL_INTERNAL_COL_2)" ] + }, + "keyColumnNames" : [ "ID" ], + "selectExpressions" : [ "KSQL_AGG_VARIABLE_0 AS COLLECTED" ], + "internalFormats" : { + "keyFormat" : { + "format" : "KAFKA", + "properties" : { } + }, + "valueFormat" : { + "format" : "PROTOBUF_NOSR", + "properties" : { } + } + } + }, + "formats" : { + "keyFormat" : { + "format" : "KAFKA", + "properties" : { } + }, + "valueFormat" : { + "format" : "PROTOBUF_NOSR", + "properties" : { } + } + }, + "topicName" : "S2", + "timestampColumn" : null + }, + "queryId" : "CTAS_S2_0", + "runtimeId" : null + } + } ], + "configs" : { + "ksql.extension.dir" : "ext", + "ksql.streams.cache.max.bytes.buffering" : "0", + "metric.reporters" : "", + "ksql.query.status.running.threshold.seconds" : "300", + "ksql.connect.basic.auth.credentials.reload" : "false", + "ksql.output.topic.name.prefix" : "", + "ksql.query.pull.stream.enabled" : "true", + "ksql.query.push.v2.interpreter.enabled" : "true", + "ksql.queryanonymizer.logs_enabled" : "true", + "ksql.variable.substitution.enable" : "true", + "ksql.streams.shutdown.timeout.ms" : "300000", + "ksql.query.pull.max.allowed.offset.lag" : "9223372036854775807", + "ksql.query.pull.max.qps" : "2147483647", + "ksql.access.validator.enable" : "auto", + "ksql.streams.bootstrap.servers" : "localhost:0", + "ksql.query.pull.metrics.enabled" : "true", + "ksql.metrics.extension" : null, + "ksql.query.push.v2.alos.enabled" : "true", + "ksql.query.push.v2.max.hourly.bandwidth.megabytes" : "2147483647", + "ksql.query.pull.range.scan.enabled" : "true", + "ksql.transient.query.cleanup.service.initial.delay.seconds" : "600", + "ksql.hidden.topics" : "_confluent.*,__confluent.*,_schemas,__consumer_offsets,__transaction_state,connect-configs,connect-offsets,connect-status,connect-statuses", + "ksql.lambdas.enabled" : "true", + "ksql.source.table.materialization.enabled" : "true", + "ksql.query.pull.max.hourly.bandwidth.megabytes" : "2147483647", + "ksql.sink.window.change.log.additional.retention" : "1000000", + "ksql.query.persistent.active.limit" : "2147483647", + "ksql.persistence.wrap.single.values" : null, + "ksql.query.transient.max.bytes.buffering.total" : "-1", + "ksql.connect.basic.auth.credentials.source" : "NONE", + "ksql.schema.registry.url" : "schema_registry.url:0", + "ksql.properties.overrides.denylist" : "", + "ksql.service.id" : "some.ksql.service.id", + "ksql.query.push.v2.max.catchup.consumers" : "5", + "ksql.query.push.v2.enabled" : "false", + "ksql.transient.query.cleanup.service.enable" : "true", + "ksql.query.push.v2.metrics.enabled" : "true", + "ksql.rowpartition.rowoffset.enabled" : "true", + "ksql.streams.commit.interval.ms" : "2000", + "ksql.query.pull.table.scan.enabled" : "true", + "ksql.streams.auto.commit.interval.ms" : "0", + "ksql.streams.topology.optimization" : "all", + "ksql.endpoint.migrate.query" : "true", + "ksql.query.push.v2.registry.installed" : "false", + "ksql.streams.num.stream.threads" : "4", + "ksql.metrics.tags.custom" : "", + "ksql.query.push.v2.catchup.consumer.msg.window" : "50", + "ksql.runtime.feature.shared.enabled" : "false", + "ksql.udf.collect.metrics" : "false", + "ksql.new.query.planner.enabled" : "false", + "ksql.connect.request.headers.plugin" : null, + "ksql.security.extension.class" : null, + "ksql.transient.prefix" : "transient_", + "ksql.headers.columns.enabled" : "true", + "ksql.streams.default.deserialization.exception.handler" : "io.confluent.ksql.errors.LogMetricAndContinueExceptionHandler", + "ksql.connect.request.timeout.ms" : "5000", + "ksql.query.pull.enable.standby.reads" : "false", + "ksql.persistence.default.format.key" : "KAFKA", + "ksql.query.persistent.max.bytes.buffering.total" : "-1", + "ksql.query.error.max.queue.size" : "10", + "ksql.query.cleanup.shutdown.timeout.ms" : "30000", + "ksql.internal.topic.min.insync.replicas" : "1", + "ksql.internal.topic.replicas" : "1", + "ksql.insert.into.values.enabled" : "true", + "ksql.queryanonymizer.cluster_namespace" : null, + "ksql.create.or.replace.enabled" : "true", + "ksql.shared.runtimes.count" : "2", + "ksql.cast.strings.preserve.nulls" : "true", + "ksql.authorization.cache.max.entries" : "10000", + "ksql.pull.queries.enable" : "true", + "ksql.transient.query.cleanup.service.period.seconds" : "600", + "ksql.suppress.enabled" : "false", + "ksql.readonly.topics" : "_confluent.*,__confluent.*,_schemas,__consumer_offsets,__transaction_state,connect-configs,connect-offsets,connect-status,connect-statuses", + "ksql.connect.basic.auth.credentials.file" : "", + "ksql.authorization.cache.expiry.time.secs" : "30", + "ksql.query.retry.backoff.initial.ms" : "15000", + "ksql.query.pull.max.concurrent.requests" : "2147483647", + "ksql.streams.auto.offset.reset" : "earliest", + "ksql.connect.url" : "http://localhost:8083", + "ksql.query.push.v2.new.latest.delay.ms" : "5000", + "ksql.query.push.v2.latest.reset.age.ms" : "30000", + "ksql.streams.default.production.exception.handler" : "io.confluent.ksql.errors.ProductionExceptionHandlerUtil$LogAndFailProductionExceptionHandler", + "ksql.query.pull.interpreter.enabled" : "true", + "ksql.query.pull.limit.clause.enabled" : "true", + "ksql.query.pull.router.thread.pool.size" : "50", + "ksql.query.push.v2.continuation.tokens.enabled" : "false", + "ksql.query.retry.backoff.max.ms" : "900000", + "ksql.timestamp.throw.on.invalid" : "false", + "ksql.persistence.default.format.value" : null, + "ksql.udfs.enabled" : "true", + "ksql.udf.enable.security.manager" : "true", + "ksql.connect.worker.config" : "", + "ksql.nested.error.set.null" : "true", + "ksql.query.pull.thread.pool.size" : "50", + "ksql.persistent.prefix" : "query_", + "ksql.metastore.backup.location" : "", + "ksql.error.classifier.regex" : "", + "ksql.suppress.buffer.size.bytes" : "-1" + } +} \ No newline at end of file diff --git a/ksqldb-functional-tests/src/test/resources/historical_plans/collect-list_-_collect_list_bool_map_-_PROTOBUF_NOSR/7.3.0_1651069545402/spec.json b/ksqldb-functional-tests/src/test/resources/historical_plans/collect-list_-_collect_list_bool_map_-_PROTOBUF_NOSR/7.3.0_1651069545402/spec.json new file mode 100644 index 000000000000..c1f553ac5d6d --- /dev/null +++ b/ksqldb-functional-tests/src/test/resources/historical_plans/collect-list_-_collect_list_bool_map_-_PROTOBUF_NOSR/7.3.0_1651069545402/spec.json @@ -0,0 +1,169 @@ +{ + "version" : "7.3.0", + "timestamp" : 1651069545402, + "path" : "query-validation-tests/collect-list.json", + "schemas" : { + "CTAS_S2_0.Aggregate.Aggregate.Materialize" : { + "schema" : "`ID` BIGINT KEY, `ID` BIGINT, `VALUE` MAP, `KSQL_AGG_VARIABLE_0` ARRAY", + "keyFormat" : { + "format" : "KAFKA" + }, + "valueFormat" : { + "format" : "PROTOBUF_NOSR" + } + }, + "CTAS_S2_0.Aggregate.Project" : { + "schema" : "`ID` BIGINT KEY, `COLLECTED` ARRAY", + "keyFormat" : { + "format" : "KAFKA" + }, + "valueFormat" : { + "format" : "PROTOBUF_NOSR" + } + }, + "CTAS_S2_0.KsqlTopic.Source" : { + "schema" : "`ID` BIGINT KEY, `NAME` STRING, `VALUE` MAP", + "keyFormat" : { + "format" : "KAFKA" + }, + "valueFormat" : { + "format" : "PROTOBUF_NOSR" + } + }, + "CTAS_S2_0.S2" : { + "schema" : "`ID` BIGINT KEY, `COLLECTED` ARRAY", + "keyFormat" : { + "format" : "KAFKA" + }, + "valueFormat" : { + "format" : "PROTOBUF_NOSR" + } + }, + "CTAS_S2_0.Aggregate.GroupBy" : { + "schema" : "`ID` BIGINT KEY, `ID` BIGINT, `VALUE` MAP, `KSQL_INTERNAL_COL_2` BOOLEAN", + "keyFormat" : { + "format" : "KAFKA" + }, + "valueFormat" : { + "format" : "PROTOBUF_NOSR" + } + } + }, + "testCase" : { + "name" : "collect_list bool map - PROTOBUF_NOSR", + "inputs" : [ { + "topic" : "test_topic", + "key" : 0, + "value" : { + "name" : "zero", + "value" : { + "key1" : true, + "key2" : false + } + } + }, { + "topic" : "test_topic", + "key" : 0, + "value" : { + "name" : "zero", + "value" : { + "key1" : false, + "key2" : true + } + } + }, { + "topic" : "test_topic", + "key" : 0, + "value" : { + "name" : "zero", + "value" : { + "key1" : true, + "key2" : true + } + } + } ], + "outputs" : [ { + "topic" : "S2", + "key" : 0, + "value" : { + "COLLECTED" : [ true ] + } + }, { + "topic" : "S2", + "key" : 0, + "value" : { + "COLLECTED" : [ true, false ] + } + }, { + "topic" : "S2", + "key" : 0, + "value" : { + "COLLECTED" : [ true, false, true ] + } + } ], + "topics" : [ { + "name" : "test_topic", + "replicas" : 1, + "numPartitions" : 4 + }, { + "name" : "S2", + "replicas" : 1, + "numPartitions" : 4 + } ], + "statements" : [ "CREATE STREAM TEST (ID BIGINT KEY, NAME varchar, VALUE map) WITH (kafka_topic='test_topic', value_format='PROTOBUF_NOSR');", "CREATE TABLE S2 as SELECT ID, collect_list(value['key1']) AS collected FROM test group by id;" ], + "post" : { + "sources" : [ { + "name" : "S2", + "type" : "TABLE", + "schema" : "`ID` BIGINT KEY, `COLLECTED` ARRAY", + "keyFormat" : { + "format" : "KAFKA" + }, + "valueFormat" : "PROTOBUF_NOSR", + "keyFeatures" : [ ], + "valueFeatures" : [ ], + "isSource" : false + }, { + "name" : "TEST", + "type" : "STREAM", + "schema" : "`ID` BIGINT KEY, `NAME` STRING, `VALUE` MAP", + "keyFormat" : { + "format" : "KAFKA" + }, + "valueFormat" : "PROTOBUF_NOSR", + "keyFeatures" : [ ], + "valueFeatures" : [ ], + "isSource" : false + } ], + "topics" : { + "topics" : [ { + "name" : "_confluent-ksql-some.ksql.service.idquery_CTAS_S2_0-Aggregate-Aggregate-Materialize-changelog", + "keyFormat" : { + "format" : "KAFKA" + }, + "valueFormat" : { + "format" : "PROTOBUF_NOSR" + } + }, { + "name" : "test_topic", + "keyFormat" : { + "format" : "KAFKA" + }, + "valueFormat" : { + "format" : "PROTOBUF_NOSR" + }, + "partitions" : 4 + }, { + "name" : "S2", + "keyFormat" : { + "format" : "KAFKA" + }, + "valueFormat" : { + "format" : "PROTOBUF_NOSR" + }, + "partitions" : 4 + } ] + } + } + } +} \ No newline at end of file diff --git a/ksqldb-functional-tests/src/test/resources/historical_plans/collect-list_-_collect_list_bool_map_-_PROTOBUF_NOSR/7.3.0_1651069545402/topology b/ksqldb-functional-tests/src/test/resources/historical_plans/collect-list_-_collect_list_bool_map_-_PROTOBUF_NOSR/7.3.0_1651069545402/topology new file mode 100644 index 000000000000..fcfc8fb28665 --- /dev/null +++ b/ksqldb-functional-tests/src/test/resources/historical_plans/collect-list_-_collect_list_bool_map_-_PROTOBUF_NOSR/7.3.0_1651069545402/topology @@ -0,0 +1,25 @@ +Topologies: + Sub-topology: 0 + Source: KSTREAM-SOURCE-0000000000 (topics: [test_topic]) + --> KSTREAM-TRANSFORMVALUES-0000000001 + Processor: KSTREAM-TRANSFORMVALUES-0000000001 (stores: []) + --> Aggregate-Prepare + <-- KSTREAM-SOURCE-0000000000 + Processor: Aggregate-Prepare (stores: []) + --> KSTREAM-AGGREGATE-0000000003 + <-- KSTREAM-TRANSFORMVALUES-0000000001 + Processor: KSTREAM-AGGREGATE-0000000003 (stores: [Aggregate-Aggregate-Materialize]) + --> Aggregate-Aggregate-ToOutputSchema + <-- Aggregate-Prepare + Processor: Aggregate-Aggregate-ToOutputSchema (stores: []) + --> Aggregate-Project + <-- KSTREAM-AGGREGATE-0000000003 + Processor: Aggregate-Project (stores: []) + --> KTABLE-TOSTREAM-0000000006 + <-- Aggregate-Aggregate-ToOutputSchema + Processor: KTABLE-TOSTREAM-0000000006 (stores: []) + --> KSTREAM-SINK-0000000007 + <-- Aggregate-Project + Sink: KSTREAM-SINK-0000000007 (topic: S2) + <-- KTABLE-TOSTREAM-0000000006 + diff --git a/ksqldb-functional-tests/src/test/resources/historical_plans/collect-list_-_collect_list_bool_map_table_-_PROTOBUF_NOSR/7.3.0_1651069580920/plan.json b/ksqldb-functional-tests/src/test/resources/historical_plans/collect-list_-_collect_list_bool_map_table_-_PROTOBUF_NOSR/7.3.0_1651069580920/plan.json new file mode 100644 index 000000000000..35509d09c983 --- /dev/null +++ b/ksqldb-functional-tests/src/test/resources/historical_plans/collect-list_-_collect_list_bool_map_table_-_PROTOBUF_NOSR/7.3.0_1651069580920/plan.json @@ -0,0 +1,275 @@ +{ + "plan" : [ { + "@type" : "ksqlPlanV1", + "statementText" : "CREATE TABLE TEST (ID BIGINT PRIMARY KEY, NAME STRING, VALUE MAP) WITH (KAFKA_TOPIC='test_topic', KEY_FORMAT='KAFKA', VALUE_FORMAT='PROTOBUF_NOSR');", + "ddlCommand" : { + "@type" : "createTableV1", + "sourceName" : "TEST", + "schema" : "`ID` BIGINT KEY, `NAME` STRING, `VALUE` MAP", + "timestampColumn" : null, + "topicName" : "test_topic", + "formats" : { + "keyFormat" : { + "format" : "KAFKA", + "properties" : { } + }, + "valueFormat" : { + "format" : "PROTOBUF_NOSR", + "properties" : { } + } + }, + "windowInfo" : null, + "orReplace" : false, + "isSource" : false + }, + "queryPlan" : null + }, { + "@type" : "ksqlPlanV1", + "statementText" : "CREATE TABLE S2 AS SELECT\n TEST.ID ID,\n COLLECT_LIST(TEST.VALUE['key1']) COLLECTED\nFROM TEST TEST\nGROUP BY TEST.ID\nEMIT CHANGES", + "ddlCommand" : { + "@type" : "createTableV1", + "sourceName" : "S2", + "schema" : "`ID` BIGINT KEY, `COLLECTED` ARRAY", + "timestampColumn" : null, + "topicName" : "S2", + "formats" : { + "keyFormat" : { + "format" : "KAFKA", + "properties" : { } + }, + "valueFormat" : { + "format" : "PROTOBUF_NOSR", + "properties" : { } + } + }, + "windowInfo" : null, + "orReplace" : false, + "isSource" : false + }, + "queryPlan" : { + "sources" : [ "TEST" ], + "sink" : "S2", + "physicalPlan" : { + "@type" : "tableSinkV1", + "properties" : { + "queryContext" : "S2" + }, + "source" : { + "@type" : "tableSelectV1", + "properties" : { + "queryContext" : "Aggregate/Project" + }, + "source" : { + "@type" : "tableAggregateV1", + "properties" : { + "queryContext" : "Aggregate/Aggregate" + }, + "source" : { + "@type" : "tableGroupByV2", + "properties" : { + "queryContext" : "Aggregate/GroupBy" + }, + "source" : { + "@type" : "tableSelectV1", + "properties" : { + "queryContext" : "Aggregate/Prepare" + }, + "source" : { + "@type" : "tableSourceV2", + "properties" : { + "queryContext" : "KsqlTopic/Source" + }, + "topicName" : "test_topic", + "formats" : { + "keyFormat" : { + "format" : "KAFKA", + "properties" : { } + }, + "valueFormat" : { + "format" : "PROTOBUF_NOSR", + "properties" : { } + } + }, + "timestampColumn" : null, + "sourceSchema" : "`ID` BIGINT KEY, `NAME` STRING, `VALUE` MAP", + "pseudoColumnVersion" : 1, + "stateStoreFormats" : { + "keyFormat" : { + "format" : "KAFKA", + "properties" : { } + }, + "valueFormat" : { + "format" : "PROTOBUF_NOSR", + "properties" : { } + } + } + }, + "keyColumnNames" : [ "ID" ], + "selectExpressions" : [ "ID AS ID", "VALUE AS VALUE", "VALUE['key1'] AS KSQL_INTERNAL_COL_2" ], + "internalFormats" : { + "keyFormat" : { + "format" : "KAFKA", + "properties" : { } + }, + "valueFormat" : { + "format" : "PROTOBUF_NOSR", + "properties" : { } + } + } + }, + "internalFormats" : { + "keyFormat" : { + "format" : "KAFKA", + "properties" : { } + }, + "valueFormat" : { + "format" : "PROTOBUF_NOSR", + "properties" : { } + } + }, + "groupByExpressions" : [ "ID" ] + }, + "internalFormats" : { + "keyFormat" : { + "format" : "KAFKA", + "properties" : { } + }, + "valueFormat" : { + "format" : "PROTOBUF_NOSR", + "properties" : { } + } + }, + "nonAggregateColumns" : [ "ID", "VALUE" ], + "aggregationFunctions" : [ "COLLECT_LIST(KSQL_INTERNAL_COL_2)" ] + }, + "keyColumnNames" : [ "ID" ], + "selectExpressions" : [ "KSQL_AGG_VARIABLE_0 AS COLLECTED" ], + "internalFormats" : { + "keyFormat" : { + "format" : "KAFKA", + "properties" : { } + }, + "valueFormat" : { + "format" : "PROTOBUF_NOSR", + "properties" : { } + } + } + }, + "formats" : { + "keyFormat" : { + "format" : "KAFKA", + "properties" : { } + }, + "valueFormat" : { + "format" : "PROTOBUF_NOSR", + "properties" : { } + } + }, + "topicName" : "S2", + "timestampColumn" : null + }, + "queryId" : "CTAS_S2_0", + "runtimeId" : null + } + } ], + "configs" : { + "ksql.extension.dir" : "ext", + "ksql.streams.cache.max.bytes.buffering" : "0", + "metric.reporters" : "", + "ksql.query.status.running.threshold.seconds" : "300", + "ksql.connect.basic.auth.credentials.reload" : "false", + "ksql.output.topic.name.prefix" : "", + "ksql.query.pull.stream.enabled" : "true", + "ksql.query.push.v2.interpreter.enabled" : "true", + "ksql.queryanonymizer.logs_enabled" : "true", + "ksql.variable.substitution.enable" : "true", + "ksql.streams.shutdown.timeout.ms" : "300000", + "ksql.query.pull.max.allowed.offset.lag" : "9223372036854775807", + "ksql.query.pull.max.qps" : "2147483647", + "ksql.access.validator.enable" : "auto", + "ksql.streams.bootstrap.servers" : "localhost:0", + "ksql.query.pull.metrics.enabled" : "true", + "ksql.metrics.extension" : null, + "ksql.query.push.v2.alos.enabled" : "true", + "ksql.query.push.v2.max.hourly.bandwidth.megabytes" : "2147483647", + "ksql.query.pull.range.scan.enabled" : "true", + "ksql.transient.query.cleanup.service.initial.delay.seconds" : "600", + "ksql.hidden.topics" : "_confluent.*,__confluent.*,_schemas,__consumer_offsets,__transaction_state,connect-configs,connect-offsets,connect-status,connect-statuses", + "ksql.lambdas.enabled" : "true", + "ksql.source.table.materialization.enabled" : "true", + "ksql.query.pull.max.hourly.bandwidth.megabytes" : "2147483647", + "ksql.sink.window.change.log.additional.retention" : "1000000", + "ksql.query.persistent.active.limit" : "2147483647", + "ksql.persistence.wrap.single.values" : null, + "ksql.query.transient.max.bytes.buffering.total" : "-1", + "ksql.connect.basic.auth.credentials.source" : "NONE", + "ksql.schema.registry.url" : "schema_registry.url:0", + "ksql.properties.overrides.denylist" : "", + "ksql.service.id" : "some.ksql.service.id", + "ksql.query.push.v2.max.catchup.consumers" : "5", + "ksql.query.push.v2.enabled" : "false", + "ksql.transient.query.cleanup.service.enable" : "true", + "ksql.query.push.v2.metrics.enabled" : "true", + "ksql.rowpartition.rowoffset.enabled" : "true", + "ksql.streams.commit.interval.ms" : "2000", + "ksql.query.pull.table.scan.enabled" : "true", + "ksql.streams.auto.commit.interval.ms" : "0", + "ksql.streams.topology.optimization" : "all", + "ksql.endpoint.migrate.query" : "true", + "ksql.query.push.v2.registry.installed" : "false", + "ksql.streams.num.stream.threads" : "4", + "ksql.metrics.tags.custom" : "", + "ksql.query.push.v2.catchup.consumer.msg.window" : "50", + "ksql.runtime.feature.shared.enabled" : "false", + "ksql.udf.collect.metrics" : "false", + "ksql.new.query.planner.enabled" : "false", + "ksql.connect.request.headers.plugin" : null, + "ksql.security.extension.class" : null, + "ksql.transient.prefix" : "transient_", + "ksql.headers.columns.enabled" : "true", + "ksql.streams.default.deserialization.exception.handler" : "io.confluent.ksql.errors.LogMetricAndContinueExceptionHandler", + "ksql.connect.request.timeout.ms" : "5000", + "ksql.query.pull.enable.standby.reads" : "false", + "ksql.persistence.default.format.key" : "KAFKA", + "ksql.query.persistent.max.bytes.buffering.total" : "-1", + "ksql.query.error.max.queue.size" : "10", + "ksql.query.cleanup.shutdown.timeout.ms" : "30000", + "ksql.internal.topic.min.insync.replicas" : "1", + "ksql.internal.topic.replicas" : "1", + "ksql.insert.into.values.enabled" : "true", + "ksql.queryanonymizer.cluster_namespace" : null, + "ksql.create.or.replace.enabled" : "true", + "ksql.shared.runtimes.count" : "2", + "ksql.cast.strings.preserve.nulls" : "true", + "ksql.authorization.cache.max.entries" : "10000", + "ksql.pull.queries.enable" : "true", + "ksql.transient.query.cleanup.service.period.seconds" : "600", + "ksql.suppress.enabled" : "false", + "ksql.readonly.topics" : "_confluent.*,__confluent.*,_schemas,__consumer_offsets,__transaction_state,connect-configs,connect-offsets,connect-status,connect-statuses", + "ksql.connect.basic.auth.credentials.file" : "", + "ksql.authorization.cache.expiry.time.secs" : "30", + "ksql.query.retry.backoff.initial.ms" : "15000", + "ksql.query.pull.max.concurrent.requests" : "2147483647", + "ksql.streams.auto.offset.reset" : "earliest", + "ksql.connect.url" : "http://localhost:8083", + "ksql.query.push.v2.new.latest.delay.ms" : "5000", + "ksql.query.push.v2.latest.reset.age.ms" : "30000", + "ksql.streams.default.production.exception.handler" : "io.confluent.ksql.errors.ProductionExceptionHandlerUtil$LogAndFailProductionExceptionHandler", + "ksql.query.pull.interpreter.enabled" : "true", + "ksql.query.pull.limit.clause.enabled" : "true", + "ksql.query.pull.router.thread.pool.size" : "50", + "ksql.query.push.v2.continuation.tokens.enabled" : "false", + "ksql.query.retry.backoff.max.ms" : "900000", + "ksql.timestamp.throw.on.invalid" : "false", + "ksql.persistence.default.format.value" : null, + "ksql.udfs.enabled" : "true", + "ksql.udf.enable.security.manager" : "true", + "ksql.connect.worker.config" : "", + "ksql.nested.error.set.null" : "true", + "ksql.query.pull.thread.pool.size" : "50", + "ksql.persistent.prefix" : "query_", + "ksql.metastore.backup.location" : "", + "ksql.error.classifier.regex" : "", + "ksql.suppress.buffer.size.bytes" : "-1" + } +} \ No newline at end of file diff --git a/ksqldb-functional-tests/src/test/resources/historical_plans/collect-list_-_collect_list_bool_map_table_-_PROTOBUF_NOSR/7.3.0_1651069580920/spec.json b/ksqldb-functional-tests/src/test/resources/historical_plans/collect-list_-_collect_list_bool_map_table_-_PROTOBUF_NOSR/7.3.0_1651069580920/spec.json new file mode 100644 index 000000000000..4991801e6974 --- /dev/null +++ b/ksqldb-functional-tests/src/test/resources/historical_plans/collect-list_-_collect_list_bool_map_table_-_PROTOBUF_NOSR/7.3.0_1651069580920/spec.json @@ -0,0 +1,215 @@ +{ + "version" : "7.3.0", + "timestamp" : 1651069580920, + "path" : "query-validation-tests/collect-list.json", + "schemas" : { + "CTAS_S2_0.Aggregate.Aggregate.Materialize" : { + "schema" : "`ID` BIGINT KEY, `ID` BIGINT, `VALUE` MAP, `KSQL_AGG_VARIABLE_0` ARRAY", + "keyFormat" : { + "format" : "KAFKA" + }, + "valueFormat" : { + "format" : "PROTOBUF_NOSR" + } + }, + "CTAS_S2_0.KsqlTopic.Source.Materialized" : { + "schema" : "`ID` BIGINT KEY, `NAME` STRING, `VALUE` MAP, `ROWPARTITION` INTEGER, `ROWOFFSET` BIGINT", + "keyFormat" : { + "format" : "KAFKA" + }, + "valueFormat" : { + "format" : "PROTOBUF_NOSR" + } + }, + "CTAS_S2_0.Aggregate.Project" : { + "schema" : "`ID` BIGINT KEY, `COLLECTED` ARRAY", + "keyFormat" : { + "format" : "KAFKA" + }, + "valueFormat" : { + "format" : "PROTOBUF_NOSR" + } + }, + "CTAS_S2_0.Aggregate.Prepare" : { + "schema" : "`ID` BIGINT KEY, `ID` BIGINT, `VALUE` MAP, `KSQL_INTERNAL_COL_2` BOOLEAN", + "keyFormat" : { + "format" : "KAFKA" + }, + "valueFormat" : { + "format" : "PROTOBUF_NOSR" + } + }, + "CTAS_S2_0.KsqlTopic.Source" : { + "schema" : "`ID` BIGINT KEY, `NAME` STRING, `VALUE` MAP", + "keyFormat" : { + "format" : "KAFKA" + }, + "valueFormat" : { + "format" : "PROTOBUF_NOSR" + } + }, + "CTAS_S2_0.S2" : { + "schema" : "`ID` BIGINT KEY, `COLLECTED` ARRAY", + "keyFormat" : { + "format" : "KAFKA" + }, + "valueFormat" : { + "format" : "PROTOBUF_NOSR" + } + }, + "CTAS_S2_0.Aggregate.GroupBy" : { + "schema" : "`ID` BIGINT KEY, `ID` BIGINT, `VALUE` MAP, `KSQL_INTERNAL_COL_2` BOOLEAN", + "keyFormat" : { + "format" : "KAFKA" + }, + "valueFormat" : { + "format" : "PROTOBUF_NOSR" + } + } + }, + "testCase" : { + "name" : "collect_list bool map table - PROTOBUF_NOSR", + "inputs" : [ { + "topic" : "test_topic", + "key" : 0, + "value" : { + "name" : "zero", + "value" : { + "key1" : true, + "key2" : false + } + } + }, { + "topic" : "test_topic", + "key" : 0, + "value" : { + "name" : "zero", + "value" : { + "key1" : false, + "key2" : true + } + } + }, { + "topic" : "test_topic", + "key" : 0, + "value" : { + "name" : "zero", + "value" : { + "key1" : true, + "key2" : true + } + } + } ], + "outputs" : [ { + "topic" : "S2", + "key" : 0, + "value" : { + "COLLECTED" : [ true ] + } + }, { + "topic" : "S2", + "key" : 0, + "value" : { + "COLLECTED" : [ ] + } + }, { + "topic" : "S2", + "key" : 0, + "value" : { + "COLLECTED" : [ false ] + } + }, { + "topic" : "S2", + "key" : 0, + "value" : { + "COLLECTED" : [ ] + } + }, { + "topic" : "S2", + "key" : 0, + "value" : { + "COLLECTED" : [ true ] + } + } ], + "topics" : [ { + "name" : "test_topic", + "replicas" : 1, + "numPartitions" : 4 + }, { + "name" : "S2", + "replicas" : 1, + "numPartitions" : 4 + } ], + "statements" : [ "CREATE TABLE TEST (ID BIGINT PRIMARY KEY, NAME varchar, VALUE map) WITH (kafka_topic='test_topic', value_format='PROTOBUF_NOSR');", "CREATE TABLE S2 as SELECT ID, collect_list(value['key1']) AS collected FROM test group by id;" ], + "post" : { + "sources" : [ { + "name" : "S2", + "type" : "TABLE", + "schema" : "`ID` BIGINT KEY, `COLLECTED` ARRAY", + "keyFormat" : { + "format" : "KAFKA" + }, + "valueFormat" : "PROTOBUF_NOSR", + "keyFeatures" : [ ], + "valueFeatures" : [ ], + "isSource" : false + }, { + "name" : "TEST", + "type" : "TABLE", + "schema" : "`ID` BIGINT KEY, `NAME` STRING, `VALUE` MAP", + "keyFormat" : { + "format" : "KAFKA" + }, + "valueFormat" : "PROTOBUF_NOSR", + "keyFeatures" : [ ], + "valueFeatures" : [ ], + "isSource" : false + } ], + "topics" : { + "topics" : [ { + "name" : "_confluent-ksql-some.ksql.service.idquery_CTAS_S2_0-Aggregate-Aggregate-Materialize-changelog", + "keyFormat" : { + "format" : "KAFKA" + }, + "valueFormat" : { + "format" : "PROTOBUF_NOSR" + } + }, { + "name" : "_confluent-ksql-some.ksql.service.idquery_CTAS_S2_0-KsqlTopic-Reduce-changelog", + "keyFormat" : { + "format" : "KAFKA" + }, + "valueFormat" : { + "format" : "PROTOBUF_NOSR" + } + }, { + "name" : "test_topic", + "keyFormat" : { + "format" : "KAFKA" + }, + "valueFormat" : { + "format" : "PROTOBUF_NOSR" + }, + "partitions" : 4 + }, { + "name" : "_confluent-ksql-some.ksql.service.idquery_CTAS_S2_0-Aggregate-GroupBy-repartition", + "keyFormat" : { + "format" : "KAFKA" + }, + "valueFormat" : { + "format" : "PROTOBUF_NOSR" + } + }, { + "name" : "S2", + "keyFormat" : { + "format" : "KAFKA" + }, + "valueFormat" : { + "format" : "PROTOBUF_NOSR" + }, + "partitions" : 4 + } ] + } + } + } +} \ No newline at end of file diff --git a/ksqldb-functional-tests/src/test/resources/historical_plans/collect-list_-_collect_list_bool_map_table_-_PROTOBUF_NOSR/7.3.0_1651069580920/topology b/ksqldb-functional-tests/src/test/resources/historical_plans/collect-list_-_collect_list_bool_map_table_-_PROTOBUF_NOSR/7.3.0_1651069580920/topology new file mode 100644 index 000000000000..fcff87be4084 --- /dev/null +++ b/ksqldb-functional-tests/src/test/resources/historical_plans/collect-list_-_collect_list_bool_map_table_-_PROTOBUF_NOSR/7.3.0_1651069580920/topology @@ -0,0 +1,43 @@ +Topologies: + Sub-topology: 0 + Source: KSTREAM-SOURCE-0000000001 (topics: [test_topic]) + --> KTABLE-SOURCE-0000000002 + Processor: KTABLE-SOURCE-0000000002 (stores: []) + --> KTABLE-TRANSFORMVALUES-0000000003 + <-- KSTREAM-SOURCE-0000000001 + Processor: KTABLE-TRANSFORMVALUES-0000000003 (stores: [KsqlTopic-Reduce]) + --> KTABLE-TRANSFORMVALUES-0000000004 + <-- KTABLE-SOURCE-0000000002 + Processor: KTABLE-TRANSFORMVALUES-0000000004 (stores: []) + --> Aggregate-Prepare + <-- KTABLE-TRANSFORMVALUES-0000000003 + Processor: Aggregate-Prepare (stores: []) + --> KTABLE-FILTER-0000000006 + <-- KTABLE-TRANSFORMVALUES-0000000004 + Processor: KTABLE-FILTER-0000000006 (stores: []) + --> Aggregate-GroupBy + <-- Aggregate-Prepare + Processor: Aggregate-GroupBy (stores: []) + --> KSTREAM-SINK-0000000008 + <-- KTABLE-FILTER-0000000006 + Sink: KSTREAM-SINK-0000000008 (topic: Aggregate-GroupBy-repartition) + <-- Aggregate-GroupBy + + Sub-topology: 1 + Source: KSTREAM-SOURCE-0000000009 (topics: [Aggregate-GroupBy-repartition]) + --> KTABLE-AGGREGATE-0000000010 + Processor: KTABLE-AGGREGATE-0000000010 (stores: [Aggregate-Aggregate-Materialize]) + --> Aggregate-Aggregate-ToOutputSchema + <-- KSTREAM-SOURCE-0000000009 + Processor: Aggregate-Aggregate-ToOutputSchema (stores: []) + --> Aggregate-Project + <-- KTABLE-AGGREGATE-0000000010 + Processor: Aggregate-Project (stores: []) + --> KTABLE-TOSTREAM-0000000013 + <-- Aggregate-Aggregate-ToOutputSchema + Processor: KTABLE-TOSTREAM-0000000013 (stores: []) + --> KSTREAM-SINK-0000000014 + <-- Aggregate-Project + Sink: KSTREAM-SINK-0000000014 (topic: S2) + <-- KTABLE-TOSTREAM-0000000013 + diff --git a/ksqldb-functional-tests/src/test/resources/historical_plans/collect-list_-_collect_list_date_map_-_PROTOBUF_NOSR/7.3.0_1651069550734/plan.json b/ksqldb-functional-tests/src/test/resources/historical_plans/collect-list_-_collect_list_date_map_-_PROTOBUF_NOSR/7.3.0_1651069550734/plan.json new file mode 100644 index 000000000000..625fa7f45292 --- /dev/null +++ b/ksqldb-functional-tests/src/test/resources/historical_plans/collect-list_-_collect_list_date_map_-_PROTOBUF_NOSR/7.3.0_1651069550734/plan.json @@ -0,0 +1,255 @@ +{ + "plan" : [ { + "@type" : "ksqlPlanV1", + "statementText" : "CREATE STREAM TEST (ID BIGINT KEY, NAME STRING, VALUE MAP) WITH (KAFKA_TOPIC='test_topic', KEY_FORMAT='KAFKA', VALUE_FORMAT='PROTOBUF_NOSR');", + "ddlCommand" : { + "@type" : "createStreamV1", + "sourceName" : "TEST", + "schema" : "`ID` BIGINT KEY, `NAME` STRING, `VALUE` MAP", + "timestampColumn" : null, + "topicName" : "test_topic", + "formats" : { + "keyFormat" : { + "format" : "KAFKA", + "properties" : { } + }, + "valueFormat" : { + "format" : "PROTOBUF_NOSR", + "properties" : { } + } + }, + "windowInfo" : null, + "orReplace" : false, + "isSource" : false + }, + "queryPlan" : null + }, { + "@type" : "ksqlPlanV1", + "statementText" : "CREATE TABLE S2 AS SELECT\n TEST.ID ID,\n COLLECT_LIST(TEST.VALUE['key1']) COLLECTED\nFROM TEST TEST\nGROUP BY TEST.ID\nEMIT CHANGES", + "ddlCommand" : { + "@type" : "createTableV1", + "sourceName" : "S2", + "schema" : "`ID` BIGINT KEY, `COLLECTED` ARRAY", + "timestampColumn" : null, + "topicName" : "S2", + "formats" : { + "keyFormat" : { + "format" : "KAFKA", + "properties" : { } + }, + "valueFormat" : { + "format" : "PROTOBUF_NOSR", + "properties" : { } + } + }, + "windowInfo" : null, + "orReplace" : false, + "isSource" : false + }, + "queryPlan" : { + "sources" : [ "TEST" ], + "sink" : "S2", + "physicalPlan" : { + "@type" : "tableSinkV1", + "properties" : { + "queryContext" : "S2" + }, + "source" : { + "@type" : "tableSelectV1", + "properties" : { + "queryContext" : "Aggregate/Project" + }, + "source" : { + "@type" : "streamAggregateV1", + "properties" : { + "queryContext" : "Aggregate/Aggregate" + }, + "source" : { + "@type" : "streamGroupByKeyV1", + "properties" : { + "queryContext" : "Aggregate/GroupBy" + }, + "source" : { + "@type" : "streamSelectV1", + "properties" : { + "queryContext" : "Aggregate/Prepare" + }, + "source" : { + "@type" : "streamSourceV1", + "properties" : { + "queryContext" : "KsqlTopic/Source" + }, + "topicName" : "test_topic", + "formats" : { + "keyFormat" : { + "format" : "KAFKA", + "properties" : { } + }, + "valueFormat" : { + "format" : "PROTOBUF_NOSR", + "properties" : { } + } + }, + "timestampColumn" : null, + "sourceSchema" : "`ID` BIGINT KEY, `NAME` STRING, `VALUE` MAP", + "pseudoColumnVersion" : 1 + }, + "keyColumnNames" : [ "ID" ], + "selectedKeys" : null, + "selectExpressions" : [ "ID AS ID", "VALUE AS VALUE", "VALUE['key1'] AS KSQL_INTERNAL_COL_2" ] + }, + "internalFormats" : { + "keyFormat" : { + "format" : "KAFKA", + "properties" : { } + }, + "valueFormat" : { + "format" : "PROTOBUF_NOSR", + "properties" : { } + } + } + }, + "internalFormats" : { + "keyFormat" : { + "format" : "KAFKA", + "properties" : { } + }, + "valueFormat" : { + "format" : "PROTOBUF_NOSR", + "properties" : { } + } + }, + "nonAggregateColumns" : [ "ID", "VALUE" ], + "aggregationFunctions" : [ "COLLECT_LIST(KSQL_INTERNAL_COL_2)" ] + }, + "keyColumnNames" : [ "ID" ], + "selectExpressions" : [ "KSQL_AGG_VARIABLE_0 AS COLLECTED" ], + "internalFormats" : { + "keyFormat" : { + "format" : "KAFKA", + "properties" : { } + }, + "valueFormat" : { + "format" : "PROTOBUF_NOSR", + "properties" : { } + } + } + }, + "formats" : { + "keyFormat" : { + "format" : "KAFKA", + "properties" : { } + }, + "valueFormat" : { + "format" : "PROTOBUF_NOSR", + "properties" : { } + } + }, + "topicName" : "S2", + "timestampColumn" : null + }, + "queryId" : "CTAS_S2_0", + "runtimeId" : null + } + } ], + "configs" : { + "ksql.extension.dir" : "ext", + "ksql.streams.cache.max.bytes.buffering" : "0", + "metric.reporters" : "", + "ksql.query.status.running.threshold.seconds" : "300", + "ksql.connect.basic.auth.credentials.reload" : "false", + "ksql.output.topic.name.prefix" : "", + "ksql.query.pull.stream.enabled" : "true", + "ksql.query.push.v2.interpreter.enabled" : "true", + "ksql.queryanonymizer.logs_enabled" : "true", + "ksql.variable.substitution.enable" : "true", + "ksql.streams.shutdown.timeout.ms" : "300000", + "ksql.query.pull.max.allowed.offset.lag" : "9223372036854775807", + "ksql.query.pull.max.qps" : "2147483647", + "ksql.access.validator.enable" : "auto", + "ksql.streams.bootstrap.servers" : "localhost:0", + "ksql.query.pull.metrics.enabled" : "true", + "ksql.metrics.extension" : null, + "ksql.query.push.v2.alos.enabled" : "true", + "ksql.query.push.v2.max.hourly.bandwidth.megabytes" : "2147483647", + "ksql.query.pull.range.scan.enabled" : "true", + "ksql.transient.query.cleanup.service.initial.delay.seconds" : "600", + "ksql.hidden.topics" : "_confluent.*,__confluent.*,_schemas,__consumer_offsets,__transaction_state,connect-configs,connect-offsets,connect-status,connect-statuses", + "ksql.lambdas.enabled" : "true", + "ksql.source.table.materialization.enabled" : "true", + "ksql.query.pull.max.hourly.bandwidth.megabytes" : "2147483647", + "ksql.sink.window.change.log.additional.retention" : "1000000", + "ksql.query.persistent.active.limit" : "2147483647", + "ksql.persistence.wrap.single.values" : null, + "ksql.query.transient.max.bytes.buffering.total" : "-1", + "ksql.connect.basic.auth.credentials.source" : "NONE", + "ksql.schema.registry.url" : "schema_registry.url:0", + "ksql.properties.overrides.denylist" : "", + "ksql.service.id" : "some.ksql.service.id", + "ksql.query.push.v2.max.catchup.consumers" : "5", + "ksql.query.push.v2.enabled" : "false", + "ksql.transient.query.cleanup.service.enable" : "true", + "ksql.query.push.v2.metrics.enabled" : "true", + "ksql.rowpartition.rowoffset.enabled" : "true", + "ksql.streams.commit.interval.ms" : "2000", + "ksql.query.pull.table.scan.enabled" : "true", + "ksql.streams.auto.commit.interval.ms" : "0", + "ksql.streams.topology.optimization" : "all", + "ksql.endpoint.migrate.query" : "true", + "ksql.query.push.v2.registry.installed" : "false", + "ksql.streams.num.stream.threads" : "4", + "ksql.metrics.tags.custom" : "", + "ksql.query.push.v2.catchup.consumer.msg.window" : "50", + "ksql.runtime.feature.shared.enabled" : "false", + "ksql.udf.collect.metrics" : "false", + "ksql.new.query.planner.enabled" : "false", + "ksql.connect.request.headers.plugin" : null, + "ksql.security.extension.class" : null, + "ksql.transient.prefix" : "transient_", + "ksql.headers.columns.enabled" : "true", + "ksql.streams.default.deserialization.exception.handler" : "io.confluent.ksql.errors.LogMetricAndContinueExceptionHandler", + "ksql.connect.request.timeout.ms" : "5000", + "ksql.query.pull.enable.standby.reads" : "false", + "ksql.persistence.default.format.key" : "KAFKA", + "ksql.query.persistent.max.bytes.buffering.total" : "-1", + "ksql.query.error.max.queue.size" : "10", + "ksql.query.cleanup.shutdown.timeout.ms" : "30000", + "ksql.internal.topic.min.insync.replicas" : "1", + "ksql.internal.topic.replicas" : "1", + "ksql.insert.into.values.enabled" : "true", + "ksql.queryanonymizer.cluster_namespace" : null, + "ksql.create.or.replace.enabled" : "true", + "ksql.shared.runtimes.count" : "2", + "ksql.cast.strings.preserve.nulls" : "true", + "ksql.authorization.cache.max.entries" : "10000", + "ksql.pull.queries.enable" : "true", + "ksql.transient.query.cleanup.service.period.seconds" : "600", + "ksql.suppress.enabled" : "false", + "ksql.readonly.topics" : "_confluent.*,__confluent.*,_schemas,__consumer_offsets,__transaction_state,connect-configs,connect-offsets,connect-status,connect-statuses", + "ksql.connect.basic.auth.credentials.file" : "", + "ksql.authorization.cache.expiry.time.secs" : "30", + "ksql.query.retry.backoff.initial.ms" : "15000", + "ksql.query.pull.max.concurrent.requests" : "2147483647", + "ksql.streams.auto.offset.reset" : "earliest", + "ksql.connect.url" : "http://localhost:8083", + "ksql.query.push.v2.new.latest.delay.ms" : "5000", + "ksql.query.push.v2.latest.reset.age.ms" : "30000", + "ksql.streams.default.production.exception.handler" : "io.confluent.ksql.errors.ProductionExceptionHandlerUtil$LogAndFailProductionExceptionHandler", + "ksql.query.pull.interpreter.enabled" : "true", + "ksql.query.pull.limit.clause.enabled" : "true", + "ksql.query.pull.router.thread.pool.size" : "50", + "ksql.query.push.v2.continuation.tokens.enabled" : "false", + "ksql.query.retry.backoff.max.ms" : "900000", + "ksql.timestamp.throw.on.invalid" : "false", + "ksql.persistence.default.format.value" : null, + "ksql.udfs.enabled" : "true", + "ksql.udf.enable.security.manager" : "true", + "ksql.connect.worker.config" : "", + "ksql.nested.error.set.null" : "true", + "ksql.query.pull.thread.pool.size" : "50", + "ksql.persistent.prefix" : "query_", + "ksql.metastore.backup.location" : "", + "ksql.error.classifier.regex" : "", + "ksql.suppress.buffer.size.bytes" : "-1" + } +} \ No newline at end of file diff --git a/ksqldb-functional-tests/src/test/resources/historical_plans/collect-list_-_collect_list_date_map_-_PROTOBUF_NOSR/7.3.0_1651069550734/spec.json b/ksqldb-functional-tests/src/test/resources/historical_plans/collect-list_-_collect_list_date_map_-_PROTOBUF_NOSR/7.3.0_1651069550734/spec.json new file mode 100644 index 000000000000..2dd073e44dee --- /dev/null +++ b/ksqldb-functional-tests/src/test/resources/historical_plans/collect-list_-_collect_list_date_map_-_PROTOBUF_NOSR/7.3.0_1651069550734/spec.json @@ -0,0 +1,169 @@ +{ + "version" : "7.3.0", + "timestamp" : 1651069550734, + "path" : "query-validation-tests/collect-list.json", + "schemas" : { + "CTAS_S2_0.Aggregate.Aggregate.Materialize" : { + "schema" : "`ID` BIGINT KEY, `ID` BIGINT, `VALUE` MAP, `KSQL_AGG_VARIABLE_0` ARRAY", + "keyFormat" : { + "format" : "KAFKA" + }, + "valueFormat" : { + "format" : "PROTOBUF_NOSR" + } + }, + "CTAS_S2_0.Aggregate.Project" : { + "schema" : "`ID` BIGINT KEY, `COLLECTED` ARRAY", + "keyFormat" : { + "format" : "KAFKA" + }, + "valueFormat" : { + "format" : "PROTOBUF_NOSR" + } + }, + "CTAS_S2_0.KsqlTopic.Source" : { + "schema" : "`ID` BIGINT KEY, `NAME` STRING, `VALUE` MAP", + "keyFormat" : { + "format" : "KAFKA" + }, + "valueFormat" : { + "format" : "PROTOBUF_NOSR" + } + }, + "CTAS_S2_0.S2" : { + "schema" : "`ID` BIGINT KEY, `COLLECTED` ARRAY", + "keyFormat" : { + "format" : "KAFKA" + }, + "valueFormat" : { + "format" : "PROTOBUF_NOSR" + } + }, + "CTAS_S2_0.Aggregate.GroupBy" : { + "schema" : "`ID` BIGINT KEY, `ID` BIGINT, `VALUE` MAP, `KSQL_INTERNAL_COL_2` DATE", + "keyFormat" : { + "format" : "KAFKA" + }, + "valueFormat" : { + "format" : "PROTOBUF_NOSR" + } + } + }, + "testCase" : { + "name" : "collect_list date map - PROTOBUF_NOSR", + "inputs" : [ { + "topic" : "test_topic", + "key" : 0, + "value" : { + "name" : "zero", + "value" : { + "key1" : 10, + "key2" : 15 + } + } + }, { + "topic" : "test_topic", + "key" : 0, + "value" : { + "name" : "zero", + "value" : { + "key1" : 20, + "key2" : 25 + } + } + }, { + "topic" : "test_topic", + "key" : 0, + "value" : { + "name" : "zero", + "value" : { + "key1" : 30, + "key2" : 35 + } + } + } ], + "outputs" : [ { + "topic" : "S2", + "key" : 0, + "value" : { + "COLLECTED" : [ 10 ] + } + }, { + "topic" : "S2", + "key" : 0, + "value" : { + "COLLECTED" : [ 10, 20 ] + } + }, { + "topic" : "S2", + "key" : 0, + "value" : { + "COLLECTED" : [ 10, 20, 30 ] + } + } ], + "topics" : [ { + "name" : "test_topic", + "replicas" : 1, + "numPartitions" : 4 + }, { + "name" : "S2", + "replicas" : 1, + "numPartitions" : 4 + } ], + "statements" : [ "CREATE STREAM TEST (ID BIGINT KEY, NAME varchar, VALUE map) WITH (kafka_topic='test_topic', value_format='PROTOBUF_NOSR');", "CREATE TABLE S2 as SELECT ID, collect_list(value['key1']) AS collected FROM test group by id;" ], + "post" : { + "sources" : [ { + "name" : "S2", + "type" : "TABLE", + "schema" : "`ID` BIGINT KEY, `COLLECTED` ARRAY", + "keyFormat" : { + "format" : "KAFKA" + }, + "valueFormat" : "PROTOBUF_NOSR", + "keyFeatures" : [ ], + "valueFeatures" : [ ], + "isSource" : false + }, { + "name" : "TEST", + "type" : "STREAM", + "schema" : "`ID` BIGINT KEY, `NAME` STRING, `VALUE` MAP", + "keyFormat" : { + "format" : "KAFKA" + }, + "valueFormat" : "PROTOBUF_NOSR", + "keyFeatures" : [ ], + "valueFeatures" : [ ], + "isSource" : false + } ], + "topics" : { + "topics" : [ { + "name" : "_confluent-ksql-some.ksql.service.idquery_CTAS_S2_0-Aggregate-Aggregate-Materialize-changelog", + "keyFormat" : { + "format" : "KAFKA" + }, + "valueFormat" : { + "format" : "PROTOBUF_NOSR" + } + }, { + "name" : "test_topic", + "keyFormat" : { + "format" : "KAFKA" + }, + "valueFormat" : { + "format" : "PROTOBUF_NOSR" + }, + "partitions" : 4 + }, { + "name" : "S2", + "keyFormat" : { + "format" : "KAFKA" + }, + "valueFormat" : { + "format" : "PROTOBUF_NOSR" + }, + "partitions" : 4 + } ] + } + } + } +} \ No newline at end of file diff --git a/ksqldb-functional-tests/src/test/resources/historical_plans/collect-list_-_collect_list_date_map_-_PROTOBUF_NOSR/7.3.0_1651069550734/topology b/ksqldb-functional-tests/src/test/resources/historical_plans/collect-list_-_collect_list_date_map_-_PROTOBUF_NOSR/7.3.0_1651069550734/topology new file mode 100644 index 000000000000..fcfc8fb28665 --- /dev/null +++ b/ksqldb-functional-tests/src/test/resources/historical_plans/collect-list_-_collect_list_date_map_-_PROTOBUF_NOSR/7.3.0_1651069550734/topology @@ -0,0 +1,25 @@ +Topologies: + Sub-topology: 0 + Source: KSTREAM-SOURCE-0000000000 (topics: [test_topic]) + --> KSTREAM-TRANSFORMVALUES-0000000001 + Processor: KSTREAM-TRANSFORMVALUES-0000000001 (stores: []) + --> Aggregate-Prepare + <-- KSTREAM-SOURCE-0000000000 + Processor: Aggregate-Prepare (stores: []) + --> KSTREAM-AGGREGATE-0000000003 + <-- KSTREAM-TRANSFORMVALUES-0000000001 + Processor: KSTREAM-AGGREGATE-0000000003 (stores: [Aggregate-Aggregate-Materialize]) + --> Aggregate-Aggregate-ToOutputSchema + <-- Aggregate-Prepare + Processor: Aggregate-Aggregate-ToOutputSchema (stores: []) + --> Aggregate-Project + <-- KSTREAM-AGGREGATE-0000000003 + Processor: Aggregate-Project (stores: []) + --> KTABLE-TOSTREAM-0000000006 + <-- Aggregate-Aggregate-ToOutputSchema + Processor: KTABLE-TOSTREAM-0000000006 (stores: []) + --> KSTREAM-SINK-0000000007 + <-- Aggregate-Project + Sink: KSTREAM-SINK-0000000007 (topic: S2) + <-- KTABLE-TOSTREAM-0000000006 + diff --git a/ksqldb-functional-tests/src/test/resources/historical_plans/collect-list_-_collect_list_date_map_table_-_PROTOBUF_NOSR/7.3.0_1651069593625/plan.json b/ksqldb-functional-tests/src/test/resources/historical_plans/collect-list_-_collect_list_date_map_table_-_PROTOBUF_NOSR/7.3.0_1651069593625/plan.json new file mode 100644 index 000000000000..080ebc350633 --- /dev/null +++ b/ksqldb-functional-tests/src/test/resources/historical_plans/collect-list_-_collect_list_date_map_table_-_PROTOBUF_NOSR/7.3.0_1651069593625/plan.json @@ -0,0 +1,275 @@ +{ + "plan" : [ { + "@type" : "ksqlPlanV1", + "statementText" : "CREATE TABLE TEST (ID BIGINT PRIMARY KEY, NAME STRING, VALUE MAP) WITH (KAFKA_TOPIC='test_topic', KEY_FORMAT='KAFKA', VALUE_FORMAT='PROTOBUF_NOSR');", + "ddlCommand" : { + "@type" : "createTableV1", + "sourceName" : "TEST", + "schema" : "`ID` BIGINT KEY, `NAME` STRING, `VALUE` MAP", + "timestampColumn" : null, + "topicName" : "test_topic", + "formats" : { + "keyFormat" : { + "format" : "KAFKA", + "properties" : { } + }, + "valueFormat" : { + "format" : "PROTOBUF_NOSR", + "properties" : { } + } + }, + "windowInfo" : null, + "orReplace" : false, + "isSource" : false + }, + "queryPlan" : null + }, { + "@type" : "ksqlPlanV1", + "statementText" : "CREATE TABLE S2 AS SELECT\n TEST.ID ID,\n COLLECT_LIST(TEST.VALUE['key1']) COLLECTED\nFROM TEST TEST\nGROUP BY TEST.ID\nEMIT CHANGES", + "ddlCommand" : { + "@type" : "createTableV1", + "sourceName" : "S2", + "schema" : "`ID` BIGINT KEY, `COLLECTED` ARRAY", + "timestampColumn" : null, + "topicName" : "S2", + "formats" : { + "keyFormat" : { + "format" : "KAFKA", + "properties" : { } + }, + "valueFormat" : { + "format" : "PROTOBUF_NOSR", + "properties" : { } + } + }, + "windowInfo" : null, + "orReplace" : false, + "isSource" : false + }, + "queryPlan" : { + "sources" : [ "TEST" ], + "sink" : "S2", + "physicalPlan" : { + "@type" : "tableSinkV1", + "properties" : { + "queryContext" : "S2" + }, + "source" : { + "@type" : "tableSelectV1", + "properties" : { + "queryContext" : "Aggregate/Project" + }, + "source" : { + "@type" : "tableAggregateV1", + "properties" : { + "queryContext" : "Aggregate/Aggregate" + }, + "source" : { + "@type" : "tableGroupByV2", + "properties" : { + "queryContext" : "Aggregate/GroupBy" + }, + "source" : { + "@type" : "tableSelectV1", + "properties" : { + "queryContext" : "Aggregate/Prepare" + }, + "source" : { + "@type" : "tableSourceV2", + "properties" : { + "queryContext" : "KsqlTopic/Source" + }, + "topicName" : "test_topic", + "formats" : { + "keyFormat" : { + "format" : "KAFKA", + "properties" : { } + }, + "valueFormat" : { + "format" : "PROTOBUF_NOSR", + "properties" : { } + } + }, + "timestampColumn" : null, + "sourceSchema" : "`ID` BIGINT KEY, `NAME` STRING, `VALUE` MAP", + "pseudoColumnVersion" : 1, + "stateStoreFormats" : { + "keyFormat" : { + "format" : "KAFKA", + "properties" : { } + }, + "valueFormat" : { + "format" : "PROTOBUF_NOSR", + "properties" : { } + } + } + }, + "keyColumnNames" : [ "ID" ], + "selectExpressions" : [ "ID AS ID", "VALUE AS VALUE", "VALUE['key1'] AS KSQL_INTERNAL_COL_2" ], + "internalFormats" : { + "keyFormat" : { + "format" : "KAFKA", + "properties" : { } + }, + "valueFormat" : { + "format" : "PROTOBUF_NOSR", + "properties" : { } + } + } + }, + "internalFormats" : { + "keyFormat" : { + "format" : "KAFKA", + "properties" : { } + }, + "valueFormat" : { + "format" : "PROTOBUF_NOSR", + "properties" : { } + } + }, + "groupByExpressions" : [ "ID" ] + }, + "internalFormats" : { + "keyFormat" : { + "format" : "KAFKA", + "properties" : { } + }, + "valueFormat" : { + "format" : "PROTOBUF_NOSR", + "properties" : { } + } + }, + "nonAggregateColumns" : [ "ID", "VALUE" ], + "aggregationFunctions" : [ "COLLECT_LIST(KSQL_INTERNAL_COL_2)" ] + }, + "keyColumnNames" : [ "ID" ], + "selectExpressions" : [ "KSQL_AGG_VARIABLE_0 AS COLLECTED" ], + "internalFormats" : { + "keyFormat" : { + "format" : "KAFKA", + "properties" : { } + }, + "valueFormat" : { + "format" : "PROTOBUF_NOSR", + "properties" : { } + } + } + }, + "formats" : { + "keyFormat" : { + "format" : "KAFKA", + "properties" : { } + }, + "valueFormat" : { + "format" : "PROTOBUF_NOSR", + "properties" : { } + } + }, + "topicName" : "S2", + "timestampColumn" : null + }, + "queryId" : "CTAS_S2_0", + "runtimeId" : null + } + } ], + "configs" : { + "ksql.extension.dir" : "ext", + "ksql.streams.cache.max.bytes.buffering" : "0", + "metric.reporters" : "", + "ksql.query.status.running.threshold.seconds" : "300", + "ksql.connect.basic.auth.credentials.reload" : "false", + "ksql.output.topic.name.prefix" : "", + "ksql.query.pull.stream.enabled" : "true", + "ksql.query.push.v2.interpreter.enabled" : "true", + "ksql.queryanonymizer.logs_enabled" : "true", + "ksql.variable.substitution.enable" : "true", + "ksql.streams.shutdown.timeout.ms" : "300000", + "ksql.query.pull.max.allowed.offset.lag" : "9223372036854775807", + "ksql.query.pull.max.qps" : "2147483647", + "ksql.access.validator.enable" : "auto", + "ksql.streams.bootstrap.servers" : "localhost:0", + "ksql.query.pull.metrics.enabled" : "true", + "ksql.metrics.extension" : null, + "ksql.query.push.v2.alos.enabled" : "true", + "ksql.query.push.v2.max.hourly.bandwidth.megabytes" : "2147483647", + "ksql.query.pull.range.scan.enabled" : "true", + "ksql.transient.query.cleanup.service.initial.delay.seconds" : "600", + "ksql.hidden.topics" : "_confluent.*,__confluent.*,_schemas,__consumer_offsets,__transaction_state,connect-configs,connect-offsets,connect-status,connect-statuses", + "ksql.lambdas.enabled" : "true", + "ksql.source.table.materialization.enabled" : "true", + "ksql.query.pull.max.hourly.bandwidth.megabytes" : "2147483647", + "ksql.sink.window.change.log.additional.retention" : "1000000", + "ksql.query.persistent.active.limit" : "2147483647", + "ksql.persistence.wrap.single.values" : null, + "ksql.query.transient.max.bytes.buffering.total" : "-1", + "ksql.connect.basic.auth.credentials.source" : "NONE", + "ksql.schema.registry.url" : "schema_registry.url:0", + "ksql.properties.overrides.denylist" : "", + "ksql.service.id" : "some.ksql.service.id", + "ksql.query.push.v2.max.catchup.consumers" : "5", + "ksql.query.push.v2.enabled" : "false", + "ksql.transient.query.cleanup.service.enable" : "true", + "ksql.query.push.v2.metrics.enabled" : "true", + "ksql.rowpartition.rowoffset.enabled" : "true", + "ksql.streams.commit.interval.ms" : "2000", + "ksql.query.pull.table.scan.enabled" : "true", + "ksql.streams.auto.commit.interval.ms" : "0", + "ksql.streams.topology.optimization" : "all", + "ksql.endpoint.migrate.query" : "true", + "ksql.query.push.v2.registry.installed" : "false", + "ksql.streams.num.stream.threads" : "4", + "ksql.metrics.tags.custom" : "", + "ksql.query.push.v2.catchup.consumer.msg.window" : "50", + "ksql.runtime.feature.shared.enabled" : "false", + "ksql.udf.collect.metrics" : "false", + "ksql.new.query.planner.enabled" : "false", + "ksql.connect.request.headers.plugin" : null, + "ksql.security.extension.class" : null, + "ksql.transient.prefix" : "transient_", + "ksql.headers.columns.enabled" : "true", + "ksql.streams.default.deserialization.exception.handler" : "io.confluent.ksql.errors.LogMetricAndContinueExceptionHandler", + "ksql.connect.request.timeout.ms" : "5000", + "ksql.query.pull.enable.standby.reads" : "false", + "ksql.persistence.default.format.key" : "KAFKA", + "ksql.query.persistent.max.bytes.buffering.total" : "-1", + "ksql.query.error.max.queue.size" : "10", + "ksql.query.cleanup.shutdown.timeout.ms" : "30000", + "ksql.internal.topic.min.insync.replicas" : "1", + "ksql.internal.topic.replicas" : "1", + "ksql.insert.into.values.enabled" : "true", + "ksql.queryanonymizer.cluster_namespace" : null, + "ksql.create.or.replace.enabled" : "true", + "ksql.shared.runtimes.count" : "2", + "ksql.cast.strings.preserve.nulls" : "true", + "ksql.authorization.cache.max.entries" : "10000", + "ksql.pull.queries.enable" : "true", + "ksql.transient.query.cleanup.service.period.seconds" : "600", + "ksql.suppress.enabled" : "false", + "ksql.readonly.topics" : "_confluent.*,__confluent.*,_schemas,__consumer_offsets,__transaction_state,connect-configs,connect-offsets,connect-status,connect-statuses", + "ksql.connect.basic.auth.credentials.file" : "", + "ksql.authorization.cache.expiry.time.secs" : "30", + "ksql.query.retry.backoff.initial.ms" : "15000", + "ksql.query.pull.max.concurrent.requests" : "2147483647", + "ksql.streams.auto.offset.reset" : "earliest", + "ksql.connect.url" : "http://localhost:8083", + "ksql.query.push.v2.new.latest.delay.ms" : "5000", + "ksql.query.push.v2.latest.reset.age.ms" : "30000", + "ksql.streams.default.production.exception.handler" : "io.confluent.ksql.errors.ProductionExceptionHandlerUtil$LogAndFailProductionExceptionHandler", + "ksql.query.pull.interpreter.enabled" : "true", + "ksql.query.pull.limit.clause.enabled" : "true", + "ksql.query.pull.router.thread.pool.size" : "50", + "ksql.query.push.v2.continuation.tokens.enabled" : "false", + "ksql.query.retry.backoff.max.ms" : "900000", + "ksql.timestamp.throw.on.invalid" : "false", + "ksql.persistence.default.format.value" : null, + "ksql.udfs.enabled" : "true", + "ksql.udf.enable.security.manager" : "true", + "ksql.connect.worker.config" : "", + "ksql.nested.error.set.null" : "true", + "ksql.query.pull.thread.pool.size" : "50", + "ksql.persistent.prefix" : "query_", + "ksql.metastore.backup.location" : "", + "ksql.error.classifier.regex" : "", + "ksql.suppress.buffer.size.bytes" : "-1" + } +} \ No newline at end of file diff --git a/ksqldb-functional-tests/src/test/resources/historical_plans/collect-list_-_collect_list_date_map_table_-_PROTOBUF_NOSR/7.3.0_1651069593625/spec.json b/ksqldb-functional-tests/src/test/resources/historical_plans/collect-list_-_collect_list_date_map_table_-_PROTOBUF_NOSR/7.3.0_1651069593625/spec.json new file mode 100644 index 000000000000..4bbaeee4ef29 --- /dev/null +++ b/ksqldb-functional-tests/src/test/resources/historical_plans/collect-list_-_collect_list_date_map_table_-_PROTOBUF_NOSR/7.3.0_1651069593625/spec.json @@ -0,0 +1,215 @@ +{ + "version" : "7.3.0", + "timestamp" : 1651069593625, + "path" : "query-validation-tests/collect-list.json", + "schemas" : { + "CTAS_S2_0.Aggregate.Aggregate.Materialize" : { + "schema" : "`ID` BIGINT KEY, `ID` BIGINT, `VALUE` MAP, `KSQL_AGG_VARIABLE_0` ARRAY", + "keyFormat" : { + "format" : "KAFKA" + }, + "valueFormat" : { + "format" : "PROTOBUF_NOSR" + } + }, + "CTAS_S2_0.KsqlTopic.Source.Materialized" : { + "schema" : "`ID` BIGINT KEY, `NAME` STRING, `VALUE` MAP, `ROWPARTITION` INTEGER, `ROWOFFSET` BIGINT", + "keyFormat" : { + "format" : "KAFKA" + }, + "valueFormat" : { + "format" : "PROTOBUF_NOSR" + } + }, + "CTAS_S2_0.Aggregate.Project" : { + "schema" : "`ID` BIGINT KEY, `COLLECTED` ARRAY", + "keyFormat" : { + "format" : "KAFKA" + }, + "valueFormat" : { + "format" : "PROTOBUF_NOSR" + } + }, + "CTAS_S2_0.Aggregate.Prepare" : { + "schema" : "`ID` BIGINT KEY, `ID` BIGINT, `VALUE` MAP, `KSQL_INTERNAL_COL_2` DATE", + "keyFormat" : { + "format" : "KAFKA" + }, + "valueFormat" : { + "format" : "PROTOBUF_NOSR" + } + }, + "CTAS_S2_0.KsqlTopic.Source" : { + "schema" : "`ID` BIGINT KEY, `NAME` STRING, `VALUE` MAP", + "keyFormat" : { + "format" : "KAFKA" + }, + "valueFormat" : { + "format" : "PROTOBUF_NOSR" + } + }, + "CTAS_S2_0.S2" : { + "schema" : "`ID` BIGINT KEY, `COLLECTED` ARRAY", + "keyFormat" : { + "format" : "KAFKA" + }, + "valueFormat" : { + "format" : "PROTOBUF_NOSR" + } + }, + "CTAS_S2_0.Aggregate.GroupBy" : { + "schema" : "`ID` BIGINT KEY, `ID` BIGINT, `VALUE` MAP, `KSQL_INTERNAL_COL_2` DATE", + "keyFormat" : { + "format" : "KAFKA" + }, + "valueFormat" : { + "format" : "PROTOBUF_NOSR" + } + } + }, + "testCase" : { + "name" : "collect_list date map table - PROTOBUF_NOSR", + "inputs" : [ { + "topic" : "test_topic", + "key" : 0, + "value" : { + "name" : "zero", + "value" : { + "key1" : 10, + "key2" : 15 + } + } + }, { + "topic" : "test_topic", + "key" : 0, + "value" : { + "name" : "zero", + "value" : { + "key1" : 20, + "key2" : 25 + } + } + }, { + "topic" : "test_topic", + "key" : 0, + "value" : { + "name" : "zero", + "value" : { + "key1" : 30, + "key2" : 35 + } + } + } ], + "outputs" : [ { + "topic" : "S2", + "key" : 0, + "value" : { + "COLLECTED" : [ 10 ] + } + }, { + "topic" : "S2", + "key" : 0, + "value" : { + "COLLECTED" : [ ] + } + }, { + "topic" : "S2", + "key" : 0, + "value" : { + "COLLECTED" : [ 20 ] + } + }, { + "topic" : "S2", + "key" : 0, + "value" : { + "COLLECTED" : [ ] + } + }, { + "topic" : "S2", + "key" : 0, + "value" : { + "COLLECTED" : [ 30 ] + } + } ], + "topics" : [ { + "name" : "test_topic", + "replicas" : 1, + "numPartitions" : 4 + }, { + "name" : "S2", + "replicas" : 1, + "numPartitions" : 4 + } ], + "statements" : [ "CREATE TABLE TEST (ID BIGINT PRIMARY KEY, NAME varchar, VALUE map) WITH (kafka_topic='test_topic', value_format='PROTOBUF_NOSR');", "CREATE TABLE S2 as SELECT ID, collect_list(value['key1']) AS collected FROM test group by id;" ], + "post" : { + "sources" : [ { + "name" : "S2", + "type" : "TABLE", + "schema" : "`ID` BIGINT KEY, `COLLECTED` ARRAY", + "keyFormat" : { + "format" : "KAFKA" + }, + "valueFormat" : "PROTOBUF_NOSR", + "keyFeatures" : [ ], + "valueFeatures" : [ ], + "isSource" : false + }, { + "name" : "TEST", + "type" : "TABLE", + "schema" : "`ID` BIGINT KEY, `NAME` STRING, `VALUE` MAP", + "keyFormat" : { + "format" : "KAFKA" + }, + "valueFormat" : "PROTOBUF_NOSR", + "keyFeatures" : [ ], + "valueFeatures" : [ ], + "isSource" : false + } ], + "topics" : { + "topics" : [ { + "name" : "_confluent-ksql-some.ksql.service.idquery_CTAS_S2_0-Aggregate-Aggregate-Materialize-changelog", + "keyFormat" : { + "format" : "KAFKA" + }, + "valueFormat" : { + "format" : "PROTOBUF_NOSR" + } + }, { + "name" : "_confluent-ksql-some.ksql.service.idquery_CTAS_S2_0-KsqlTopic-Reduce-changelog", + "keyFormat" : { + "format" : "KAFKA" + }, + "valueFormat" : { + "format" : "PROTOBUF_NOSR" + } + }, { + "name" : "test_topic", + "keyFormat" : { + "format" : "KAFKA" + }, + "valueFormat" : { + "format" : "PROTOBUF_NOSR" + }, + "partitions" : 4 + }, { + "name" : "_confluent-ksql-some.ksql.service.idquery_CTAS_S2_0-Aggregate-GroupBy-repartition", + "keyFormat" : { + "format" : "KAFKA" + }, + "valueFormat" : { + "format" : "PROTOBUF_NOSR" + } + }, { + "name" : "S2", + "keyFormat" : { + "format" : "KAFKA" + }, + "valueFormat" : { + "format" : "PROTOBUF_NOSR" + }, + "partitions" : 4 + } ] + } + } + } +} \ No newline at end of file diff --git a/ksqldb-functional-tests/src/test/resources/historical_plans/collect-list_-_collect_list_date_map_table_-_PROTOBUF_NOSR/7.3.0_1651069593625/topology b/ksqldb-functional-tests/src/test/resources/historical_plans/collect-list_-_collect_list_date_map_table_-_PROTOBUF_NOSR/7.3.0_1651069593625/topology new file mode 100644 index 000000000000..fcff87be4084 --- /dev/null +++ b/ksqldb-functional-tests/src/test/resources/historical_plans/collect-list_-_collect_list_date_map_table_-_PROTOBUF_NOSR/7.3.0_1651069593625/topology @@ -0,0 +1,43 @@ +Topologies: + Sub-topology: 0 + Source: KSTREAM-SOURCE-0000000001 (topics: [test_topic]) + --> KTABLE-SOURCE-0000000002 + Processor: KTABLE-SOURCE-0000000002 (stores: []) + --> KTABLE-TRANSFORMVALUES-0000000003 + <-- KSTREAM-SOURCE-0000000001 + Processor: KTABLE-TRANSFORMVALUES-0000000003 (stores: [KsqlTopic-Reduce]) + --> KTABLE-TRANSFORMVALUES-0000000004 + <-- KTABLE-SOURCE-0000000002 + Processor: KTABLE-TRANSFORMVALUES-0000000004 (stores: []) + --> Aggregate-Prepare + <-- KTABLE-TRANSFORMVALUES-0000000003 + Processor: Aggregate-Prepare (stores: []) + --> KTABLE-FILTER-0000000006 + <-- KTABLE-TRANSFORMVALUES-0000000004 + Processor: KTABLE-FILTER-0000000006 (stores: []) + --> Aggregate-GroupBy + <-- Aggregate-Prepare + Processor: Aggregate-GroupBy (stores: []) + --> KSTREAM-SINK-0000000008 + <-- KTABLE-FILTER-0000000006 + Sink: KSTREAM-SINK-0000000008 (topic: Aggregate-GroupBy-repartition) + <-- Aggregate-GroupBy + + Sub-topology: 1 + Source: KSTREAM-SOURCE-0000000009 (topics: [Aggregate-GroupBy-repartition]) + --> KTABLE-AGGREGATE-0000000010 + Processor: KTABLE-AGGREGATE-0000000010 (stores: [Aggregate-Aggregate-Materialize]) + --> Aggregate-Aggregate-ToOutputSchema + <-- KSTREAM-SOURCE-0000000009 + Processor: Aggregate-Aggregate-ToOutputSchema (stores: []) + --> Aggregate-Project + <-- KTABLE-AGGREGATE-0000000010 + Processor: Aggregate-Project (stores: []) + --> KTABLE-TOSTREAM-0000000013 + <-- Aggregate-Aggregate-ToOutputSchema + Processor: KTABLE-TOSTREAM-0000000013 (stores: []) + --> KSTREAM-SINK-0000000014 + <-- Aggregate-Project + Sink: KSTREAM-SINK-0000000014 (topic: S2) + <-- KTABLE-TOSTREAM-0000000013 + diff --git a/ksqldb-functional-tests/src/test/resources/historical_plans/collect-list_-_collect_list_double_table_-_PROTOBUF_NOSR/7.3.0_1651069569163/plan.json b/ksqldb-functional-tests/src/test/resources/historical_plans/collect-list_-_collect_list_double_table_-_PROTOBUF_NOSR/7.3.0_1651069569163/plan.json new file mode 100644 index 000000000000..c6c8044fc0bf --- /dev/null +++ b/ksqldb-functional-tests/src/test/resources/historical_plans/collect-list_-_collect_list_double_table_-_PROTOBUF_NOSR/7.3.0_1651069569163/plan.json @@ -0,0 +1,275 @@ +{ + "plan" : [ { + "@type" : "ksqlPlanV1", + "statementText" : "CREATE TABLE TEST (ID BIGINT PRIMARY KEY, VALUE DOUBLE) WITH (KAFKA_TOPIC='test_topic', KEY_FORMAT='KAFKA', VALUE_FORMAT='PROTOBUF_NOSR');", + "ddlCommand" : { + "@type" : "createTableV1", + "sourceName" : "TEST", + "schema" : "`ID` BIGINT KEY, `VALUE` DOUBLE", + "timestampColumn" : null, + "topicName" : "test_topic", + "formats" : { + "keyFormat" : { + "format" : "KAFKA", + "properties" : { } + }, + "valueFormat" : { + "format" : "PROTOBUF_NOSR", + "properties" : { } + } + }, + "windowInfo" : null, + "orReplace" : false, + "isSource" : false + }, + "queryPlan" : null + }, { + "@type" : "ksqlPlanV1", + "statementText" : "CREATE TABLE S2 AS SELECT\n TEST.ID ID,\n COLLECT_LIST(TEST.VALUE) COLLECTED\nFROM TEST TEST\nGROUP BY TEST.ID\nEMIT CHANGES", + "ddlCommand" : { + "@type" : "createTableV1", + "sourceName" : "S2", + "schema" : "`ID` BIGINT KEY, `COLLECTED` ARRAY", + "timestampColumn" : null, + "topicName" : "S2", + "formats" : { + "keyFormat" : { + "format" : "KAFKA", + "properties" : { } + }, + "valueFormat" : { + "format" : "PROTOBUF_NOSR", + "properties" : { } + } + }, + "windowInfo" : null, + "orReplace" : false, + "isSource" : false + }, + "queryPlan" : { + "sources" : [ "TEST" ], + "sink" : "S2", + "physicalPlan" : { + "@type" : "tableSinkV1", + "properties" : { + "queryContext" : "S2" + }, + "source" : { + "@type" : "tableSelectV1", + "properties" : { + "queryContext" : "Aggregate/Project" + }, + "source" : { + "@type" : "tableAggregateV1", + "properties" : { + "queryContext" : "Aggregate/Aggregate" + }, + "source" : { + "@type" : "tableGroupByV2", + "properties" : { + "queryContext" : "Aggregate/GroupBy" + }, + "source" : { + "@type" : "tableSelectV1", + "properties" : { + "queryContext" : "Aggregate/Prepare" + }, + "source" : { + "@type" : "tableSourceV2", + "properties" : { + "queryContext" : "KsqlTopic/Source" + }, + "topicName" : "test_topic", + "formats" : { + "keyFormat" : { + "format" : "KAFKA", + "properties" : { } + }, + "valueFormat" : { + "format" : "PROTOBUF_NOSR", + "properties" : { } + } + }, + "timestampColumn" : null, + "sourceSchema" : "`ID` BIGINT KEY, `VALUE` DOUBLE", + "pseudoColumnVersion" : 1, + "stateStoreFormats" : { + "keyFormat" : { + "format" : "KAFKA", + "properties" : { } + }, + "valueFormat" : { + "format" : "PROTOBUF_NOSR", + "properties" : { } + } + } + }, + "keyColumnNames" : [ "ID" ], + "selectExpressions" : [ "ID AS ID", "VALUE AS VALUE" ], + "internalFormats" : { + "keyFormat" : { + "format" : "KAFKA", + "properties" : { } + }, + "valueFormat" : { + "format" : "PROTOBUF_NOSR", + "properties" : { } + } + } + }, + "internalFormats" : { + "keyFormat" : { + "format" : "KAFKA", + "properties" : { } + }, + "valueFormat" : { + "format" : "PROTOBUF_NOSR", + "properties" : { } + } + }, + "groupByExpressions" : [ "ID" ] + }, + "internalFormats" : { + "keyFormat" : { + "format" : "KAFKA", + "properties" : { } + }, + "valueFormat" : { + "format" : "PROTOBUF_NOSR", + "properties" : { } + } + }, + "nonAggregateColumns" : [ "ID", "VALUE" ], + "aggregationFunctions" : [ "COLLECT_LIST(VALUE)" ] + }, + "keyColumnNames" : [ "ID" ], + "selectExpressions" : [ "KSQL_AGG_VARIABLE_0 AS COLLECTED" ], + "internalFormats" : { + "keyFormat" : { + "format" : "KAFKA", + "properties" : { } + }, + "valueFormat" : { + "format" : "PROTOBUF_NOSR", + "properties" : { } + } + } + }, + "formats" : { + "keyFormat" : { + "format" : "KAFKA", + "properties" : { } + }, + "valueFormat" : { + "format" : "PROTOBUF_NOSR", + "properties" : { } + } + }, + "topicName" : "S2", + "timestampColumn" : null + }, + "queryId" : "CTAS_S2_0", + "runtimeId" : null + } + } ], + "configs" : { + "ksql.extension.dir" : "ext", + "ksql.streams.cache.max.bytes.buffering" : "0", + "metric.reporters" : "", + "ksql.query.status.running.threshold.seconds" : "300", + "ksql.connect.basic.auth.credentials.reload" : "false", + "ksql.output.topic.name.prefix" : "", + "ksql.query.pull.stream.enabled" : "true", + "ksql.query.push.v2.interpreter.enabled" : "true", + "ksql.queryanonymizer.logs_enabled" : "true", + "ksql.variable.substitution.enable" : "true", + "ksql.streams.shutdown.timeout.ms" : "300000", + "ksql.query.pull.max.allowed.offset.lag" : "9223372036854775807", + "ksql.query.pull.max.qps" : "2147483647", + "ksql.access.validator.enable" : "auto", + "ksql.streams.bootstrap.servers" : "localhost:0", + "ksql.query.pull.metrics.enabled" : "true", + "ksql.metrics.extension" : null, + "ksql.query.push.v2.alos.enabled" : "true", + "ksql.query.push.v2.max.hourly.bandwidth.megabytes" : "2147483647", + "ksql.query.pull.range.scan.enabled" : "true", + "ksql.transient.query.cleanup.service.initial.delay.seconds" : "600", + "ksql.hidden.topics" : "_confluent.*,__confluent.*,_schemas,__consumer_offsets,__transaction_state,connect-configs,connect-offsets,connect-status,connect-statuses", + "ksql.lambdas.enabled" : "true", + "ksql.source.table.materialization.enabled" : "true", + "ksql.query.pull.max.hourly.bandwidth.megabytes" : "2147483647", + "ksql.sink.window.change.log.additional.retention" : "1000000", + "ksql.query.persistent.active.limit" : "2147483647", + "ksql.persistence.wrap.single.values" : null, + "ksql.query.transient.max.bytes.buffering.total" : "-1", + "ksql.connect.basic.auth.credentials.source" : "NONE", + "ksql.schema.registry.url" : "schema_registry.url:0", + "ksql.properties.overrides.denylist" : "", + "ksql.service.id" : "some.ksql.service.id", + "ksql.query.push.v2.max.catchup.consumers" : "5", + "ksql.query.push.v2.enabled" : "false", + "ksql.transient.query.cleanup.service.enable" : "true", + "ksql.query.push.v2.metrics.enabled" : "true", + "ksql.rowpartition.rowoffset.enabled" : "true", + "ksql.streams.commit.interval.ms" : "2000", + "ksql.query.pull.table.scan.enabled" : "true", + "ksql.streams.auto.commit.interval.ms" : "0", + "ksql.streams.topology.optimization" : "all", + "ksql.endpoint.migrate.query" : "true", + "ksql.query.push.v2.registry.installed" : "false", + "ksql.streams.num.stream.threads" : "4", + "ksql.metrics.tags.custom" : "", + "ksql.query.push.v2.catchup.consumer.msg.window" : "50", + "ksql.runtime.feature.shared.enabled" : "false", + "ksql.udf.collect.metrics" : "false", + "ksql.new.query.planner.enabled" : "false", + "ksql.connect.request.headers.plugin" : null, + "ksql.security.extension.class" : null, + "ksql.transient.prefix" : "transient_", + "ksql.headers.columns.enabled" : "true", + "ksql.streams.default.deserialization.exception.handler" : "io.confluent.ksql.errors.LogMetricAndContinueExceptionHandler", + "ksql.connect.request.timeout.ms" : "5000", + "ksql.query.pull.enable.standby.reads" : "false", + "ksql.persistence.default.format.key" : "KAFKA", + "ksql.query.persistent.max.bytes.buffering.total" : "-1", + "ksql.query.error.max.queue.size" : "10", + "ksql.query.cleanup.shutdown.timeout.ms" : "30000", + "ksql.internal.topic.min.insync.replicas" : "1", + "ksql.internal.topic.replicas" : "1", + "ksql.insert.into.values.enabled" : "true", + "ksql.queryanonymizer.cluster_namespace" : null, + "ksql.create.or.replace.enabled" : "true", + "ksql.shared.runtimes.count" : "2", + "ksql.cast.strings.preserve.nulls" : "true", + "ksql.authorization.cache.max.entries" : "10000", + "ksql.pull.queries.enable" : "true", + "ksql.transient.query.cleanup.service.period.seconds" : "600", + "ksql.suppress.enabled" : "false", + "ksql.readonly.topics" : "_confluent.*,__confluent.*,_schemas,__consumer_offsets,__transaction_state,connect-configs,connect-offsets,connect-status,connect-statuses", + "ksql.connect.basic.auth.credentials.file" : "", + "ksql.authorization.cache.expiry.time.secs" : "30", + "ksql.query.retry.backoff.initial.ms" : "15000", + "ksql.query.pull.max.concurrent.requests" : "2147483647", + "ksql.streams.auto.offset.reset" : "earliest", + "ksql.connect.url" : "http://localhost:8083", + "ksql.query.push.v2.new.latest.delay.ms" : "5000", + "ksql.query.push.v2.latest.reset.age.ms" : "30000", + "ksql.streams.default.production.exception.handler" : "io.confluent.ksql.errors.ProductionExceptionHandlerUtil$LogAndFailProductionExceptionHandler", + "ksql.query.pull.interpreter.enabled" : "true", + "ksql.query.pull.limit.clause.enabled" : "true", + "ksql.query.pull.router.thread.pool.size" : "50", + "ksql.query.push.v2.continuation.tokens.enabled" : "false", + "ksql.query.retry.backoff.max.ms" : "900000", + "ksql.timestamp.throw.on.invalid" : "false", + "ksql.persistence.default.format.value" : null, + "ksql.udfs.enabled" : "true", + "ksql.udf.enable.security.manager" : "true", + "ksql.connect.worker.config" : "", + "ksql.nested.error.set.null" : "true", + "ksql.query.pull.thread.pool.size" : "50", + "ksql.persistent.prefix" : "query_", + "ksql.metastore.backup.location" : "", + "ksql.error.classifier.regex" : "", + "ksql.suppress.buffer.size.bytes" : "-1" + } +} \ No newline at end of file diff --git a/ksqldb-functional-tests/src/test/resources/historical_plans/collect-list_-_collect_list_double_table_-_PROTOBUF_NOSR/7.3.0_1651069569163/spec.json b/ksqldb-functional-tests/src/test/resources/historical_plans/collect-list_-_collect_list_double_table_-_PROTOBUF_NOSR/7.3.0_1651069569163/spec.json new file mode 100644 index 000000000000..0521d05f86d0 --- /dev/null +++ b/ksqldb-functional-tests/src/test/resources/historical_plans/collect-list_-_collect_list_double_table_-_PROTOBUF_NOSR/7.3.0_1651069569163/spec.json @@ -0,0 +1,215 @@ +{ + "version" : "7.3.0", + "timestamp" : 1651069569163, + "path" : "query-validation-tests/collect-list.json", + "schemas" : { + "CTAS_S2_0.Aggregate.Aggregate.Materialize" : { + "schema" : "`ID` BIGINT KEY, `ID` BIGINT, `VALUE` DOUBLE, `KSQL_AGG_VARIABLE_0` ARRAY", + "keyFormat" : { + "format" : "KAFKA" + }, + "valueFormat" : { + "format" : "PROTOBUF_NOSR" + } + }, + "CTAS_S2_0.KsqlTopic.Source.Materialized" : { + "schema" : "`ID` BIGINT KEY, `VALUE` DOUBLE, `ROWPARTITION` INTEGER, `ROWOFFSET` BIGINT", + "keyFormat" : { + "format" : "KAFKA" + }, + "valueFormat" : { + "format" : "PROTOBUF_NOSR" + } + }, + "CTAS_S2_0.Aggregate.Project" : { + "schema" : "`ID` BIGINT KEY, `COLLECTED` ARRAY", + "keyFormat" : { + "format" : "KAFKA" + }, + "valueFormat" : { + "format" : "PROTOBUF_NOSR" + } + }, + "CTAS_S2_0.Aggregate.Prepare" : { + "schema" : "`ID` BIGINT KEY, `ID` BIGINT, `VALUE` DOUBLE", + "keyFormat" : { + "format" : "KAFKA" + }, + "valueFormat" : { + "format" : "PROTOBUF_NOSR" + } + }, + "CTAS_S2_0.KsqlTopic.Source" : { + "schema" : "`ID` BIGINT KEY, `VALUE` DOUBLE", + "keyFormat" : { + "format" : "KAFKA" + }, + "valueFormat" : { + "format" : "PROTOBUF_NOSR" + } + }, + "CTAS_S2_0.S2" : { + "schema" : "`ID` BIGINT KEY, `COLLECTED` ARRAY", + "keyFormat" : { + "format" : "KAFKA" + }, + "valueFormat" : { + "format" : "PROTOBUF_NOSR" + } + }, + "CTAS_S2_0.Aggregate.GroupBy" : { + "schema" : "`ID` BIGINT KEY, `ID` BIGINT, `VALUE` DOUBLE", + "keyFormat" : { + "format" : "KAFKA" + }, + "valueFormat" : { + "format" : "PROTOBUF_NOSR" + } + } + }, + "testCase" : { + "name" : "collect_list double table - PROTOBUF_NOSR", + "inputs" : [ { + "topic" : "test_topic", + "key" : 0, + "value" : { + "VALUE" : 5.4 + } + }, { + "topic" : "test_topic", + "key" : 0, + "value" : { + "VALUE" : 100.1 + } + }, { + "topic" : "test_topic", + "key" : 100, + "value" : { + "VALUE" : 500.9 + } + }, { + "topic" : "test_topic", + "key" : 100, + "value" : { + "VALUE" : 300.8 + } + } ], + "outputs" : [ { + "topic" : "S2", + "key" : 0, + "value" : { + "COLLECTED" : [ 5.4 ] + } + }, { + "topic" : "S2", + "key" : 0, + "value" : { + "COLLECTED" : [ ] + } + }, { + "topic" : "S2", + "key" : 0, + "value" : { + "COLLECTED" : [ 100.1 ] + } + }, { + "topic" : "S2", + "key" : 100, + "value" : { + "COLLECTED" : [ 500.9 ] + } + }, { + "topic" : "S2", + "key" : 100, + "value" : { + "COLLECTED" : [ ] + } + }, { + "topic" : "S2", + "key" : 100, + "value" : { + "COLLECTED" : [ 300.8 ] + } + } ], + "topics" : [ { + "name" : "test_topic", + "replicas" : 1, + "numPartitions" : 4 + }, { + "name" : "S2", + "replicas" : 1, + "numPartitions" : 4 + } ], + "statements" : [ "CREATE TABLE TEST (ID BIGINT PRIMARY KEY, VALUE double) WITH (kafka_topic='test_topic', value_format='PROTOBUF_NOSR');", "CREATE TABLE S2 as SELECT ID, collect_list(value) as collected FROM test group by id;" ], + "post" : { + "sources" : [ { + "name" : "S2", + "type" : "TABLE", + "schema" : "`ID` BIGINT KEY, `COLLECTED` ARRAY", + "keyFormat" : { + "format" : "KAFKA" + }, + "valueFormat" : "PROTOBUF_NOSR", + "keyFeatures" : [ ], + "valueFeatures" : [ ], + "isSource" : false + }, { + "name" : "TEST", + "type" : "TABLE", + "schema" : "`ID` BIGINT KEY, `VALUE` DOUBLE", + "keyFormat" : { + "format" : "KAFKA" + }, + "valueFormat" : "PROTOBUF_NOSR", + "keyFeatures" : [ ], + "valueFeatures" : [ ], + "isSource" : false + } ], + "topics" : { + "topics" : [ { + "name" : "_confluent-ksql-some.ksql.service.idquery_CTAS_S2_0-Aggregate-Aggregate-Materialize-changelog", + "keyFormat" : { + "format" : "KAFKA" + }, + "valueFormat" : { + "format" : "PROTOBUF_NOSR" + } + }, { + "name" : "_confluent-ksql-some.ksql.service.idquery_CTAS_S2_0-KsqlTopic-Reduce-changelog", + "keyFormat" : { + "format" : "KAFKA" + }, + "valueFormat" : { + "format" : "PROTOBUF_NOSR" + } + }, { + "name" : "test_topic", + "keyFormat" : { + "format" : "KAFKA" + }, + "valueFormat" : { + "format" : "PROTOBUF_NOSR" + }, + "partitions" : 4 + }, { + "name" : "_confluent-ksql-some.ksql.service.idquery_CTAS_S2_0-Aggregate-GroupBy-repartition", + "keyFormat" : { + "format" : "KAFKA" + }, + "valueFormat" : { + "format" : "PROTOBUF_NOSR" + } + }, { + "name" : "S2", + "keyFormat" : { + "format" : "KAFKA" + }, + "valueFormat" : { + "format" : "PROTOBUF_NOSR" + }, + "partitions" : 4 + } ] + } + } + } +} \ No newline at end of file diff --git a/ksqldb-functional-tests/src/test/resources/historical_plans/collect-list_-_collect_list_double_table_-_PROTOBUF_NOSR/7.3.0_1651069569163/topology b/ksqldb-functional-tests/src/test/resources/historical_plans/collect-list_-_collect_list_double_table_-_PROTOBUF_NOSR/7.3.0_1651069569163/topology new file mode 100644 index 000000000000..fcff87be4084 --- /dev/null +++ b/ksqldb-functional-tests/src/test/resources/historical_plans/collect-list_-_collect_list_double_table_-_PROTOBUF_NOSR/7.3.0_1651069569163/topology @@ -0,0 +1,43 @@ +Topologies: + Sub-topology: 0 + Source: KSTREAM-SOURCE-0000000001 (topics: [test_topic]) + --> KTABLE-SOURCE-0000000002 + Processor: KTABLE-SOURCE-0000000002 (stores: []) + --> KTABLE-TRANSFORMVALUES-0000000003 + <-- KSTREAM-SOURCE-0000000001 + Processor: KTABLE-TRANSFORMVALUES-0000000003 (stores: [KsqlTopic-Reduce]) + --> KTABLE-TRANSFORMVALUES-0000000004 + <-- KTABLE-SOURCE-0000000002 + Processor: KTABLE-TRANSFORMVALUES-0000000004 (stores: []) + --> Aggregate-Prepare + <-- KTABLE-TRANSFORMVALUES-0000000003 + Processor: Aggregate-Prepare (stores: []) + --> KTABLE-FILTER-0000000006 + <-- KTABLE-TRANSFORMVALUES-0000000004 + Processor: KTABLE-FILTER-0000000006 (stores: []) + --> Aggregate-GroupBy + <-- Aggregate-Prepare + Processor: Aggregate-GroupBy (stores: []) + --> KSTREAM-SINK-0000000008 + <-- KTABLE-FILTER-0000000006 + Sink: KSTREAM-SINK-0000000008 (topic: Aggregate-GroupBy-repartition) + <-- Aggregate-GroupBy + + Sub-topology: 1 + Source: KSTREAM-SOURCE-0000000009 (topics: [Aggregate-GroupBy-repartition]) + --> KTABLE-AGGREGATE-0000000010 + Processor: KTABLE-AGGREGATE-0000000010 (stores: [Aggregate-Aggregate-Materialize]) + --> Aggregate-Aggregate-ToOutputSchema + <-- KSTREAM-SOURCE-0000000009 + Processor: Aggregate-Aggregate-ToOutputSchema (stores: []) + --> Aggregate-Project + <-- KTABLE-AGGREGATE-0000000010 + Processor: Aggregate-Project (stores: []) + --> KTABLE-TOSTREAM-0000000013 + <-- Aggregate-Aggregate-ToOutputSchema + Processor: KTABLE-TOSTREAM-0000000013 (stores: []) + --> KSTREAM-SINK-0000000014 + <-- Aggregate-Project + Sink: KSTREAM-SINK-0000000014 (topic: S2) + <-- KTABLE-TOSTREAM-0000000013 + diff --git a/ksqldb-functional-tests/src/test/resources/historical_plans/collect-list_-_collect_list_int_-_PROTOBUF_NOSR/7.3.0_1651069534710/plan.json b/ksqldb-functional-tests/src/test/resources/historical_plans/collect-list_-_collect_list_int_-_PROTOBUF_NOSR/7.3.0_1651069534710/plan.json new file mode 100644 index 000000000000..12526cc42cbd --- /dev/null +++ b/ksqldb-functional-tests/src/test/resources/historical_plans/collect-list_-_collect_list_int_-_PROTOBUF_NOSR/7.3.0_1651069534710/plan.json @@ -0,0 +1,255 @@ +{ + "plan" : [ { + "@type" : "ksqlPlanV1", + "statementText" : "CREATE STREAM TEST (ID BIGINT KEY, VALUE INTEGER) WITH (KAFKA_TOPIC='test_topic', KEY_FORMAT='KAFKA', VALUE_FORMAT='PROTOBUF_NOSR');", + "ddlCommand" : { + "@type" : "createStreamV1", + "sourceName" : "TEST", + "schema" : "`ID` BIGINT KEY, `VALUE` INTEGER", + "timestampColumn" : null, + "topicName" : "test_topic", + "formats" : { + "keyFormat" : { + "format" : "KAFKA", + "properties" : { } + }, + "valueFormat" : { + "format" : "PROTOBUF_NOSR", + "properties" : { } + } + }, + "windowInfo" : null, + "orReplace" : false, + "isSource" : false + }, + "queryPlan" : null + }, { + "@type" : "ksqlPlanV1", + "statementText" : "CREATE TABLE S2 AS SELECT\n TEST.ID ID,\n COLLECT_LIST(TEST.VALUE) COLLECTED\nFROM TEST TEST\nGROUP BY TEST.ID\nEMIT CHANGES", + "ddlCommand" : { + "@type" : "createTableV1", + "sourceName" : "S2", + "schema" : "`ID` BIGINT KEY, `COLLECTED` ARRAY", + "timestampColumn" : null, + "topicName" : "S2", + "formats" : { + "keyFormat" : { + "format" : "KAFKA", + "properties" : { } + }, + "valueFormat" : { + "format" : "PROTOBUF_NOSR", + "properties" : { } + } + }, + "windowInfo" : null, + "orReplace" : false, + "isSource" : false + }, + "queryPlan" : { + "sources" : [ "TEST" ], + "sink" : "S2", + "physicalPlan" : { + "@type" : "tableSinkV1", + "properties" : { + "queryContext" : "S2" + }, + "source" : { + "@type" : "tableSelectV1", + "properties" : { + "queryContext" : "Aggregate/Project" + }, + "source" : { + "@type" : "streamAggregateV1", + "properties" : { + "queryContext" : "Aggregate/Aggregate" + }, + "source" : { + "@type" : "streamGroupByKeyV1", + "properties" : { + "queryContext" : "Aggregate/GroupBy" + }, + "source" : { + "@type" : "streamSelectV1", + "properties" : { + "queryContext" : "Aggregate/Prepare" + }, + "source" : { + "@type" : "streamSourceV1", + "properties" : { + "queryContext" : "KsqlTopic/Source" + }, + "topicName" : "test_topic", + "formats" : { + "keyFormat" : { + "format" : "KAFKA", + "properties" : { } + }, + "valueFormat" : { + "format" : "PROTOBUF_NOSR", + "properties" : { } + } + }, + "timestampColumn" : null, + "sourceSchema" : "`ID` BIGINT KEY, `VALUE` INTEGER", + "pseudoColumnVersion" : 1 + }, + "keyColumnNames" : [ "ID" ], + "selectedKeys" : null, + "selectExpressions" : [ "ID AS ID", "VALUE AS VALUE" ] + }, + "internalFormats" : { + "keyFormat" : { + "format" : "KAFKA", + "properties" : { } + }, + "valueFormat" : { + "format" : "PROTOBUF_NOSR", + "properties" : { } + } + } + }, + "internalFormats" : { + "keyFormat" : { + "format" : "KAFKA", + "properties" : { } + }, + "valueFormat" : { + "format" : "PROTOBUF_NOSR", + "properties" : { } + } + }, + "nonAggregateColumns" : [ "ID", "VALUE" ], + "aggregationFunctions" : [ "COLLECT_LIST(VALUE)" ] + }, + "keyColumnNames" : [ "ID" ], + "selectExpressions" : [ "KSQL_AGG_VARIABLE_0 AS COLLECTED" ], + "internalFormats" : { + "keyFormat" : { + "format" : "KAFKA", + "properties" : { } + }, + "valueFormat" : { + "format" : "PROTOBUF_NOSR", + "properties" : { } + } + } + }, + "formats" : { + "keyFormat" : { + "format" : "KAFKA", + "properties" : { } + }, + "valueFormat" : { + "format" : "PROTOBUF_NOSR", + "properties" : { } + } + }, + "topicName" : "S2", + "timestampColumn" : null + }, + "queryId" : "CTAS_S2_0", + "runtimeId" : null + } + } ], + "configs" : { + "ksql.extension.dir" : "ext", + "ksql.streams.cache.max.bytes.buffering" : "0", + "metric.reporters" : "", + "ksql.query.status.running.threshold.seconds" : "300", + "ksql.connect.basic.auth.credentials.reload" : "false", + "ksql.output.topic.name.prefix" : "", + "ksql.query.pull.stream.enabled" : "true", + "ksql.query.push.v2.interpreter.enabled" : "true", + "ksql.queryanonymizer.logs_enabled" : "true", + "ksql.variable.substitution.enable" : "true", + "ksql.streams.shutdown.timeout.ms" : "300000", + "ksql.query.pull.max.allowed.offset.lag" : "9223372036854775807", + "ksql.query.pull.max.qps" : "2147483647", + "ksql.access.validator.enable" : "auto", + "ksql.streams.bootstrap.servers" : "localhost:0", + "ksql.query.pull.metrics.enabled" : "true", + "ksql.metrics.extension" : null, + "ksql.query.push.v2.alos.enabled" : "true", + "ksql.query.push.v2.max.hourly.bandwidth.megabytes" : "2147483647", + "ksql.query.pull.range.scan.enabled" : "true", + "ksql.transient.query.cleanup.service.initial.delay.seconds" : "600", + "ksql.hidden.topics" : "_confluent.*,__confluent.*,_schemas,__consumer_offsets,__transaction_state,connect-configs,connect-offsets,connect-status,connect-statuses", + "ksql.lambdas.enabled" : "true", + "ksql.source.table.materialization.enabled" : "true", + "ksql.query.pull.max.hourly.bandwidth.megabytes" : "2147483647", + "ksql.sink.window.change.log.additional.retention" : "1000000", + "ksql.query.persistent.active.limit" : "2147483647", + "ksql.persistence.wrap.single.values" : null, + "ksql.query.transient.max.bytes.buffering.total" : "-1", + "ksql.connect.basic.auth.credentials.source" : "NONE", + "ksql.schema.registry.url" : "schema_registry.url:0", + "ksql.properties.overrides.denylist" : "", + "ksql.service.id" : "some.ksql.service.id", + "ksql.query.push.v2.max.catchup.consumers" : "5", + "ksql.query.push.v2.enabled" : "false", + "ksql.transient.query.cleanup.service.enable" : "true", + "ksql.query.push.v2.metrics.enabled" : "true", + "ksql.rowpartition.rowoffset.enabled" : "true", + "ksql.streams.commit.interval.ms" : "2000", + "ksql.query.pull.table.scan.enabled" : "true", + "ksql.streams.auto.commit.interval.ms" : "0", + "ksql.streams.topology.optimization" : "all", + "ksql.endpoint.migrate.query" : "true", + "ksql.query.push.v2.registry.installed" : "false", + "ksql.streams.num.stream.threads" : "4", + "ksql.metrics.tags.custom" : "", + "ksql.query.push.v2.catchup.consumer.msg.window" : "50", + "ksql.runtime.feature.shared.enabled" : "false", + "ksql.udf.collect.metrics" : "false", + "ksql.new.query.planner.enabled" : "false", + "ksql.connect.request.headers.plugin" : null, + "ksql.security.extension.class" : null, + "ksql.transient.prefix" : "transient_", + "ksql.headers.columns.enabled" : "true", + "ksql.streams.default.deserialization.exception.handler" : "io.confluent.ksql.errors.LogMetricAndContinueExceptionHandler", + "ksql.connect.request.timeout.ms" : "5000", + "ksql.query.pull.enable.standby.reads" : "false", + "ksql.persistence.default.format.key" : "KAFKA", + "ksql.query.persistent.max.bytes.buffering.total" : "-1", + "ksql.query.error.max.queue.size" : "10", + "ksql.query.cleanup.shutdown.timeout.ms" : "30000", + "ksql.internal.topic.min.insync.replicas" : "1", + "ksql.internal.topic.replicas" : "1", + "ksql.insert.into.values.enabled" : "true", + "ksql.queryanonymizer.cluster_namespace" : null, + "ksql.create.or.replace.enabled" : "true", + "ksql.shared.runtimes.count" : "2", + "ksql.cast.strings.preserve.nulls" : "true", + "ksql.authorization.cache.max.entries" : "10000", + "ksql.pull.queries.enable" : "true", + "ksql.transient.query.cleanup.service.period.seconds" : "600", + "ksql.suppress.enabled" : "false", + "ksql.readonly.topics" : "_confluent.*,__confluent.*,_schemas,__consumer_offsets,__transaction_state,connect-configs,connect-offsets,connect-status,connect-statuses", + "ksql.connect.basic.auth.credentials.file" : "", + "ksql.authorization.cache.expiry.time.secs" : "30", + "ksql.query.retry.backoff.initial.ms" : "15000", + "ksql.query.pull.max.concurrent.requests" : "2147483647", + "ksql.streams.auto.offset.reset" : "earliest", + "ksql.connect.url" : "http://localhost:8083", + "ksql.query.push.v2.new.latest.delay.ms" : "5000", + "ksql.query.push.v2.latest.reset.age.ms" : "30000", + "ksql.streams.default.production.exception.handler" : "io.confluent.ksql.errors.ProductionExceptionHandlerUtil$LogAndFailProductionExceptionHandler", + "ksql.query.pull.interpreter.enabled" : "true", + "ksql.query.pull.limit.clause.enabled" : "true", + "ksql.query.pull.router.thread.pool.size" : "50", + "ksql.query.push.v2.continuation.tokens.enabled" : "false", + "ksql.query.retry.backoff.max.ms" : "900000", + "ksql.timestamp.throw.on.invalid" : "false", + "ksql.persistence.default.format.value" : null, + "ksql.udfs.enabled" : "true", + "ksql.udf.enable.security.manager" : "true", + "ksql.connect.worker.config" : "", + "ksql.nested.error.set.null" : "true", + "ksql.query.pull.thread.pool.size" : "50", + "ksql.persistent.prefix" : "query_", + "ksql.metastore.backup.location" : "", + "ksql.error.classifier.regex" : "", + "ksql.suppress.buffer.size.bytes" : "-1" + } +} \ No newline at end of file diff --git a/ksqldb-functional-tests/src/test/resources/historical_plans/collect-list_-_collect_list_int_-_PROTOBUF_NOSR/7.3.0_1651069534710/spec.json b/ksqldb-functional-tests/src/test/resources/historical_plans/collect-list_-_collect_list_int_-_PROTOBUF_NOSR/7.3.0_1651069534710/spec.json new file mode 100644 index 000000000000..503247099dfc --- /dev/null +++ b/ksqldb-functional-tests/src/test/resources/historical_plans/collect-list_-_collect_list_int_-_PROTOBUF_NOSR/7.3.0_1651069534710/spec.json @@ -0,0 +1,169 @@ +{ + "version" : "7.3.0", + "timestamp" : 1651069534710, + "path" : "query-validation-tests/collect-list.json", + "schemas" : { + "CTAS_S2_0.Aggregate.Aggregate.Materialize" : { + "schema" : "`ID` BIGINT KEY, `ID` BIGINT, `VALUE` INTEGER, `KSQL_AGG_VARIABLE_0` ARRAY", + "keyFormat" : { + "format" : "KAFKA" + }, + "valueFormat" : { + "format" : "PROTOBUF_NOSR" + } + }, + "CTAS_S2_0.Aggregate.Project" : { + "schema" : "`ID` BIGINT KEY, `COLLECTED` ARRAY", + "keyFormat" : { + "format" : "KAFKA" + }, + "valueFormat" : { + "format" : "PROTOBUF_NOSR" + } + }, + "CTAS_S2_0.KsqlTopic.Source" : { + "schema" : "`ID` BIGINT KEY, `VALUE` INTEGER", + "keyFormat" : { + "format" : "KAFKA" + }, + "valueFormat" : { + "format" : "PROTOBUF_NOSR" + } + }, + "CTAS_S2_0.S2" : { + "schema" : "`ID` BIGINT KEY, `COLLECTED` ARRAY", + "keyFormat" : { + "format" : "KAFKA" + }, + "valueFormat" : { + "format" : "PROTOBUF_NOSR" + } + }, + "CTAS_S2_0.Aggregate.GroupBy" : { + "schema" : "`ID` BIGINT KEY, `ID` BIGINT, `VALUE` INTEGER", + "keyFormat" : { + "format" : "KAFKA" + }, + "valueFormat" : { + "format" : "PROTOBUF_NOSR" + } + } + }, + "testCase" : { + "name" : "collect_list int - PROTOBUF_NOSR", + "inputs" : [ { + "topic" : "test_topic", + "key" : 0, + "value" : { + "VALUE" : 0 + } + }, { + "topic" : "test_topic", + "key" : 0, + "value" : { + "VALUE" : 100 + } + }, { + "topic" : "test_topic", + "key" : 100, + "value" : { + "VALUE" : 500 + } + }, { + "topic" : "test_topic", + "key" : 100, + "value" : { + "VALUE" : 100 + } + } ], + "outputs" : [ { + "topic" : "S2", + "key" : 0, + "value" : { + "COLLECTED" : [ 0 ] + } + }, { + "topic" : "S2", + "key" : 0, + "value" : { + "COLLECTED" : [ 0, 100 ] + } + }, { + "topic" : "S2", + "key" : 100, + "value" : { + "COLLECTED" : [ 500 ] + } + }, { + "topic" : "S2", + "key" : 100, + "value" : { + "COLLECTED" : [ 500, 100 ] + } + } ], + "topics" : [ { + "name" : "test_topic", + "replicas" : 1, + "numPartitions" : 4 + }, { + "name" : "S2", + "replicas" : 1, + "numPartitions" : 4 + } ], + "statements" : [ "CREATE STREAM TEST (ID BIGINT KEY, VALUE integer) WITH (kafka_topic='test_topic',value_format='PROTOBUF_NOSR');", "CREATE TABLE S2 as SELECT ID, collect_list(value) as collected FROM test group by id;" ], + "post" : { + "sources" : [ { + "name" : "S2", + "type" : "TABLE", + "schema" : "`ID` BIGINT KEY, `COLLECTED` ARRAY", + "keyFormat" : { + "format" : "KAFKA" + }, + "valueFormat" : "PROTOBUF_NOSR", + "keyFeatures" : [ ], + "valueFeatures" : [ ], + "isSource" : false + }, { + "name" : "TEST", + "type" : "STREAM", + "schema" : "`ID` BIGINT KEY, `VALUE` INTEGER", + "keyFormat" : { + "format" : "KAFKA" + }, + "valueFormat" : "PROTOBUF_NOSR", + "keyFeatures" : [ ], + "valueFeatures" : [ ], + "isSource" : false + } ], + "topics" : { + "topics" : [ { + "name" : "_confluent-ksql-some.ksql.service.idquery_CTAS_S2_0-Aggregate-Aggregate-Materialize-changelog", + "keyFormat" : { + "format" : "KAFKA" + }, + "valueFormat" : { + "format" : "PROTOBUF_NOSR" + } + }, { + "name" : "test_topic", + "keyFormat" : { + "format" : "KAFKA" + }, + "valueFormat" : { + "format" : "PROTOBUF_NOSR" + }, + "partitions" : 4 + }, { + "name" : "S2", + "keyFormat" : { + "format" : "KAFKA" + }, + "valueFormat" : { + "format" : "PROTOBUF_NOSR" + }, + "partitions" : 4 + } ] + } + } + } +} \ No newline at end of file diff --git a/ksqldb-functional-tests/src/test/resources/historical_plans/collect-list_-_collect_list_int_-_PROTOBUF_NOSR/7.3.0_1651069534710/topology b/ksqldb-functional-tests/src/test/resources/historical_plans/collect-list_-_collect_list_int_-_PROTOBUF_NOSR/7.3.0_1651069534710/topology new file mode 100644 index 000000000000..fcfc8fb28665 --- /dev/null +++ b/ksqldb-functional-tests/src/test/resources/historical_plans/collect-list_-_collect_list_int_-_PROTOBUF_NOSR/7.3.0_1651069534710/topology @@ -0,0 +1,25 @@ +Topologies: + Sub-topology: 0 + Source: KSTREAM-SOURCE-0000000000 (topics: [test_topic]) + --> KSTREAM-TRANSFORMVALUES-0000000001 + Processor: KSTREAM-TRANSFORMVALUES-0000000001 (stores: []) + --> Aggregate-Prepare + <-- KSTREAM-SOURCE-0000000000 + Processor: Aggregate-Prepare (stores: []) + --> KSTREAM-AGGREGATE-0000000003 + <-- KSTREAM-TRANSFORMVALUES-0000000001 + Processor: KSTREAM-AGGREGATE-0000000003 (stores: [Aggregate-Aggregate-Materialize]) + --> Aggregate-Aggregate-ToOutputSchema + <-- Aggregate-Prepare + Processor: Aggregate-Aggregate-ToOutputSchema (stores: []) + --> Aggregate-Project + <-- KSTREAM-AGGREGATE-0000000003 + Processor: Aggregate-Project (stores: []) + --> KTABLE-TOSTREAM-0000000006 + <-- Aggregate-Aggregate-ToOutputSchema + Processor: KTABLE-TOSTREAM-0000000006 (stores: []) + --> KSTREAM-SINK-0000000007 + <-- Aggregate-Project + Sink: KSTREAM-SINK-0000000007 (topic: S2) + <-- KTABLE-TOSTREAM-0000000006 + diff --git a/ksqldb-functional-tests/src/test/resources/historical_plans/collect-list_-_collect_list_int_table_-_PROTOBUF_NOSR/7.3.0_1651069556550/plan.json b/ksqldb-functional-tests/src/test/resources/historical_plans/collect-list_-_collect_list_int_table_-_PROTOBUF_NOSR/7.3.0_1651069556550/plan.json new file mode 100644 index 000000000000..0a27b74f48f3 --- /dev/null +++ b/ksqldb-functional-tests/src/test/resources/historical_plans/collect-list_-_collect_list_int_table_-_PROTOBUF_NOSR/7.3.0_1651069556550/plan.json @@ -0,0 +1,275 @@ +{ + "plan" : [ { + "@type" : "ksqlPlanV1", + "statementText" : "CREATE TABLE TEST (ID BIGINT PRIMARY KEY, VALUE INTEGER) WITH (KAFKA_TOPIC='test_topic', KEY_FORMAT='KAFKA', VALUE_FORMAT='PROTOBUF_NOSR');", + "ddlCommand" : { + "@type" : "createTableV1", + "sourceName" : "TEST", + "schema" : "`ID` BIGINT KEY, `VALUE` INTEGER", + "timestampColumn" : null, + "topicName" : "test_topic", + "formats" : { + "keyFormat" : { + "format" : "KAFKA", + "properties" : { } + }, + "valueFormat" : { + "format" : "PROTOBUF_NOSR", + "properties" : { } + } + }, + "windowInfo" : null, + "orReplace" : false, + "isSource" : false + }, + "queryPlan" : null + }, { + "@type" : "ksqlPlanV1", + "statementText" : "CREATE TABLE S2 AS SELECT\n TEST.ID ID,\n COLLECT_LIST(TEST.VALUE) COLLECTED\nFROM TEST TEST\nGROUP BY TEST.ID\nEMIT CHANGES", + "ddlCommand" : { + "@type" : "createTableV1", + "sourceName" : "S2", + "schema" : "`ID` BIGINT KEY, `COLLECTED` ARRAY", + "timestampColumn" : null, + "topicName" : "S2", + "formats" : { + "keyFormat" : { + "format" : "KAFKA", + "properties" : { } + }, + "valueFormat" : { + "format" : "PROTOBUF_NOSR", + "properties" : { } + } + }, + "windowInfo" : null, + "orReplace" : false, + "isSource" : false + }, + "queryPlan" : { + "sources" : [ "TEST" ], + "sink" : "S2", + "physicalPlan" : { + "@type" : "tableSinkV1", + "properties" : { + "queryContext" : "S2" + }, + "source" : { + "@type" : "tableSelectV1", + "properties" : { + "queryContext" : "Aggregate/Project" + }, + "source" : { + "@type" : "tableAggregateV1", + "properties" : { + "queryContext" : "Aggregate/Aggregate" + }, + "source" : { + "@type" : "tableGroupByV2", + "properties" : { + "queryContext" : "Aggregate/GroupBy" + }, + "source" : { + "@type" : "tableSelectV1", + "properties" : { + "queryContext" : "Aggregate/Prepare" + }, + "source" : { + "@type" : "tableSourceV2", + "properties" : { + "queryContext" : "KsqlTopic/Source" + }, + "topicName" : "test_topic", + "formats" : { + "keyFormat" : { + "format" : "KAFKA", + "properties" : { } + }, + "valueFormat" : { + "format" : "PROTOBUF_NOSR", + "properties" : { } + } + }, + "timestampColumn" : null, + "sourceSchema" : "`ID` BIGINT KEY, `VALUE` INTEGER", + "pseudoColumnVersion" : 1, + "stateStoreFormats" : { + "keyFormat" : { + "format" : "KAFKA", + "properties" : { } + }, + "valueFormat" : { + "format" : "PROTOBUF_NOSR", + "properties" : { } + } + } + }, + "keyColumnNames" : [ "ID" ], + "selectExpressions" : [ "ID AS ID", "VALUE AS VALUE" ], + "internalFormats" : { + "keyFormat" : { + "format" : "KAFKA", + "properties" : { } + }, + "valueFormat" : { + "format" : "PROTOBUF_NOSR", + "properties" : { } + } + } + }, + "internalFormats" : { + "keyFormat" : { + "format" : "KAFKA", + "properties" : { } + }, + "valueFormat" : { + "format" : "PROTOBUF_NOSR", + "properties" : { } + } + }, + "groupByExpressions" : [ "ID" ] + }, + "internalFormats" : { + "keyFormat" : { + "format" : "KAFKA", + "properties" : { } + }, + "valueFormat" : { + "format" : "PROTOBUF_NOSR", + "properties" : { } + } + }, + "nonAggregateColumns" : [ "ID", "VALUE" ], + "aggregationFunctions" : [ "COLLECT_LIST(VALUE)" ] + }, + "keyColumnNames" : [ "ID" ], + "selectExpressions" : [ "KSQL_AGG_VARIABLE_0 AS COLLECTED" ], + "internalFormats" : { + "keyFormat" : { + "format" : "KAFKA", + "properties" : { } + }, + "valueFormat" : { + "format" : "PROTOBUF_NOSR", + "properties" : { } + } + } + }, + "formats" : { + "keyFormat" : { + "format" : "KAFKA", + "properties" : { } + }, + "valueFormat" : { + "format" : "PROTOBUF_NOSR", + "properties" : { } + } + }, + "topicName" : "S2", + "timestampColumn" : null + }, + "queryId" : "CTAS_S2_0", + "runtimeId" : null + } + } ], + "configs" : { + "ksql.extension.dir" : "ext", + "ksql.streams.cache.max.bytes.buffering" : "0", + "metric.reporters" : "", + "ksql.query.status.running.threshold.seconds" : "300", + "ksql.connect.basic.auth.credentials.reload" : "false", + "ksql.output.topic.name.prefix" : "", + "ksql.query.pull.stream.enabled" : "true", + "ksql.query.push.v2.interpreter.enabled" : "true", + "ksql.queryanonymizer.logs_enabled" : "true", + "ksql.variable.substitution.enable" : "true", + "ksql.streams.shutdown.timeout.ms" : "300000", + "ksql.query.pull.max.allowed.offset.lag" : "9223372036854775807", + "ksql.query.pull.max.qps" : "2147483647", + "ksql.access.validator.enable" : "auto", + "ksql.streams.bootstrap.servers" : "localhost:0", + "ksql.query.pull.metrics.enabled" : "true", + "ksql.metrics.extension" : null, + "ksql.query.push.v2.alos.enabled" : "true", + "ksql.query.push.v2.max.hourly.bandwidth.megabytes" : "2147483647", + "ksql.query.pull.range.scan.enabled" : "true", + "ksql.transient.query.cleanup.service.initial.delay.seconds" : "600", + "ksql.hidden.topics" : "_confluent.*,__confluent.*,_schemas,__consumer_offsets,__transaction_state,connect-configs,connect-offsets,connect-status,connect-statuses", + "ksql.lambdas.enabled" : "true", + "ksql.source.table.materialization.enabled" : "true", + "ksql.query.pull.max.hourly.bandwidth.megabytes" : "2147483647", + "ksql.sink.window.change.log.additional.retention" : "1000000", + "ksql.query.persistent.active.limit" : "2147483647", + "ksql.persistence.wrap.single.values" : null, + "ksql.query.transient.max.bytes.buffering.total" : "-1", + "ksql.connect.basic.auth.credentials.source" : "NONE", + "ksql.schema.registry.url" : "schema_registry.url:0", + "ksql.properties.overrides.denylist" : "", + "ksql.service.id" : "some.ksql.service.id", + "ksql.query.push.v2.max.catchup.consumers" : "5", + "ksql.query.push.v2.enabled" : "false", + "ksql.transient.query.cleanup.service.enable" : "true", + "ksql.query.push.v2.metrics.enabled" : "true", + "ksql.rowpartition.rowoffset.enabled" : "true", + "ksql.streams.commit.interval.ms" : "2000", + "ksql.query.pull.table.scan.enabled" : "true", + "ksql.streams.auto.commit.interval.ms" : "0", + "ksql.streams.topology.optimization" : "all", + "ksql.endpoint.migrate.query" : "true", + "ksql.query.push.v2.registry.installed" : "false", + "ksql.streams.num.stream.threads" : "4", + "ksql.metrics.tags.custom" : "", + "ksql.query.push.v2.catchup.consumer.msg.window" : "50", + "ksql.runtime.feature.shared.enabled" : "false", + "ksql.udf.collect.metrics" : "false", + "ksql.new.query.planner.enabled" : "false", + "ksql.connect.request.headers.plugin" : null, + "ksql.security.extension.class" : null, + "ksql.transient.prefix" : "transient_", + "ksql.headers.columns.enabled" : "true", + "ksql.streams.default.deserialization.exception.handler" : "io.confluent.ksql.errors.LogMetricAndContinueExceptionHandler", + "ksql.connect.request.timeout.ms" : "5000", + "ksql.query.pull.enable.standby.reads" : "false", + "ksql.persistence.default.format.key" : "KAFKA", + "ksql.query.persistent.max.bytes.buffering.total" : "-1", + "ksql.query.error.max.queue.size" : "10", + "ksql.query.cleanup.shutdown.timeout.ms" : "30000", + "ksql.internal.topic.min.insync.replicas" : "1", + "ksql.internal.topic.replicas" : "1", + "ksql.insert.into.values.enabled" : "true", + "ksql.queryanonymizer.cluster_namespace" : null, + "ksql.create.or.replace.enabled" : "true", + "ksql.shared.runtimes.count" : "2", + "ksql.cast.strings.preserve.nulls" : "true", + "ksql.authorization.cache.max.entries" : "10000", + "ksql.pull.queries.enable" : "true", + "ksql.transient.query.cleanup.service.period.seconds" : "600", + "ksql.suppress.enabled" : "false", + "ksql.readonly.topics" : "_confluent.*,__confluent.*,_schemas,__consumer_offsets,__transaction_state,connect-configs,connect-offsets,connect-status,connect-statuses", + "ksql.connect.basic.auth.credentials.file" : "", + "ksql.authorization.cache.expiry.time.secs" : "30", + "ksql.query.retry.backoff.initial.ms" : "15000", + "ksql.query.pull.max.concurrent.requests" : "2147483647", + "ksql.streams.auto.offset.reset" : "earliest", + "ksql.connect.url" : "http://localhost:8083", + "ksql.query.push.v2.new.latest.delay.ms" : "5000", + "ksql.query.push.v2.latest.reset.age.ms" : "30000", + "ksql.streams.default.production.exception.handler" : "io.confluent.ksql.errors.ProductionExceptionHandlerUtil$LogAndFailProductionExceptionHandler", + "ksql.query.pull.interpreter.enabled" : "true", + "ksql.query.pull.limit.clause.enabled" : "true", + "ksql.query.pull.router.thread.pool.size" : "50", + "ksql.query.push.v2.continuation.tokens.enabled" : "false", + "ksql.query.retry.backoff.max.ms" : "900000", + "ksql.timestamp.throw.on.invalid" : "false", + "ksql.persistence.default.format.value" : null, + "ksql.udfs.enabled" : "true", + "ksql.udf.enable.security.manager" : "true", + "ksql.connect.worker.config" : "", + "ksql.nested.error.set.null" : "true", + "ksql.query.pull.thread.pool.size" : "50", + "ksql.persistent.prefix" : "query_", + "ksql.metastore.backup.location" : "", + "ksql.error.classifier.regex" : "", + "ksql.suppress.buffer.size.bytes" : "-1" + } +} \ No newline at end of file diff --git a/ksqldb-functional-tests/src/test/resources/historical_plans/collect-list_-_collect_list_int_table_-_PROTOBUF_NOSR/7.3.0_1651069556550/spec.json b/ksqldb-functional-tests/src/test/resources/historical_plans/collect-list_-_collect_list_int_table_-_PROTOBUF_NOSR/7.3.0_1651069556550/spec.json new file mode 100644 index 000000000000..8ed18128d8d9 --- /dev/null +++ b/ksqldb-functional-tests/src/test/resources/historical_plans/collect-list_-_collect_list_int_table_-_PROTOBUF_NOSR/7.3.0_1651069556550/spec.json @@ -0,0 +1,215 @@ +{ + "version" : "7.3.0", + "timestamp" : 1651069556550, + "path" : "query-validation-tests/collect-list.json", + "schemas" : { + "CTAS_S2_0.Aggregate.Aggregate.Materialize" : { + "schema" : "`ID` BIGINT KEY, `ID` BIGINT, `VALUE` INTEGER, `KSQL_AGG_VARIABLE_0` ARRAY", + "keyFormat" : { + "format" : "KAFKA" + }, + "valueFormat" : { + "format" : "PROTOBUF_NOSR" + } + }, + "CTAS_S2_0.KsqlTopic.Source.Materialized" : { + "schema" : "`ID` BIGINT KEY, `VALUE` INTEGER, `ROWPARTITION` INTEGER, `ROWOFFSET` BIGINT", + "keyFormat" : { + "format" : "KAFKA" + }, + "valueFormat" : { + "format" : "PROTOBUF_NOSR" + } + }, + "CTAS_S2_0.Aggregate.Project" : { + "schema" : "`ID` BIGINT KEY, `COLLECTED` ARRAY", + "keyFormat" : { + "format" : "KAFKA" + }, + "valueFormat" : { + "format" : "PROTOBUF_NOSR" + } + }, + "CTAS_S2_0.Aggregate.Prepare" : { + "schema" : "`ID` BIGINT KEY, `ID` BIGINT, `VALUE` INTEGER", + "keyFormat" : { + "format" : "KAFKA" + }, + "valueFormat" : { + "format" : "PROTOBUF_NOSR" + } + }, + "CTAS_S2_0.KsqlTopic.Source" : { + "schema" : "`ID` BIGINT KEY, `VALUE` INTEGER", + "keyFormat" : { + "format" : "KAFKA" + }, + "valueFormat" : { + "format" : "PROTOBUF_NOSR" + } + }, + "CTAS_S2_0.S2" : { + "schema" : "`ID` BIGINT KEY, `COLLECTED` ARRAY", + "keyFormat" : { + "format" : "KAFKA" + }, + "valueFormat" : { + "format" : "PROTOBUF_NOSR" + } + }, + "CTAS_S2_0.Aggregate.GroupBy" : { + "schema" : "`ID` BIGINT KEY, `ID` BIGINT, `VALUE` INTEGER", + "keyFormat" : { + "format" : "KAFKA" + }, + "valueFormat" : { + "format" : "PROTOBUF_NOSR" + } + } + }, + "testCase" : { + "name" : "collect_list int table - PROTOBUF_NOSR", + "inputs" : [ { + "topic" : "test_topic", + "key" : 0, + "value" : { + "VALUE" : 0 + } + }, { + "topic" : "test_topic", + "key" : 0, + "value" : { + "VALUE" : 100 + } + }, { + "topic" : "test_topic", + "key" : 100, + "value" : { + "VALUE" : 500 + } + }, { + "topic" : "test_topic", + "key" : 100, + "value" : { + "VALUE" : 100 + } + } ], + "outputs" : [ { + "topic" : "S2", + "key" : 0, + "value" : { + "COLLECTED" : [ 0 ] + } + }, { + "topic" : "S2", + "key" : 0, + "value" : { + "COLLECTED" : [ ] + } + }, { + "topic" : "S2", + "key" : 0, + "value" : { + "COLLECTED" : [ 100 ] + } + }, { + "topic" : "S2", + "key" : 100, + "value" : { + "COLLECTED" : [ 500 ] + } + }, { + "topic" : "S2", + "key" : 100, + "value" : { + "COLLECTED" : [ ] + } + }, { + "topic" : "S2", + "key" : 100, + "value" : { + "COLLECTED" : [ 100 ] + } + } ], + "topics" : [ { + "name" : "test_topic", + "replicas" : 1, + "numPartitions" : 4 + }, { + "name" : "S2", + "replicas" : 1, + "numPartitions" : 4 + } ], + "statements" : [ "CREATE TABLE TEST (ID BIGINT PRIMARY KEY, VALUE integer) WITH (kafka_topic='test_topic',value_format='PROTOBUF_NOSR');", "CREATE TABLE S2 as SELECT ID, collect_list(value) as collected FROM test group by id;" ], + "post" : { + "sources" : [ { + "name" : "S2", + "type" : "TABLE", + "schema" : "`ID` BIGINT KEY, `COLLECTED` ARRAY", + "keyFormat" : { + "format" : "KAFKA" + }, + "valueFormat" : "PROTOBUF_NOSR", + "keyFeatures" : [ ], + "valueFeatures" : [ ], + "isSource" : false + }, { + "name" : "TEST", + "type" : "TABLE", + "schema" : "`ID` BIGINT KEY, `VALUE` INTEGER", + "keyFormat" : { + "format" : "KAFKA" + }, + "valueFormat" : "PROTOBUF_NOSR", + "keyFeatures" : [ ], + "valueFeatures" : [ ], + "isSource" : false + } ], + "topics" : { + "topics" : [ { + "name" : "_confluent-ksql-some.ksql.service.idquery_CTAS_S2_0-Aggregate-Aggregate-Materialize-changelog", + "keyFormat" : { + "format" : "KAFKA" + }, + "valueFormat" : { + "format" : "PROTOBUF_NOSR" + } + }, { + "name" : "_confluent-ksql-some.ksql.service.idquery_CTAS_S2_0-KsqlTopic-Reduce-changelog", + "keyFormat" : { + "format" : "KAFKA" + }, + "valueFormat" : { + "format" : "PROTOBUF_NOSR" + } + }, { + "name" : "test_topic", + "keyFormat" : { + "format" : "KAFKA" + }, + "valueFormat" : { + "format" : "PROTOBUF_NOSR" + }, + "partitions" : 4 + }, { + "name" : "_confluent-ksql-some.ksql.service.idquery_CTAS_S2_0-Aggregate-GroupBy-repartition", + "keyFormat" : { + "format" : "KAFKA" + }, + "valueFormat" : { + "format" : "PROTOBUF_NOSR" + } + }, { + "name" : "S2", + "keyFormat" : { + "format" : "KAFKA" + }, + "valueFormat" : { + "format" : "PROTOBUF_NOSR" + }, + "partitions" : 4 + } ] + } + } + } +} \ No newline at end of file diff --git a/ksqldb-functional-tests/src/test/resources/historical_plans/collect-list_-_collect_list_int_table_-_PROTOBUF_NOSR/7.3.0_1651069556550/topology b/ksqldb-functional-tests/src/test/resources/historical_plans/collect-list_-_collect_list_int_table_-_PROTOBUF_NOSR/7.3.0_1651069556550/topology new file mode 100644 index 000000000000..fcff87be4084 --- /dev/null +++ b/ksqldb-functional-tests/src/test/resources/historical_plans/collect-list_-_collect_list_int_table_-_PROTOBUF_NOSR/7.3.0_1651069556550/topology @@ -0,0 +1,43 @@ +Topologies: + Sub-topology: 0 + Source: KSTREAM-SOURCE-0000000001 (topics: [test_topic]) + --> KTABLE-SOURCE-0000000002 + Processor: KTABLE-SOURCE-0000000002 (stores: []) + --> KTABLE-TRANSFORMVALUES-0000000003 + <-- KSTREAM-SOURCE-0000000001 + Processor: KTABLE-TRANSFORMVALUES-0000000003 (stores: [KsqlTopic-Reduce]) + --> KTABLE-TRANSFORMVALUES-0000000004 + <-- KTABLE-SOURCE-0000000002 + Processor: KTABLE-TRANSFORMVALUES-0000000004 (stores: []) + --> Aggregate-Prepare + <-- KTABLE-TRANSFORMVALUES-0000000003 + Processor: Aggregate-Prepare (stores: []) + --> KTABLE-FILTER-0000000006 + <-- KTABLE-TRANSFORMVALUES-0000000004 + Processor: KTABLE-FILTER-0000000006 (stores: []) + --> Aggregate-GroupBy + <-- Aggregate-Prepare + Processor: Aggregate-GroupBy (stores: []) + --> KSTREAM-SINK-0000000008 + <-- KTABLE-FILTER-0000000006 + Sink: KSTREAM-SINK-0000000008 (topic: Aggregate-GroupBy-repartition) + <-- Aggregate-GroupBy + + Sub-topology: 1 + Source: KSTREAM-SOURCE-0000000009 (topics: [Aggregate-GroupBy-repartition]) + --> KTABLE-AGGREGATE-0000000010 + Processor: KTABLE-AGGREGATE-0000000010 (stores: [Aggregate-Aggregate-Materialize]) + --> Aggregate-Aggregate-ToOutputSchema + <-- KSTREAM-SOURCE-0000000009 + Processor: Aggregate-Aggregate-ToOutputSchema (stores: []) + --> Aggregate-Project + <-- KTABLE-AGGREGATE-0000000010 + Processor: Aggregate-Project (stores: []) + --> KTABLE-TOSTREAM-0000000013 + <-- Aggregate-Aggregate-ToOutputSchema + Processor: KTABLE-TOSTREAM-0000000013 (stores: []) + --> KSTREAM-SINK-0000000014 + <-- Aggregate-Project + Sink: KSTREAM-SINK-0000000014 (topic: S2) + <-- KTABLE-TOSTREAM-0000000013 + diff --git a/ksqldb-functional-tests/src/test/resources/historical_plans/collect-list_-_collect_list_long_-_PROTOBUF_NOSR/7.3.0_1651069537580/plan.json b/ksqldb-functional-tests/src/test/resources/historical_plans/collect-list_-_collect_list_long_-_PROTOBUF_NOSR/7.3.0_1651069537580/plan.json new file mode 100644 index 000000000000..6b7ccb095d50 --- /dev/null +++ b/ksqldb-functional-tests/src/test/resources/historical_plans/collect-list_-_collect_list_long_-_PROTOBUF_NOSR/7.3.0_1651069537580/plan.json @@ -0,0 +1,255 @@ +{ + "plan" : [ { + "@type" : "ksqlPlanV1", + "statementText" : "CREATE STREAM TEST (ID BIGINT KEY, VALUE BIGINT) WITH (KAFKA_TOPIC='test_topic', KEY_FORMAT='KAFKA', VALUE_FORMAT='PROTOBUF_NOSR');", + "ddlCommand" : { + "@type" : "createStreamV1", + "sourceName" : "TEST", + "schema" : "`ID` BIGINT KEY, `VALUE` BIGINT", + "timestampColumn" : null, + "topicName" : "test_topic", + "formats" : { + "keyFormat" : { + "format" : "KAFKA", + "properties" : { } + }, + "valueFormat" : { + "format" : "PROTOBUF_NOSR", + "properties" : { } + } + }, + "windowInfo" : null, + "orReplace" : false, + "isSource" : false + }, + "queryPlan" : null + }, { + "@type" : "ksqlPlanV1", + "statementText" : "CREATE TABLE S2 AS SELECT\n TEST.ID ID,\n COLLECT_LIST(TEST.VALUE) COLLECTED\nFROM TEST TEST\nGROUP BY TEST.ID\nEMIT CHANGES", + "ddlCommand" : { + "@type" : "createTableV1", + "sourceName" : "S2", + "schema" : "`ID` BIGINT KEY, `COLLECTED` ARRAY", + "timestampColumn" : null, + "topicName" : "S2", + "formats" : { + "keyFormat" : { + "format" : "KAFKA", + "properties" : { } + }, + "valueFormat" : { + "format" : "PROTOBUF_NOSR", + "properties" : { } + } + }, + "windowInfo" : null, + "orReplace" : false, + "isSource" : false + }, + "queryPlan" : { + "sources" : [ "TEST" ], + "sink" : "S2", + "physicalPlan" : { + "@type" : "tableSinkV1", + "properties" : { + "queryContext" : "S2" + }, + "source" : { + "@type" : "tableSelectV1", + "properties" : { + "queryContext" : "Aggregate/Project" + }, + "source" : { + "@type" : "streamAggregateV1", + "properties" : { + "queryContext" : "Aggregate/Aggregate" + }, + "source" : { + "@type" : "streamGroupByKeyV1", + "properties" : { + "queryContext" : "Aggregate/GroupBy" + }, + "source" : { + "@type" : "streamSelectV1", + "properties" : { + "queryContext" : "Aggregate/Prepare" + }, + "source" : { + "@type" : "streamSourceV1", + "properties" : { + "queryContext" : "KsqlTopic/Source" + }, + "topicName" : "test_topic", + "formats" : { + "keyFormat" : { + "format" : "KAFKA", + "properties" : { } + }, + "valueFormat" : { + "format" : "PROTOBUF_NOSR", + "properties" : { } + } + }, + "timestampColumn" : null, + "sourceSchema" : "`ID` BIGINT KEY, `VALUE` BIGINT", + "pseudoColumnVersion" : 1 + }, + "keyColumnNames" : [ "ID" ], + "selectedKeys" : null, + "selectExpressions" : [ "ID AS ID", "VALUE AS VALUE" ] + }, + "internalFormats" : { + "keyFormat" : { + "format" : "KAFKA", + "properties" : { } + }, + "valueFormat" : { + "format" : "PROTOBUF_NOSR", + "properties" : { } + } + } + }, + "internalFormats" : { + "keyFormat" : { + "format" : "KAFKA", + "properties" : { } + }, + "valueFormat" : { + "format" : "PROTOBUF_NOSR", + "properties" : { } + } + }, + "nonAggregateColumns" : [ "ID", "VALUE" ], + "aggregationFunctions" : [ "COLLECT_LIST(VALUE)" ] + }, + "keyColumnNames" : [ "ID" ], + "selectExpressions" : [ "KSQL_AGG_VARIABLE_0 AS COLLECTED" ], + "internalFormats" : { + "keyFormat" : { + "format" : "KAFKA", + "properties" : { } + }, + "valueFormat" : { + "format" : "PROTOBUF_NOSR", + "properties" : { } + } + } + }, + "formats" : { + "keyFormat" : { + "format" : "KAFKA", + "properties" : { } + }, + "valueFormat" : { + "format" : "PROTOBUF_NOSR", + "properties" : { } + } + }, + "topicName" : "S2", + "timestampColumn" : null + }, + "queryId" : "CTAS_S2_0", + "runtimeId" : null + } + } ], + "configs" : { + "ksql.extension.dir" : "ext", + "ksql.streams.cache.max.bytes.buffering" : "0", + "metric.reporters" : "", + "ksql.query.status.running.threshold.seconds" : "300", + "ksql.connect.basic.auth.credentials.reload" : "false", + "ksql.output.topic.name.prefix" : "", + "ksql.query.pull.stream.enabled" : "true", + "ksql.query.push.v2.interpreter.enabled" : "true", + "ksql.queryanonymizer.logs_enabled" : "true", + "ksql.variable.substitution.enable" : "true", + "ksql.streams.shutdown.timeout.ms" : "300000", + "ksql.query.pull.max.allowed.offset.lag" : "9223372036854775807", + "ksql.query.pull.max.qps" : "2147483647", + "ksql.access.validator.enable" : "auto", + "ksql.streams.bootstrap.servers" : "localhost:0", + "ksql.query.pull.metrics.enabled" : "true", + "ksql.metrics.extension" : null, + "ksql.query.push.v2.alos.enabled" : "true", + "ksql.query.push.v2.max.hourly.bandwidth.megabytes" : "2147483647", + "ksql.query.pull.range.scan.enabled" : "true", + "ksql.transient.query.cleanup.service.initial.delay.seconds" : "600", + "ksql.hidden.topics" : "_confluent.*,__confluent.*,_schemas,__consumer_offsets,__transaction_state,connect-configs,connect-offsets,connect-status,connect-statuses", + "ksql.lambdas.enabled" : "true", + "ksql.source.table.materialization.enabled" : "true", + "ksql.query.pull.max.hourly.bandwidth.megabytes" : "2147483647", + "ksql.sink.window.change.log.additional.retention" : "1000000", + "ksql.query.persistent.active.limit" : "2147483647", + "ksql.persistence.wrap.single.values" : null, + "ksql.query.transient.max.bytes.buffering.total" : "-1", + "ksql.connect.basic.auth.credentials.source" : "NONE", + "ksql.schema.registry.url" : "schema_registry.url:0", + "ksql.properties.overrides.denylist" : "", + "ksql.service.id" : "some.ksql.service.id", + "ksql.query.push.v2.max.catchup.consumers" : "5", + "ksql.query.push.v2.enabled" : "false", + "ksql.transient.query.cleanup.service.enable" : "true", + "ksql.query.push.v2.metrics.enabled" : "true", + "ksql.rowpartition.rowoffset.enabled" : "true", + "ksql.streams.commit.interval.ms" : "2000", + "ksql.query.pull.table.scan.enabled" : "true", + "ksql.streams.auto.commit.interval.ms" : "0", + "ksql.streams.topology.optimization" : "all", + "ksql.endpoint.migrate.query" : "true", + "ksql.query.push.v2.registry.installed" : "false", + "ksql.streams.num.stream.threads" : "4", + "ksql.metrics.tags.custom" : "", + "ksql.query.push.v2.catchup.consumer.msg.window" : "50", + "ksql.runtime.feature.shared.enabled" : "false", + "ksql.udf.collect.metrics" : "false", + "ksql.new.query.planner.enabled" : "false", + "ksql.connect.request.headers.plugin" : null, + "ksql.security.extension.class" : null, + "ksql.transient.prefix" : "transient_", + "ksql.headers.columns.enabled" : "true", + "ksql.streams.default.deserialization.exception.handler" : "io.confluent.ksql.errors.LogMetricAndContinueExceptionHandler", + "ksql.connect.request.timeout.ms" : "5000", + "ksql.query.pull.enable.standby.reads" : "false", + "ksql.persistence.default.format.key" : "KAFKA", + "ksql.query.persistent.max.bytes.buffering.total" : "-1", + "ksql.query.error.max.queue.size" : "10", + "ksql.query.cleanup.shutdown.timeout.ms" : "30000", + "ksql.internal.topic.min.insync.replicas" : "1", + "ksql.internal.topic.replicas" : "1", + "ksql.insert.into.values.enabled" : "true", + "ksql.queryanonymizer.cluster_namespace" : null, + "ksql.create.or.replace.enabled" : "true", + "ksql.shared.runtimes.count" : "2", + "ksql.cast.strings.preserve.nulls" : "true", + "ksql.authorization.cache.max.entries" : "10000", + "ksql.pull.queries.enable" : "true", + "ksql.transient.query.cleanup.service.period.seconds" : "600", + "ksql.suppress.enabled" : "false", + "ksql.readonly.topics" : "_confluent.*,__confluent.*,_schemas,__consumer_offsets,__transaction_state,connect-configs,connect-offsets,connect-status,connect-statuses", + "ksql.connect.basic.auth.credentials.file" : "", + "ksql.authorization.cache.expiry.time.secs" : "30", + "ksql.query.retry.backoff.initial.ms" : "15000", + "ksql.query.pull.max.concurrent.requests" : "2147483647", + "ksql.streams.auto.offset.reset" : "earliest", + "ksql.connect.url" : "http://localhost:8083", + "ksql.query.push.v2.new.latest.delay.ms" : "5000", + "ksql.query.push.v2.latest.reset.age.ms" : "30000", + "ksql.streams.default.production.exception.handler" : "io.confluent.ksql.errors.ProductionExceptionHandlerUtil$LogAndFailProductionExceptionHandler", + "ksql.query.pull.interpreter.enabled" : "true", + "ksql.query.pull.limit.clause.enabled" : "true", + "ksql.query.pull.router.thread.pool.size" : "50", + "ksql.query.push.v2.continuation.tokens.enabled" : "false", + "ksql.query.retry.backoff.max.ms" : "900000", + "ksql.timestamp.throw.on.invalid" : "false", + "ksql.persistence.default.format.value" : null, + "ksql.udfs.enabled" : "true", + "ksql.udf.enable.security.manager" : "true", + "ksql.connect.worker.config" : "", + "ksql.nested.error.set.null" : "true", + "ksql.query.pull.thread.pool.size" : "50", + "ksql.persistent.prefix" : "query_", + "ksql.metastore.backup.location" : "", + "ksql.error.classifier.regex" : "", + "ksql.suppress.buffer.size.bytes" : "-1" + } +} \ No newline at end of file diff --git a/ksqldb-functional-tests/src/test/resources/historical_plans/collect-list_-_collect_list_long_-_PROTOBUF_NOSR/7.3.0_1651069537580/spec.json b/ksqldb-functional-tests/src/test/resources/historical_plans/collect-list_-_collect_list_long_-_PROTOBUF_NOSR/7.3.0_1651069537580/spec.json new file mode 100644 index 000000000000..ebc5c8d400e1 --- /dev/null +++ b/ksqldb-functional-tests/src/test/resources/historical_plans/collect-list_-_collect_list_long_-_PROTOBUF_NOSR/7.3.0_1651069537580/spec.json @@ -0,0 +1,169 @@ +{ + "version" : "7.3.0", + "timestamp" : 1651069537580, + "path" : "query-validation-tests/collect-list.json", + "schemas" : { + "CTAS_S2_0.Aggregate.Aggregate.Materialize" : { + "schema" : "`ID` BIGINT KEY, `ID` BIGINT, `VALUE` BIGINT, `KSQL_AGG_VARIABLE_0` ARRAY", + "keyFormat" : { + "format" : "KAFKA" + }, + "valueFormat" : { + "format" : "PROTOBUF_NOSR" + } + }, + "CTAS_S2_0.Aggregate.Project" : { + "schema" : "`ID` BIGINT KEY, `COLLECTED` ARRAY", + "keyFormat" : { + "format" : "KAFKA" + }, + "valueFormat" : { + "format" : "PROTOBUF_NOSR" + } + }, + "CTAS_S2_0.KsqlTopic.Source" : { + "schema" : "`ID` BIGINT KEY, `VALUE` BIGINT", + "keyFormat" : { + "format" : "KAFKA" + }, + "valueFormat" : { + "format" : "PROTOBUF_NOSR" + } + }, + "CTAS_S2_0.S2" : { + "schema" : "`ID` BIGINT KEY, `COLLECTED` ARRAY", + "keyFormat" : { + "format" : "KAFKA" + }, + "valueFormat" : { + "format" : "PROTOBUF_NOSR" + } + }, + "CTAS_S2_0.Aggregate.GroupBy" : { + "schema" : "`ID` BIGINT KEY, `ID` BIGINT, `VALUE` BIGINT", + "keyFormat" : { + "format" : "KAFKA" + }, + "valueFormat" : { + "format" : "PROTOBUF_NOSR" + } + } + }, + "testCase" : { + "name" : "collect_list long - PROTOBUF_NOSR", + "inputs" : [ { + "topic" : "test_topic", + "key" : 0, + "value" : { + "VALUE" : 2147483648 + } + }, { + "topic" : "test_topic", + "key" : 0, + "value" : { + "VALUE" : 100 + } + }, { + "topic" : "test_topic", + "key" : 100, + "value" : { + "VALUE" : 500 + } + }, { + "topic" : "test_topic", + "key" : 100, + "value" : { + "VALUE" : 100 + } + } ], + "outputs" : [ { + "topic" : "S2", + "key" : 0, + "value" : { + "COLLECTED" : [ 2147483648 ] + } + }, { + "topic" : "S2", + "key" : 0, + "value" : { + "COLLECTED" : [ 2147483648, 100 ] + } + }, { + "topic" : "S2", + "key" : 100, + "value" : { + "COLLECTED" : [ 500 ] + } + }, { + "topic" : "S2", + "key" : 100, + "value" : { + "COLLECTED" : [ 500, 100 ] + } + } ], + "topics" : [ { + "name" : "test_topic", + "replicas" : 1, + "numPartitions" : 4 + }, { + "name" : "S2", + "replicas" : 1, + "numPartitions" : 4 + } ], + "statements" : [ "CREATE STREAM TEST (ID BIGINT KEY, VALUE bigint) WITH (kafka_topic='test_topic', value_format='PROTOBUF_NOSR');", "CREATE TABLE S2 as SELECT ID, collect_list(value) as collected FROM test group by id;" ], + "post" : { + "sources" : [ { + "name" : "S2", + "type" : "TABLE", + "schema" : "`ID` BIGINT KEY, `COLLECTED` ARRAY", + "keyFormat" : { + "format" : "KAFKA" + }, + "valueFormat" : "PROTOBUF_NOSR", + "keyFeatures" : [ ], + "valueFeatures" : [ ], + "isSource" : false + }, { + "name" : "TEST", + "type" : "STREAM", + "schema" : "`ID` BIGINT KEY, `VALUE` BIGINT", + "keyFormat" : { + "format" : "KAFKA" + }, + "valueFormat" : "PROTOBUF_NOSR", + "keyFeatures" : [ ], + "valueFeatures" : [ ], + "isSource" : false + } ], + "topics" : { + "topics" : [ { + "name" : "_confluent-ksql-some.ksql.service.idquery_CTAS_S2_0-Aggregate-Aggregate-Materialize-changelog", + "keyFormat" : { + "format" : "KAFKA" + }, + "valueFormat" : { + "format" : "PROTOBUF_NOSR" + } + }, { + "name" : "test_topic", + "keyFormat" : { + "format" : "KAFKA" + }, + "valueFormat" : { + "format" : "PROTOBUF_NOSR" + }, + "partitions" : 4 + }, { + "name" : "S2", + "keyFormat" : { + "format" : "KAFKA" + }, + "valueFormat" : { + "format" : "PROTOBUF_NOSR" + }, + "partitions" : 4 + } ] + } + } + } +} \ No newline at end of file diff --git a/ksqldb-functional-tests/src/test/resources/historical_plans/collect-list_-_collect_list_long_-_PROTOBUF_NOSR/7.3.0_1651069537580/topology b/ksqldb-functional-tests/src/test/resources/historical_plans/collect-list_-_collect_list_long_-_PROTOBUF_NOSR/7.3.0_1651069537580/topology new file mode 100644 index 000000000000..fcfc8fb28665 --- /dev/null +++ b/ksqldb-functional-tests/src/test/resources/historical_plans/collect-list_-_collect_list_long_-_PROTOBUF_NOSR/7.3.0_1651069537580/topology @@ -0,0 +1,25 @@ +Topologies: + Sub-topology: 0 + Source: KSTREAM-SOURCE-0000000000 (topics: [test_topic]) + --> KSTREAM-TRANSFORMVALUES-0000000001 + Processor: KSTREAM-TRANSFORMVALUES-0000000001 (stores: []) + --> Aggregate-Prepare + <-- KSTREAM-SOURCE-0000000000 + Processor: Aggregate-Prepare (stores: []) + --> KSTREAM-AGGREGATE-0000000003 + <-- KSTREAM-TRANSFORMVALUES-0000000001 + Processor: KSTREAM-AGGREGATE-0000000003 (stores: [Aggregate-Aggregate-Materialize]) + --> Aggregate-Aggregate-ToOutputSchema + <-- Aggregate-Prepare + Processor: Aggregate-Aggregate-ToOutputSchema (stores: []) + --> Aggregate-Project + <-- KSTREAM-AGGREGATE-0000000003 + Processor: Aggregate-Project (stores: []) + --> KTABLE-TOSTREAM-0000000006 + <-- Aggregate-Aggregate-ToOutputSchema + Processor: KTABLE-TOSTREAM-0000000006 (stores: []) + --> KSTREAM-SINK-0000000007 + <-- Aggregate-Project + Sink: KSTREAM-SINK-0000000007 (topic: S2) + <-- KTABLE-TOSTREAM-0000000006 + diff --git a/ksqldb-functional-tests/src/test/resources/historical_plans/collect-list_-_collect_list_long_table_-_PROTOBUF_NOSR/7.3.0_1651069562858/plan.json b/ksqldb-functional-tests/src/test/resources/historical_plans/collect-list_-_collect_list_long_table_-_PROTOBUF_NOSR/7.3.0_1651069562858/plan.json new file mode 100644 index 000000000000..a36d271bd1c1 --- /dev/null +++ b/ksqldb-functional-tests/src/test/resources/historical_plans/collect-list_-_collect_list_long_table_-_PROTOBUF_NOSR/7.3.0_1651069562858/plan.json @@ -0,0 +1,275 @@ +{ + "plan" : [ { + "@type" : "ksqlPlanV1", + "statementText" : "CREATE TABLE TEST (ID BIGINT PRIMARY KEY, VALUE BIGINT) WITH (KAFKA_TOPIC='test_topic', KEY_FORMAT='KAFKA', VALUE_FORMAT='PROTOBUF_NOSR');", + "ddlCommand" : { + "@type" : "createTableV1", + "sourceName" : "TEST", + "schema" : "`ID` BIGINT KEY, `VALUE` BIGINT", + "timestampColumn" : null, + "topicName" : "test_topic", + "formats" : { + "keyFormat" : { + "format" : "KAFKA", + "properties" : { } + }, + "valueFormat" : { + "format" : "PROTOBUF_NOSR", + "properties" : { } + } + }, + "windowInfo" : null, + "orReplace" : false, + "isSource" : false + }, + "queryPlan" : null + }, { + "@type" : "ksqlPlanV1", + "statementText" : "CREATE TABLE S2 AS SELECT\n TEST.ID ID,\n COLLECT_LIST(TEST.VALUE) COLLECTED\nFROM TEST TEST\nGROUP BY TEST.ID\nEMIT CHANGES", + "ddlCommand" : { + "@type" : "createTableV1", + "sourceName" : "S2", + "schema" : "`ID` BIGINT KEY, `COLLECTED` ARRAY", + "timestampColumn" : null, + "topicName" : "S2", + "formats" : { + "keyFormat" : { + "format" : "KAFKA", + "properties" : { } + }, + "valueFormat" : { + "format" : "PROTOBUF_NOSR", + "properties" : { } + } + }, + "windowInfo" : null, + "orReplace" : false, + "isSource" : false + }, + "queryPlan" : { + "sources" : [ "TEST" ], + "sink" : "S2", + "physicalPlan" : { + "@type" : "tableSinkV1", + "properties" : { + "queryContext" : "S2" + }, + "source" : { + "@type" : "tableSelectV1", + "properties" : { + "queryContext" : "Aggregate/Project" + }, + "source" : { + "@type" : "tableAggregateV1", + "properties" : { + "queryContext" : "Aggregate/Aggregate" + }, + "source" : { + "@type" : "tableGroupByV2", + "properties" : { + "queryContext" : "Aggregate/GroupBy" + }, + "source" : { + "@type" : "tableSelectV1", + "properties" : { + "queryContext" : "Aggregate/Prepare" + }, + "source" : { + "@type" : "tableSourceV2", + "properties" : { + "queryContext" : "KsqlTopic/Source" + }, + "topicName" : "test_topic", + "formats" : { + "keyFormat" : { + "format" : "KAFKA", + "properties" : { } + }, + "valueFormat" : { + "format" : "PROTOBUF_NOSR", + "properties" : { } + } + }, + "timestampColumn" : null, + "sourceSchema" : "`ID` BIGINT KEY, `VALUE` BIGINT", + "pseudoColumnVersion" : 1, + "stateStoreFormats" : { + "keyFormat" : { + "format" : "KAFKA", + "properties" : { } + }, + "valueFormat" : { + "format" : "PROTOBUF_NOSR", + "properties" : { } + } + } + }, + "keyColumnNames" : [ "ID" ], + "selectExpressions" : [ "ID AS ID", "VALUE AS VALUE" ], + "internalFormats" : { + "keyFormat" : { + "format" : "KAFKA", + "properties" : { } + }, + "valueFormat" : { + "format" : "PROTOBUF_NOSR", + "properties" : { } + } + } + }, + "internalFormats" : { + "keyFormat" : { + "format" : "KAFKA", + "properties" : { } + }, + "valueFormat" : { + "format" : "PROTOBUF_NOSR", + "properties" : { } + } + }, + "groupByExpressions" : [ "ID" ] + }, + "internalFormats" : { + "keyFormat" : { + "format" : "KAFKA", + "properties" : { } + }, + "valueFormat" : { + "format" : "PROTOBUF_NOSR", + "properties" : { } + } + }, + "nonAggregateColumns" : [ "ID", "VALUE" ], + "aggregationFunctions" : [ "COLLECT_LIST(VALUE)" ] + }, + "keyColumnNames" : [ "ID" ], + "selectExpressions" : [ "KSQL_AGG_VARIABLE_0 AS COLLECTED" ], + "internalFormats" : { + "keyFormat" : { + "format" : "KAFKA", + "properties" : { } + }, + "valueFormat" : { + "format" : "PROTOBUF_NOSR", + "properties" : { } + } + } + }, + "formats" : { + "keyFormat" : { + "format" : "KAFKA", + "properties" : { } + }, + "valueFormat" : { + "format" : "PROTOBUF_NOSR", + "properties" : { } + } + }, + "topicName" : "S2", + "timestampColumn" : null + }, + "queryId" : "CTAS_S2_0", + "runtimeId" : null + } + } ], + "configs" : { + "ksql.extension.dir" : "ext", + "ksql.streams.cache.max.bytes.buffering" : "0", + "metric.reporters" : "", + "ksql.query.status.running.threshold.seconds" : "300", + "ksql.connect.basic.auth.credentials.reload" : "false", + "ksql.output.topic.name.prefix" : "", + "ksql.query.pull.stream.enabled" : "true", + "ksql.query.push.v2.interpreter.enabled" : "true", + "ksql.queryanonymizer.logs_enabled" : "true", + "ksql.variable.substitution.enable" : "true", + "ksql.streams.shutdown.timeout.ms" : "300000", + "ksql.query.pull.max.allowed.offset.lag" : "9223372036854775807", + "ksql.query.pull.max.qps" : "2147483647", + "ksql.access.validator.enable" : "auto", + "ksql.streams.bootstrap.servers" : "localhost:0", + "ksql.query.pull.metrics.enabled" : "true", + "ksql.metrics.extension" : null, + "ksql.query.push.v2.alos.enabled" : "true", + "ksql.query.push.v2.max.hourly.bandwidth.megabytes" : "2147483647", + "ksql.query.pull.range.scan.enabled" : "true", + "ksql.transient.query.cleanup.service.initial.delay.seconds" : "600", + "ksql.hidden.topics" : "_confluent.*,__confluent.*,_schemas,__consumer_offsets,__transaction_state,connect-configs,connect-offsets,connect-status,connect-statuses", + "ksql.lambdas.enabled" : "true", + "ksql.source.table.materialization.enabled" : "true", + "ksql.query.pull.max.hourly.bandwidth.megabytes" : "2147483647", + "ksql.sink.window.change.log.additional.retention" : "1000000", + "ksql.query.persistent.active.limit" : "2147483647", + "ksql.persistence.wrap.single.values" : null, + "ksql.query.transient.max.bytes.buffering.total" : "-1", + "ksql.connect.basic.auth.credentials.source" : "NONE", + "ksql.schema.registry.url" : "schema_registry.url:0", + "ksql.properties.overrides.denylist" : "", + "ksql.service.id" : "some.ksql.service.id", + "ksql.query.push.v2.max.catchup.consumers" : "5", + "ksql.query.push.v2.enabled" : "false", + "ksql.transient.query.cleanup.service.enable" : "true", + "ksql.query.push.v2.metrics.enabled" : "true", + "ksql.rowpartition.rowoffset.enabled" : "true", + "ksql.streams.commit.interval.ms" : "2000", + "ksql.query.pull.table.scan.enabled" : "true", + "ksql.streams.auto.commit.interval.ms" : "0", + "ksql.streams.topology.optimization" : "all", + "ksql.endpoint.migrate.query" : "true", + "ksql.query.push.v2.registry.installed" : "false", + "ksql.streams.num.stream.threads" : "4", + "ksql.metrics.tags.custom" : "", + "ksql.query.push.v2.catchup.consumer.msg.window" : "50", + "ksql.runtime.feature.shared.enabled" : "false", + "ksql.udf.collect.metrics" : "false", + "ksql.new.query.planner.enabled" : "false", + "ksql.connect.request.headers.plugin" : null, + "ksql.security.extension.class" : null, + "ksql.transient.prefix" : "transient_", + "ksql.headers.columns.enabled" : "true", + "ksql.streams.default.deserialization.exception.handler" : "io.confluent.ksql.errors.LogMetricAndContinueExceptionHandler", + "ksql.connect.request.timeout.ms" : "5000", + "ksql.query.pull.enable.standby.reads" : "false", + "ksql.persistence.default.format.key" : "KAFKA", + "ksql.query.persistent.max.bytes.buffering.total" : "-1", + "ksql.query.error.max.queue.size" : "10", + "ksql.query.cleanup.shutdown.timeout.ms" : "30000", + "ksql.internal.topic.min.insync.replicas" : "1", + "ksql.internal.topic.replicas" : "1", + "ksql.insert.into.values.enabled" : "true", + "ksql.queryanonymizer.cluster_namespace" : null, + "ksql.create.or.replace.enabled" : "true", + "ksql.shared.runtimes.count" : "2", + "ksql.cast.strings.preserve.nulls" : "true", + "ksql.authorization.cache.max.entries" : "10000", + "ksql.pull.queries.enable" : "true", + "ksql.transient.query.cleanup.service.period.seconds" : "600", + "ksql.suppress.enabled" : "false", + "ksql.readonly.topics" : "_confluent.*,__confluent.*,_schemas,__consumer_offsets,__transaction_state,connect-configs,connect-offsets,connect-status,connect-statuses", + "ksql.connect.basic.auth.credentials.file" : "", + "ksql.authorization.cache.expiry.time.secs" : "30", + "ksql.query.retry.backoff.initial.ms" : "15000", + "ksql.query.pull.max.concurrent.requests" : "2147483647", + "ksql.streams.auto.offset.reset" : "earliest", + "ksql.connect.url" : "http://localhost:8083", + "ksql.query.push.v2.new.latest.delay.ms" : "5000", + "ksql.query.push.v2.latest.reset.age.ms" : "30000", + "ksql.streams.default.production.exception.handler" : "io.confluent.ksql.errors.ProductionExceptionHandlerUtil$LogAndFailProductionExceptionHandler", + "ksql.query.pull.interpreter.enabled" : "true", + "ksql.query.pull.limit.clause.enabled" : "true", + "ksql.query.pull.router.thread.pool.size" : "50", + "ksql.query.push.v2.continuation.tokens.enabled" : "false", + "ksql.query.retry.backoff.max.ms" : "900000", + "ksql.timestamp.throw.on.invalid" : "false", + "ksql.persistence.default.format.value" : null, + "ksql.udfs.enabled" : "true", + "ksql.udf.enable.security.manager" : "true", + "ksql.connect.worker.config" : "", + "ksql.nested.error.set.null" : "true", + "ksql.query.pull.thread.pool.size" : "50", + "ksql.persistent.prefix" : "query_", + "ksql.metastore.backup.location" : "", + "ksql.error.classifier.regex" : "", + "ksql.suppress.buffer.size.bytes" : "-1" + } +} \ No newline at end of file diff --git a/ksqldb-functional-tests/src/test/resources/historical_plans/collect-list_-_collect_list_long_table_-_PROTOBUF_NOSR/7.3.0_1651069562858/spec.json b/ksqldb-functional-tests/src/test/resources/historical_plans/collect-list_-_collect_list_long_table_-_PROTOBUF_NOSR/7.3.0_1651069562858/spec.json new file mode 100644 index 000000000000..f0ac83ebac99 --- /dev/null +++ b/ksqldb-functional-tests/src/test/resources/historical_plans/collect-list_-_collect_list_long_table_-_PROTOBUF_NOSR/7.3.0_1651069562858/spec.json @@ -0,0 +1,215 @@ +{ + "version" : "7.3.0", + "timestamp" : 1651069562858, + "path" : "query-validation-tests/collect-list.json", + "schemas" : { + "CTAS_S2_0.Aggregate.Aggregate.Materialize" : { + "schema" : "`ID` BIGINT KEY, `ID` BIGINT, `VALUE` BIGINT, `KSQL_AGG_VARIABLE_0` ARRAY", + "keyFormat" : { + "format" : "KAFKA" + }, + "valueFormat" : { + "format" : "PROTOBUF_NOSR" + } + }, + "CTAS_S2_0.KsqlTopic.Source.Materialized" : { + "schema" : "`ID` BIGINT KEY, `VALUE` BIGINT, `ROWPARTITION` INTEGER, `ROWOFFSET` BIGINT", + "keyFormat" : { + "format" : "KAFKA" + }, + "valueFormat" : { + "format" : "PROTOBUF_NOSR" + } + }, + "CTAS_S2_0.Aggregate.Project" : { + "schema" : "`ID` BIGINT KEY, `COLLECTED` ARRAY", + "keyFormat" : { + "format" : "KAFKA" + }, + "valueFormat" : { + "format" : "PROTOBUF_NOSR" + } + }, + "CTAS_S2_0.Aggregate.Prepare" : { + "schema" : "`ID` BIGINT KEY, `ID` BIGINT, `VALUE` BIGINT", + "keyFormat" : { + "format" : "KAFKA" + }, + "valueFormat" : { + "format" : "PROTOBUF_NOSR" + } + }, + "CTAS_S2_0.KsqlTopic.Source" : { + "schema" : "`ID` BIGINT KEY, `VALUE` BIGINT", + "keyFormat" : { + "format" : "KAFKA" + }, + "valueFormat" : { + "format" : "PROTOBUF_NOSR" + } + }, + "CTAS_S2_0.S2" : { + "schema" : "`ID` BIGINT KEY, `COLLECTED` ARRAY", + "keyFormat" : { + "format" : "KAFKA" + }, + "valueFormat" : { + "format" : "PROTOBUF_NOSR" + } + }, + "CTAS_S2_0.Aggregate.GroupBy" : { + "schema" : "`ID` BIGINT KEY, `ID` BIGINT, `VALUE` BIGINT", + "keyFormat" : { + "format" : "KAFKA" + }, + "valueFormat" : { + "format" : "PROTOBUF_NOSR" + } + } + }, + "testCase" : { + "name" : "collect_list long table - PROTOBUF_NOSR", + "inputs" : [ { + "topic" : "test_topic", + "key" : 0, + "value" : { + "VALUE" : 2147483648 + } + }, { + "topic" : "test_topic", + "key" : 0, + "value" : { + "VALUE" : 100 + } + }, { + "topic" : "test_topic", + "key" : 100, + "value" : { + "VALUE" : 500 + } + }, { + "topic" : "test_topic", + "key" : 100, + "value" : { + "VALUE" : 100 + } + } ], + "outputs" : [ { + "topic" : "S2", + "key" : 0, + "value" : { + "COLLECTED" : [ 2147483648 ] + } + }, { + "topic" : "S2", + "key" : 0, + "value" : { + "COLLECTED" : [ ] + } + }, { + "topic" : "S2", + "key" : 0, + "value" : { + "COLLECTED" : [ 100 ] + } + }, { + "topic" : "S2", + "key" : 100, + "value" : { + "COLLECTED" : [ 500 ] + } + }, { + "topic" : "S2", + "key" : 100, + "value" : { + "COLLECTED" : [ ] + } + }, { + "topic" : "S2", + "key" : 100, + "value" : { + "COLLECTED" : [ 100 ] + } + } ], + "topics" : [ { + "name" : "test_topic", + "replicas" : 1, + "numPartitions" : 4 + }, { + "name" : "S2", + "replicas" : 1, + "numPartitions" : 4 + } ], + "statements" : [ "CREATE TABLE TEST (ID BIGINT PRIMARY KEY, VALUE bigint) WITH (kafka_topic='test_topic', value_format='PROTOBUF_NOSR');", "CREATE TABLE S2 as SELECT ID, collect_list(value) as collected FROM test group by id;" ], + "post" : { + "sources" : [ { + "name" : "S2", + "type" : "TABLE", + "schema" : "`ID` BIGINT KEY, `COLLECTED` ARRAY", + "keyFormat" : { + "format" : "KAFKA" + }, + "valueFormat" : "PROTOBUF_NOSR", + "keyFeatures" : [ ], + "valueFeatures" : [ ], + "isSource" : false + }, { + "name" : "TEST", + "type" : "TABLE", + "schema" : "`ID` BIGINT KEY, `VALUE` BIGINT", + "keyFormat" : { + "format" : "KAFKA" + }, + "valueFormat" : "PROTOBUF_NOSR", + "keyFeatures" : [ ], + "valueFeatures" : [ ], + "isSource" : false + } ], + "topics" : { + "topics" : [ { + "name" : "_confluent-ksql-some.ksql.service.idquery_CTAS_S2_0-Aggregate-Aggregate-Materialize-changelog", + "keyFormat" : { + "format" : "KAFKA" + }, + "valueFormat" : { + "format" : "PROTOBUF_NOSR" + } + }, { + "name" : "_confluent-ksql-some.ksql.service.idquery_CTAS_S2_0-KsqlTopic-Reduce-changelog", + "keyFormat" : { + "format" : "KAFKA" + }, + "valueFormat" : { + "format" : "PROTOBUF_NOSR" + } + }, { + "name" : "test_topic", + "keyFormat" : { + "format" : "KAFKA" + }, + "valueFormat" : { + "format" : "PROTOBUF_NOSR" + }, + "partitions" : 4 + }, { + "name" : "_confluent-ksql-some.ksql.service.idquery_CTAS_S2_0-Aggregate-GroupBy-repartition", + "keyFormat" : { + "format" : "KAFKA" + }, + "valueFormat" : { + "format" : "PROTOBUF_NOSR" + } + }, { + "name" : "S2", + "keyFormat" : { + "format" : "KAFKA" + }, + "valueFormat" : { + "format" : "PROTOBUF_NOSR" + }, + "partitions" : 4 + } ] + } + } + } +} \ No newline at end of file diff --git a/ksqldb-functional-tests/src/test/resources/historical_plans/collect-list_-_collect_list_long_table_-_PROTOBUF_NOSR/7.3.0_1651069562858/topology b/ksqldb-functional-tests/src/test/resources/historical_plans/collect-list_-_collect_list_long_table_-_PROTOBUF_NOSR/7.3.0_1651069562858/topology new file mode 100644 index 000000000000..fcff87be4084 --- /dev/null +++ b/ksqldb-functional-tests/src/test/resources/historical_plans/collect-list_-_collect_list_long_table_-_PROTOBUF_NOSR/7.3.0_1651069562858/topology @@ -0,0 +1,43 @@ +Topologies: + Sub-topology: 0 + Source: KSTREAM-SOURCE-0000000001 (topics: [test_topic]) + --> KTABLE-SOURCE-0000000002 + Processor: KTABLE-SOURCE-0000000002 (stores: []) + --> KTABLE-TRANSFORMVALUES-0000000003 + <-- KSTREAM-SOURCE-0000000001 + Processor: KTABLE-TRANSFORMVALUES-0000000003 (stores: [KsqlTopic-Reduce]) + --> KTABLE-TRANSFORMVALUES-0000000004 + <-- KTABLE-SOURCE-0000000002 + Processor: KTABLE-TRANSFORMVALUES-0000000004 (stores: []) + --> Aggregate-Prepare + <-- KTABLE-TRANSFORMVALUES-0000000003 + Processor: Aggregate-Prepare (stores: []) + --> KTABLE-FILTER-0000000006 + <-- KTABLE-TRANSFORMVALUES-0000000004 + Processor: KTABLE-FILTER-0000000006 (stores: []) + --> Aggregate-GroupBy + <-- Aggregate-Prepare + Processor: Aggregate-GroupBy (stores: []) + --> KSTREAM-SINK-0000000008 + <-- KTABLE-FILTER-0000000006 + Sink: KSTREAM-SINK-0000000008 (topic: Aggregate-GroupBy-repartition) + <-- Aggregate-GroupBy + + Sub-topology: 1 + Source: KSTREAM-SOURCE-0000000009 (topics: [Aggregate-GroupBy-repartition]) + --> KTABLE-AGGREGATE-0000000010 + Processor: KTABLE-AGGREGATE-0000000010 (stores: [Aggregate-Aggregate-Materialize]) + --> Aggregate-Aggregate-ToOutputSchema + <-- KSTREAM-SOURCE-0000000009 + Processor: Aggregate-Aggregate-ToOutputSchema (stores: []) + --> Aggregate-Project + <-- KTABLE-AGGREGATE-0000000010 + Processor: Aggregate-Project (stores: []) + --> KTABLE-TOSTREAM-0000000013 + <-- Aggregate-Aggregate-ToOutputSchema + Processor: KTABLE-TOSTREAM-0000000013 (stores: []) + --> KSTREAM-SINK-0000000014 + <-- Aggregate-Project + Sink: KSTREAM-SINK-0000000014 (topic: S2) + <-- KTABLE-TOSTREAM-0000000013 + diff --git a/ksqldb-functional-tests/src/test/resources/historical_plans/collect-list_-_collect_list_string_table_-_PROTOBUF_NOSR/7.3.0_1651069576265/plan.json b/ksqldb-functional-tests/src/test/resources/historical_plans/collect-list_-_collect_list_string_table_-_PROTOBUF_NOSR/7.3.0_1651069576265/plan.json new file mode 100644 index 000000000000..81c3d1728870 --- /dev/null +++ b/ksqldb-functional-tests/src/test/resources/historical_plans/collect-list_-_collect_list_string_table_-_PROTOBUF_NOSR/7.3.0_1651069576265/plan.json @@ -0,0 +1,275 @@ +{ + "plan" : [ { + "@type" : "ksqlPlanV1", + "statementText" : "CREATE TABLE TEST (ID BIGINT PRIMARY KEY, VALUE STRING) WITH (KAFKA_TOPIC='test_topic', KEY_FORMAT='KAFKA', VALUE_FORMAT='PROTOBUF_NOSR');", + "ddlCommand" : { + "@type" : "createTableV1", + "sourceName" : "TEST", + "schema" : "`ID` BIGINT KEY, `VALUE` STRING", + "timestampColumn" : null, + "topicName" : "test_topic", + "formats" : { + "keyFormat" : { + "format" : "KAFKA", + "properties" : { } + }, + "valueFormat" : { + "format" : "PROTOBUF_NOSR", + "properties" : { } + } + }, + "windowInfo" : null, + "orReplace" : false, + "isSource" : false + }, + "queryPlan" : null + }, { + "@type" : "ksqlPlanV1", + "statementText" : "CREATE TABLE S2 AS SELECT\n TEST.ID ID,\n COLLECT_LIST(TEST.VALUE) COLLECTED\nFROM TEST TEST\nGROUP BY TEST.ID\nEMIT CHANGES", + "ddlCommand" : { + "@type" : "createTableV1", + "sourceName" : "S2", + "schema" : "`ID` BIGINT KEY, `COLLECTED` ARRAY", + "timestampColumn" : null, + "topicName" : "S2", + "formats" : { + "keyFormat" : { + "format" : "KAFKA", + "properties" : { } + }, + "valueFormat" : { + "format" : "PROTOBUF_NOSR", + "properties" : { } + } + }, + "windowInfo" : null, + "orReplace" : false, + "isSource" : false + }, + "queryPlan" : { + "sources" : [ "TEST" ], + "sink" : "S2", + "physicalPlan" : { + "@type" : "tableSinkV1", + "properties" : { + "queryContext" : "S2" + }, + "source" : { + "@type" : "tableSelectV1", + "properties" : { + "queryContext" : "Aggregate/Project" + }, + "source" : { + "@type" : "tableAggregateV1", + "properties" : { + "queryContext" : "Aggregate/Aggregate" + }, + "source" : { + "@type" : "tableGroupByV2", + "properties" : { + "queryContext" : "Aggregate/GroupBy" + }, + "source" : { + "@type" : "tableSelectV1", + "properties" : { + "queryContext" : "Aggregate/Prepare" + }, + "source" : { + "@type" : "tableSourceV2", + "properties" : { + "queryContext" : "KsqlTopic/Source" + }, + "topicName" : "test_topic", + "formats" : { + "keyFormat" : { + "format" : "KAFKA", + "properties" : { } + }, + "valueFormat" : { + "format" : "PROTOBUF_NOSR", + "properties" : { } + } + }, + "timestampColumn" : null, + "sourceSchema" : "`ID` BIGINT KEY, `VALUE` STRING", + "pseudoColumnVersion" : 1, + "stateStoreFormats" : { + "keyFormat" : { + "format" : "KAFKA", + "properties" : { } + }, + "valueFormat" : { + "format" : "PROTOBUF_NOSR", + "properties" : { } + } + } + }, + "keyColumnNames" : [ "ID" ], + "selectExpressions" : [ "ID AS ID", "VALUE AS VALUE" ], + "internalFormats" : { + "keyFormat" : { + "format" : "KAFKA", + "properties" : { } + }, + "valueFormat" : { + "format" : "PROTOBUF_NOSR", + "properties" : { } + } + } + }, + "internalFormats" : { + "keyFormat" : { + "format" : "KAFKA", + "properties" : { } + }, + "valueFormat" : { + "format" : "PROTOBUF_NOSR", + "properties" : { } + } + }, + "groupByExpressions" : [ "ID" ] + }, + "internalFormats" : { + "keyFormat" : { + "format" : "KAFKA", + "properties" : { } + }, + "valueFormat" : { + "format" : "PROTOBUF_NOSR", + "properties" : { } + } + }, + "nonAggregateColumns" : [ "ID", "VALUE" ], + "aggregationFunctions" : [ "COLLECT_LIST(VALUE)" ] + }, + "keyColumnNames" : [ "ID" ], + "selectExpressions" : [ "KSQL_AGG_VARIABLE_0 AS COLLECTED" ], + "internalFormats" : { + "keyFormat" : { + "format" : "KAFKA", + "properties" : { } + }, + "valueFormat" : { + "format" : "PROTOBUF_NOSR", + "properties" : { } + } + } + }, + "formats" : { + "keyFormat" : { + "format" : "KAFKA", + "properties" : { } + }, + "valueFormat" : { + "format" : "PROTOBUF_NOSR", + "properties" : { } + } + }, + "topicName" : "S2", + "timestampColumn" : null + }, + "queryId" : "CTAS_S2_0", + "runtimeId" : null + } + } ], + "configs" : { + "ksql.extension.dir" : "ext", + "ksql.streams.cache.max.bytes.buffering" : "0", + "metric.reporters" : "", + "ksql.query.status.running.threshold.seconds" : "300", + "ksql.connect.basic.auth.credentials.reload" : "false", + "ksql.output.topic.name.prefix" : "", + "ksql.query.pull.stream.enabled" : "true", + "ksql.query.push.v2.interpreter.enabled" : "true", + "ksql.queryanonymizer.logs_enabled" : "true", + "ksql.variable.substitution.enable" : "true", + "ksql.streams.shutdown.timeout.ms" : "300000", + "ksql.query.pull.max.allowed.offset.lag" : "9223372036854775807", + "ksql.query.pull.max.qps" : "2147483647", + "ksql.access.validator.enable" : "auto", + "ksql.streams.bootstrap.servers" : "localhost:0", + "ksql.query.pull.metrics.enabled" : "true", + "ksql.metrics.extension" : null, + "ksql.query.push.v2.alos.enabled" : "true", + "ksql.query.push.v2.max.hourly.bandwidth.megabytes" : "2147483647", + "ksql.query.pull.range.scan.enabled" : "true", + "ksql.transient.query.cleanup.service.initial.delay.seconds" : "600", + "ksql.hidden.topics" : "_confluent.*,__confluent.*,_schemas,__consumer_offsets,__transaction_state,connect-configs,connect-offsets,connect-status,connect-statuses", + "ksql.lambdas.enabled" : "true", + "ksql.source.table.materialization.enabled" : "true", + "ksql.query.pull.max.hourly.bandwidth.megabytes" : "2147483647", + "ksql.sink.window.change.log.additional.retention" : "1000000", + "ksql.query.persistent.active.limit" : "2147483647", + "ksql.persistence.wrap.single.values" : null, + "ksql.query.transient.max.bytes.buffering.total" : "-1", + "ksql.connect.basic.auth.credentials.source" : "NONE", + "ksql.schema.registry.url" : "schema_registry.url:0", + "ksql.properties.overrides.denylist" : "", + "ksql.service.id" : "some.ksql.service.id", + "ksql.query.push.v2.max.catchup.consumers" : "5", + "ksql.query.push.v2.enabled" : "false", + "ksql.transient.query.cleanup.service.enable" : "true", + "ksql.query.push.v2.metrics.enabled" : "true", + "ksql.rowpartition.rowoffset.enabled" : "true", + "ksql.streams.commit.interval.ms" : "2000", + "ksql.query.pull.table.scan.enabled" : "true", + "ksql.streams.auto.commit.interval.ms" : "0", + "ksql.streams.topology.optimization" : "all", + "ksql.endpoint.migrate.query" : "true", + "ksql.query.push.v2.registry.installed" : "false", + "ksql.streams.num.stream.threads" : "4", + "ksql.metrics.tags.custom" : "", + "ksql.query.push.v2.catchup.consumer.msg.window" : "50", + "ksql.runtime.feature.shared.enabled" : "false", + "ksql.udf.collect.metrics" : "false", + "ksql.new.query.planner.enabled" : "false", + "ksql.connect.request.headers.plugin" : null, + "ksql.security.extension.class" : null, + "ksql.transient.prefix" : "transient_", + "ksql.headers.columns.enabled" : "true", + "ksql.streams.default.deserialization.exception.handler" : "io.confluent.ksql.errors.LogMetricAndContinueExceptionHandler", + "ksql.connect.request.timeout.ms" : "5000", + "ksql.query.pull.enable.standby.reads" : "false", + "ksql.persistence.default.format.key" : "KAFKA", + "ksql.query.persistent.max.bytes.buffering.total" : "-1", + "ksql.query.error.max.queue.size" : "10", + "ksql.query.cleanup.shutdown.timeout.ms" : "30000", + "ksql.internal.topic.min.insync.replicas" : "1", + "ksql.internal.topic.replicas" : "1", + "ksql.insert.into.values.enabled" : "true", + "ksql.queryanonymizer.cluster_namespace" : null, + "ksql.create.or.replace.enabled" : "true", + "ksql.shared.runtimes.count" : "2", + "ksql.cast.strings.preserve.nulls" : "true", + "ksql.authorization.cache.max.entries" : "10000", + "ksql.pull.queries.enable" : "true", + "ksql.transient.query.cleanup.service.period.seconds" : "600", + "ksql.suppress.enabled" : "false", + "ksql.readonly.topics" : "_confluent.*,__confluent.*,_schemas,__consumer_offsets,__transaction_state,connect-configs,connect-offsets,connect-status,connect-statuses", + "ksql.connect.basic.auth.credentials.file" : "", + "ksql.authorization.cache.expiry.time.secs" : "30", + "ksql.query.retry.backoff.initial.ms" : "15000", + "ksql.query.pull.max.concurrent.requests" : "2147483647", + "ksql.streams.auto.offset.reset" : "earliest", + "ksql.connect.url" : "http://localhost:8083", + "ksql.query.push.v2.new.latest.delay.ms" : "5000", + "ksql.query.push.v2.latest.reset.age.ms" : "30000", + "ksql.streams.default.production.exception.handler" : "io.confluent.ksql.errors.ProductionExceptionHandlerUtil$LogAndFailProductionExceptionHandler", + "ksql.query.pull.interpreter.enabled" : "true", + "ksql.query.pull.limit.clause.enabled" : "true", + "ksql.query.pull.router.thread.pool.size" : "50", + "ksql.query.push.v2.continuation.tokens.enabled" : "false", + "ksql.query.retry.backoff.max.ms" : "900000", + "ksql.timestamp.throw.on.invalid" : "false", + "ksql.persistence.default.format.value" : null, + "ksql.udfs.enabled" : "true", + "ksql.udf.enable.security.manager" : "true", + "ksql.connect.worker.config" : "", + "ksql.nested.error.set.null" : "true", + "ksql.query.pull.thread.pool.size" : "50", + "ksql.persistent.prefix" : "query_", + "ksql.metastore.backup.location" : "", + "ksql.error.classifier.regex" : "", + "ksql.suppress.buffer.size.bytes" : "-1" + } +} \ No newline at end of file diff --git a/ksqldb-functional-tests/src/test/resources/historical_plans/collect-list_-_collect_list_string_table_-_PROTOBUF_NOSR/7.3.0_1651069576265/spec.json b/ksqldb-functional-tests/src/test/resources/historical_plans/collect-list_-_collect_list_string_table_-_PROTOBUF_NOSR/7.3.0_1651069576265/spec.json new file mode 100644 index 000000000000..f1ca08be7ed2 --- /dev/null +++ b/ksqldb-functional-tests/src/test/resources/historical_plans/collect-list_-_collect_list_string_table_-_PROTOBUF_NOSR/7.3.0_1651069576265/spec.json @@ -0,0 +1,233 @@ +{ + "version" : "7.3.0", + "timestamp" : 1651069576265, + "path" : "query-validation-tests/collect-list.json", + "schemas" : { + "CTAS_S2_0.Aggregate.Aggregate.Materialize" : { + "schema" : "`ID` BIGINT KEY, `ID` BIGINT, `VALUE` STRING, `KSQL_AGG_VARIABLE_0` ARRAY", + "keyFormat" : { + "format" : "KAFKA" + }, + "valueFormat" : { + "format" : "PROTOBUF_NOSR" + } + }, + "CTAS_S2_0.KsqlTopic.Source.Materialized" : { + "schema" : "`ID` BIGINT KEY, `VALUE` STRING, `ROWPARTITION` INTEGER, `ROWOFFSET` BIGINT", + "keyFormat" : { + "format" : "KAFKA" + }, + "valueFormat" : { + "format" : "PROTOBUF_NOSR" + } + }, + "CTAS_S2_0.Aggregate.Project" : { + "schema" : "`ID` BIGINT KEY, `COLLECTED` ARRAY", + "keyFormat" : { + "format" : "KAFKA" + }, + "valueFormat" : { + "format" : "PROTOBUF_NOSR" + } + }, + "CTAS_S2_0.Aggregate.Prepare" : { + "schema" : "`ID` BIGINT KEY, `ID` BIGINT, `VALUE` STRING", + "keyFormat" : { + "format" : "KAFKA" + }, + "valueFormat" : { + "format" : "PROTOBUF_NOSR" + } + }, + "CTAS_S2_0.KsqlTopic.Source" : { + "schema" : "`ID` BIGINT KEY, `VALUE` STRING", + "keyFormat" : { + "format" : "KAFKA" + }, + "valueFormat" : { + "format" : "PROTOBUF_NOSR" + } + }, + "CTAS_S2_0.S2" : { + "schema" : "`ID` BIGINT KEY, `COLLECTED` ARRAY", + "keyFormat" : { + "format" : "KAFKA" + }, + "valueFormat" : { + "format" : "PROTOBUF_NOSR" + } + }, + "CTAS_S2_0.Aggregate.GroupBy" : { + "schema" : "`ID` BIGINT KEY, `ID` BIGINT, `VALUE` STRING", + "keyFormat" : { + "format" : "KAFKA" + }, + "valueFormat" : { + "format" : "PROTOBUF_NOSR" + } + } + }, + "testCase" : { + "name" : "collect_list string table - PROTOBUF_NOSR", + "inputs" : [ { + "topic" : "test_topic", + "key" : 0, + "value" : { + "VALUE" : "foo" + } + }, { + "topic" : "test_topic", + "key" : 100, + "value" : { + "VALUE" : "baz" + } + }, { + "topic" : "test_topic", + "key" : 0, + "value" : { + "VALUE" : "bar" + } + }, { + "topic" : "test_topic", + "key" : 100, + "value" : { + "VALUE" : "baz" + } + }, { + "topic" : "test_topic", + "key" : 100, + "value" : { + "VALUE" : "foo" + } + } ], + "outputs" : [ { + "topic" : "S2", + "key" : 0, + "value" : { + "COLLECTED" : [ "foo" ] + } + }, { + "topic" : "S2", + "key" : 100, + "value" : { + "COLLECTED" : [ "baz" ] + } + }, { + "topic" : "S2", + "key" : 0, + "value" : { + "COLLECTED" : [ ] + } + }, { + "topic" : "S2", + "key" : 0, + "value" : { + "COLLECTED" : [ "bar" ] + } + }, { + "topic" : "S2", + "key" : 100, + "value" : { + "COLLECTED" : [ ] + } + }, { + "topic" : "S2", + "key" : 100, + "value" : { + "COLLECTED" : [ "baz" ] + } + }, { + "topic" : "S2", + "key" : 100, + "value" : { + "COLLECTED" : [ ] + } + }, { + "topic" : "S2", + "key" : 100, + "value" : { + "COLLECTED" : [ "foo" ] + } + } ], + "topics" : [ { + "name" : "test_topic", + "replicas" : 1, + "numPartitions" : 4 + }, { + "name" : "S2", + "replicas" : 1, + "numPartitions" : 4 + } ], + "statements" : [ "CREATE TABLE TEST (ID BIGINT PRIMARY KEY, VALUE varchar) WITH (kafka_topic='test_topic', value_format='PROTOBUF_NOSR');", "CREATE TABLE S2 as SELECT ID, collect_list(value) as collected FROM test group by id;" ], + "post" : { + "sources" : [ { + "name" : "S2", + "type" : "TABLE", + "schema" : "`ID` BIGINT KEY, `COLLECTED` ARRAY", + "keyFormat" : { + "format" : "KAFKA" + }, + "valueFormat" : "PROTOBUF_NOSR", + "keyFeatures" : [ ], + "valueFeatures" : [ ], + "isSource" : false + }, { + "name" : "TEST", + "type" : "TABLE", + "schema" : "`ID` BIGINT KEY, `VALUE` STRING", + "keyFormat" : { + "format" : "KAFKA" + }, + "valueFormat" : "PROTOBUF_NOSR", + "keyFeatures" : [ ], + "valueFeatures" : [ ], + "isSource" : false + } ], + "topics" : { + "topics" : [ { + "name" : "_confluent-ksql-some.ksql.service.idquery_CTAS_S2_0-Aggregate-Aggregate-Materialize-changelog", + "keyFormat" : { + "format" : "KAFKA" + }, + "valueFormat" : { + "format" : "PROTOBUF_NOSR" + } + }, { + "name" : "_confluent-ksql-some.ksql.service.idquery_CTAS_S2_0-KsqlTopic-Reduce-changelog", + "keyFormat" : { + "format" : "KAFKA" + }, + "valueFormat" : { + "format" : "PROTOBUF_NOSR" + } + }, { + "name" : "test_topic", + "keyFormat" : { + "format" : "KAFKA" + }, + "valueFormat" : { + "format" : "PROTOBUF_NOSR" + }, + "partitions" : 4 + }, { + "name" : "_confluent-ksql-some.ksql.service.idquery_CTAS_S2_0-Aggregate-GroupBy-repartition", + "keyFormat" : { + "format" : "KAFKA" + }, + "valueFormat" : { + "format" : "PROTOBUF_NOSR" + } + }, { + "name" : "S2", + "keyFormat" : { + "format" : "KAFKA" + }, + "valueFormat" : { + "format" : "PROTOBUF_NOSR" + }, + "partitions" : 4 + } ] + } + } + } +} \ No newline at end of file diff --git a/ksqldb-functional-tests/src/test/resources/historical_plans/collect-list_-_collect_list_string_table_-_PROTOBUF_NOSR/7.3.0_1651069576265/topology b/ksqldb-functional-tests/src/test/resources/historical_plans/collect-list_-_collect_list_string_table_-_PROTOBUF_NOSR/7.3.0_1651069576265/topology new file mode 100644 index 000000000000..fcff87be4084 --- /dev/null +++ b/ksqldb-functional-tests/src/test/resources/historical_plans/collect-list_-_collect_list_string_table_-_PROTOBUF_NOSR/7.3.0_1651069576265/topology @@ -0,0 +1,43 @@ +Topologies: + Sub-topology: 0 + Source: KSTREAM-SOURCE-0000000001 (topics: [test_topic]) + --> KTABLE-SOURCE-0000000002 + Processor: KTABLE-SOURCE-0000000002 (stores: []) + --> KTABLE-TRANSFORMVALUES-0000000003 + <-- KSTREAM-SOURCE-0000000001 + Processor: KTABLE-TRANSFORMVALUES-0000000003 (stores: [KsqlTopic-Reduce]) + --> KTABLE-TRANSFORMVALUES-0000000004 + <-- KTABLE-SOURCE-0000000002 + Processor: KTABLE-TRANSFORMVALUES-0000000004 (stores: []) + --> Aggregate-Prepare + <-- KTABLE-TRANSFORMVALUES-0000000003 + Processor: Aggregate-Prepare (stores: []) + --> KTABLE-FILTER-0000000006 + <-- KTABLE-TRANSFORMVALUES-0000000004 + Processor: KTABLE-FILTER-0000000006 (stores: []) + --> Aggregate-GroupBy + <-- Aggregate-Prepare + Processor: Aggregate-GroupBy (stores: []) + --> KSTREAM-SINK-0000000008 + <-- KTABLE-FILTER-0000000006 + Sink: KSTREAM-SINK-0000000008 (topic: Aggregate-GroupBy-repartition) + <-- Aggregate-GroupBy + + Sub-topology: 1 + Source: KSTREAM-SOURCE-0000000009 (topics: [Aggregate-GroupBy-repartition]) + --> KTABLE-AGGREGATE-0000000010 + Processor: KTABLE-AGGREGATE-0000000010 (stores: [Aggregate-Aggregate-Materialize]) + --> Aggregate-Aggregate-ToOutputSchema + <-- KSTREAM-SOURCE-0000000009 + Processor: Aggregate-Aggregate-ToOutputSchema (stores: []) + --> Aggregate-Project + <-- KTABLE-AGGREGATE-0000000010 + Processor: Aggregate-Project (stores: []) + --> KTABLE-TOSTREAM-0000000013 + <-- Aggregate-Aggregate-ToOutputSchema + Processor: KTABLE-TOSTREAM-0000000013 (stores: []) + --> KSTREAM-SINK-0000000014 + <-- Aggregate-Project + Sink: KSTREAM-SINK-0000000014 (topic: S2) + <-- KTABLE-TOSTREAM-0000000013 + diff --git a/ksqldb-functional-tests/src/test/resources/historical_plans/collect-list_-_collect_list_struct_-_PROTOBUF_NOSR/7.3.0_1651069531802/plan.json b/ksqldb-functional-tests/src/test/resources/historical_plans/collect-list_-_collect_list_struct_-_PROTOBUF_NOSR/7.3.0_1651069531802/plan.json new file mode 100644 index 000000000000..8c64bb9ad843 --- /dev/null +++ b/ksqldb-functional-tests/src/test/resources/historical_plans/collect-list_-_collect_list_struct_-_PROTOBUF_NOSR/7.3.0_1651069531802/plan.json @@ -0,0 +1,255 @@ +{ + "plan" : [ { + "@type" : "ksqlPlanV1", + "statementText" : "CREATE STREAM TEST (ID BIGINT KEY, VALUE STRUCT) WITH (KAFKA_TOPIC='test_topic', KEY_FORMAT='KAFKA', VALUE_FORMAT='PROTOBUF_NOSR');", + "ddlCommand" : { + "@type" : "createStreamV1", + "sourceName" : "TEST", + "schema" : "`ID` BIGINT KEY, `VALUE` STRUCT<`A` STRING, `B` BIGINT>", + "timestampColumn" : null, + "topicName" : "test_topic", + "formats" : { + "keyFormat" : { + "format" : "KAFKA", + "properties" : { } + }, + "valueFormat" : { + "format" : "PROTOBUF_NOSR", + "properties" : { } + } + }, + "windowInfo" : null, + "orReplace" : false, + "isSource" : false + }, + "queryPlan" : null + }, { + "@type" : "ksqlPlanV1", + "statementText" : "CREATE TABLE S2 AS SELECT\n TEST.ID ID,\n COLLECT_LIST(TEST.VALUE) COLLECTED\nFROM TEST TEST\nGROUP BY TEST.ID\nEMIT CHANGES", + "ddlCommand" : { + "@type" : "createTableV1", + "sourceName" : "S2", + "schema" : "`ID` BIGINT KEY, `COLLECTED` ARRAY>", + "timestampColumn" : null, + "topicName" : "S2", + "formats" : { + "keyFormat" : { + "format" : "KAFKA", + "properties" : { } + }, + "valueFormat" : { + "format" : "PROTOBUF_NOSR", + "properties" : { } + } + }, + "windowInfo" : null, + "orReplace" : false, + "isSource" : false + }, + "queryPlan" : { + "sources" : [ "TEST" ], + "sink" : "S2", + "physicalPlan" : { + "@type" : "tableSinkV1", + "properties" : { + "queryContext" : "S2" + }, + "source" : { + "@type" : "tableSelectV1", + "properties" : { + "queryContext" : "Aggregate/Project" + }, + "source" : { + "@type" : "streamAggregateV1", + "properties" : { + "queryContext" : "Aggregate/Aggregate" + }, + "source" : { + "@type" : "streamGroupByKeyV1", + "properties" : { + "queryContext" : "Aggregate/GroupBy" + }, + "source" : { + "@type" : "streamSelectV1", + "properties" : { + "queryContext" : "Aggregate/Prepare" + }, + "source" : { + "@type" : "streamSourceV1", + "properties" : { + "queryContext" : "KsqlTopic/Source" + }, + "topicName" : "test_topic", + "formats" : { + "keyFormat" : { + "format" : "KAFKA", + "properties" : { } + }, + "valueFormat" : { + "format" : "PROTOBUF_NOSR", + "properties" : { } + } + }, + "timestampColumn" : null, + "sourceSchema" : "`ID` BIGINT KEY, `VALUE` STRUCT<`A` STRING, `B` BIGINT>", + "pseudoColumnVersion" : 1 + }, + "keyColumnNames" : [ "ID" ], + "selectedKeys" : null, + "selectExpressions" : [ "ID AS ID", "VALUE AS VALUE" ] + }, + "internalFormats" : { + "keyFormat" : { + "format" : "KAFKA", + "properties" : { } + }, + "valueFormat" : { + "format" : "PROTOBUF_NOSR", + "properties" : { } + } + } + }, + "internalFormats" : { + "keyFormat" : { + "format" : "KAFKA", + "properties" : { } + }, + "valueFormat" : { + "format" : "PROTOBUF_NOSR", + "properties" : { } + } + }, + "nonAggregateColumns" : [ "ID", "VALUE" ], + "aggregationFunctions" : [ "COLLECT_LIST(VALUE)" ] + }, + "keyColumnNames" : [ "ID" ], + "selectExpressions" : [ "KSQL_AGG_VARIABLE_0 AS COLLECTED" ], + "internalFormats" : { + "keyFormat" : { + "format" : "KAFKA", + "properties" : { } + }, + "valueFormat" : { + "format" : "PROTOBUF_NOSR", + "properties" : { } + } + } + }, + "formats" : { + "keyFormat" : { + "format" : "KAFKA", + "properties" : { } + }, + "valueFormat" : { + "format" : "PROTOBUF_NOSR", + "properties" : { } + } + }, + "topicName" : "S2", + "timestampColumn" : null + }, + "queryId" : "CTAS_S2_0", + "runtimeId" : null + } + } ], + "configs" : { + "ksql.extension.dir" : "ext", + "ksql.streams.cache.max.bytes.buffering" : "0", + "metric.reporters" : "", + "ksql.query.status.running.threshold.seconds" : "300", + "ksql.connect.basic.auth.credentials.reload" : "false", + "ksql.output.topic.name.prefix" : "", + "ksql.query.pull.stream.enabled" : "true", + "ksql.query.push.v2.interpreter.enabled" : "true", + "ksql.queryanonymizer.logs_enabled" : "true", + "ksql.variable.substitution.enable" : "true", + "ksql.streams.shutdown.timeout.ms" : "300000", + "ksql.query.pull.max.allowed.offset.lag" : "9223372036854775807", + "ksql.query.pull.max.qps" : "2147483647", + "ksql.access.validator.enable" : "auto", + "ksql.streams.bootstrap.servers" : "localhost:0", + "ksql.query.pull.metrics.enabled" : "true", + "ksql.metrics.extension" : null, + "ksql.query.push.v2.alos.enabled" : "true", + "ksql.query.push.v2.max.hourly.bandwidth.megabytes" : "2147483647", + "ksql.query.pull.range.scan.enabled" : "true", + "ksql.transient.query.cleanup.service.initial.delay.seconds" : "600", + "ksql.hidden.topics" : "_confluent.*,__confluent.*,_schemas,__consumer_offsets,__transaction_state,connect-configs,connect-offsets,connect-status,connect-statuses", + "ksql.lambdas.enabled" : "true", + "ksql.source.table.materialization.enabled" : "true", + "ksql.query.pull.max.hourly.bandwidth.megabytes" : "2147483647", + "ksql.sink.window.change.log.additional.retention" : "1000000", + "ksql.query.persistent.active.limit" : "2147483647", + "ksql.persistence.wrap.single.values" : null, + "ksql.query.transient.max.bytes.buffering.total" : "-1", + "ksql.connect.basic.auth.credentials.source" : "NONE", + "ksql.schema.registry.url" : "schema_registry.url:0", + "ksql.properties.overrides.denylist" : "", + "ksql.service.id" : "some.ksql.service.id", + "ksql.query.push.v2.max.catchup.consumers" : "5", + "ksql.query.push.v2.enabled" : "false", + "ksql.transient.query.cleanup.service.enable" : "true", + "ksql.query.push.v2.metrics.enabled" : "true", + "ksql.rowpartition.rowoffset.enabled" : "true", + "ksql.streams.commit.interval.ms" : "2000", + "ksql.query.pull.table.scan.enabled" : "true", + "ksql.streams.auto.commit.interval.ms" : "0", + "ksql.streams.topology.optimization" : "all", + "ksql.endpoint.migrate.query" : "true", + "ksql.query.push.v2.registry.installed" : "false", + "ksql.streams.num.stream.threads" : "4", + "ksql.metrics.tags.custom" : "", + "ksql.query.push.v2.catchup.consumer.msg.window" : "50", + "ksql.runtime.feature.shared.enabled" : "false", + "ksql.udf.collect.metrics" : "false", + "ksql.new.query.planner.enabled" : "false", + "ksql.connect.request.headers.plugin" : null, + "ksql.security.extension.class" : null, + "ksql.transient.prefix" : "transient_", + "ksql.headers.columns.enabled" : "true", + "ksql.streams.default.deserialization.exception.handler" : "io.confluent.ksql.errors.LogMetricAndContinueExceptionHandler", + "ksql.connect.request.timeout.ms" : "5000", + "ksql.query.pull.enable.standby.reads" : "false", + "ksql.persistence.default.format.key" : "KAFKA", + "ksql.query.persistent.max.bytes.buffering.total" : "-1", + "ksql.query.error.max.queue.size" : "10", + "ksql.query.cleanup.shutdown.timeout.ms" : "30000", + "ksql.internal.topic.min.insync.replicas" : "1", + "ksql.internal.topic.replicas" : "1", + "ksql.insert.into.values.enabled" : "true", + "ksql.queryanonymizer.cluster_namespace" : null, + "ksql.create.or.replace.enabled" : "true", + "ksql.shared.runtimes.count" : "2", + "ksql.cast.strings.preserve.nulls" : "true", + "ksql.authorization.cache.max.entries" : "10000", + "ksql.pull.queries.enable" : "true", + "ksql.transient.query.cleanup.service.period.seconds" : "600", + "ksql.suppress.enabled" : "false", + "ksql.readonly.topics" : "_confluent.*,__confluent.*,_schemas,__consumer_offsets,__transaction_state,connect-configs,connect-offsets,connect-status,connect-statuses", + "ksql.connect.basic.auth.credentials.file" : "", + "ksql.authorization.cache.expiry.time.secs" : "30", + "ksql.query.retry.backoff.initial.ms" : "15000", + "ksql.query.pull.max.concurrent.requests" : "2147483647", + "ksql.streams.auto.offset.reset" : "earliest", + "ksql.connect.url" : "http://localhost:8083", + "ksql.query.push.v2.new.latest.delay.ms" : "5000", + "ksql.query.push.v2.latest.reset.age.ms" : "30000", + "ksql.streams.default.production.exception.handler" : "io.confluent.ksql.errors.ProductionExceptionHandlerUtil$LogAndFailProductionExceptionHandler", + "ksql.query.pull.interpreter.enabled" : "true", + "ksql.query.pull.limit.clause.enabled" : "true", + "ksql.query.pull.router.thread.pool.size" : "50", + "ksql.query.push.v2.continuation.tokens.enabled" : "false", + "ksql.query.retry.backoff.max.ms" : "900000", + "ksql.timestamp.throw.on.invalid" : "false", + "ksql.persistence.default.format.value" : null, + "ksql.udfs.enabled" : "true", + "ksql.udf.enable.security.manager" : "true", + "ksql.connect.worker.config" : "", + "ksql.nested.error.set.null" : "true", + "ksql.query.pull.thread.pool.size" : "50", + "ksql.persistent.prefix" : "query_", + "ksql.metastore.backup.location" : "", + "ksql.error.classifier.regex" : "", + "ksql.suppress.buffer.size.bytes" : "-1" + } +} \ No newline at end of file diff --git a/ksqldb-functional-tests/src/test/resources/historical_plans/collect-list_-_collect_list_struct_-_PROTOBUF_NOSR/7.3.0_1651069531802/spec.json b/ksqldb-functional-tests/src/test/resources/historical_plans/collect-list_-_collect_list_struct_-_PROTOBUF_NOSR/7.3.0_1651069531802/spec.json new file mode 100644 index 000000000000..a235ee3db371 --- /dev/null +++ b/ksqldb-functional-tests/src/test/resources/historical_plans/collect-list_-_collect_list_struct_-_PROTOBUF_NOSR/7.3.0_1651069531802/spec.json @@ -0,0 +1,199 @@ +{ + "version" : "7.3.0", + "timestamp" : 1651069531802, + "path" : "query-validation-tests/collect-list.json", + "schemas" : { + "CTAS_S2_0.Aggregate.Aggregate.Materialize" : { + "schema" : "`ID` BIGINT KEY, `ID` BIGINT, `VALUE` STRUCT<`A` STRING, `B` BIGINT>, `KSQL_AGG_VARIABLE_0` ARRAY>", + "keyFormat" : { + "format" : "KAFKA" + }, + "valueFormat" : { + "format" : "PROTOBUF_NOSR" + } + }, + "CTAS_S2_0.Aggregate.Project" : { + "schema" : "`ID` BIGINT KEY, `COLLECTED` ARRAY>", + "keyFormat" : { + "format" : "KAFKA" + }, + "valueFormat" : { + "format" : "PROTOBUF_NOSR" + } + }, + "CTAS_S2_0.KsqlTopic.Source" : { + "schema" : "`ID` BIGINT KEY, `VALUE` STRUCT<`A` STRING, `B` BIGINT>", + "keyFormat" : { + "format" : "KAFKA" + }, + "valueFormat" : { + "format" : "PROTOBUF_NOSR" + } + }, + "CTAS_S2_0.S2" : { + "schema" : "`ID` BIGINT KEY, `COLLECTED` ARRAY>", + "keyFormat" : { + "format" : "KAFKA" + }, + "valueFormat" : { + "format" : "PROTOBUF_NOSR" + } + }, + "CTAS_S2_0.Aggregate.GroupBy" : { + "schema" : "`ID` BIGINT KEY, `ID` BIGINT, `VALUE` STRUCT<`A` STRING, `B` BIGINT>", + "keyFormat" : { + "format" : "KAFKA" + }, + "valueFormat" : { + "format" : "PROTOBUF_NOSR" + } + } + }, + "testCase" : { + "name" : "collect_list struct - PROTOBUF_NOSR", + "inputs" : [ { + "topic" : "test_topic", + "key" : 0, + "value" : { + "VALUE" : { + "A" : "Record0", + "B" : 1 + } + } + }, { + "topic" : "test_topic", + "key" : 0, + "value" : { + "VALUE" : { + "A" : "Record1", + "B" : 100 + } + } + }, { + "topic" : "test_topic", + "key" : 100, + "value" : { + "VALUE" : { + "A" : "Record0", + "B" : 100 + } + } + }, { + "topic" : "test_topic", + "key" : 100, + "value" : { + "VALUE" : { + "A" : "Record0", + "B" : 100 + } + } + } ], + "outputs" : [ { + "topic" : "S2", + "key" : 0, + "value" : { + "COLLECTED" : [ { + "A" : "Record0", + "B" : 1 + } ] + } + }, { + "topic" : "S2", + "key" : 0, + "value" : { + "COLLECTED" : [ { + "A" : "Record0", + "B" : 1 + }, { + "A" : "Record1", + "B" : 100 + } ] + } + }, { + "topic" : "S2", + "key" : 100, + "value" : { + "COLLECTED" : [ { + "A" : "Record0", + "B" : 100 + } ] + } + }, { + "topic" : "S2", + "key" : 100, + "value" : { + "COLLECTED" : [ { + "A" : "Record0", + "B" : 100 + }, { + "A" : "Record0", + "B" : 100 + } ] + } + } ], + "topics" : [ { + "name" : "test_topic", + "replicas" : 1, + "numPartitions" : 4 + }, { + "name" : "S2", + "replicas" : 1, + "numPartitions" : 4 + } ], + "statements" : [ "CREATE STREAM TEST (ID BIGINT KEY, VALUE STRUCT) WITH (kafka_topic='test_topic',value_format='PROTOBUF_NOSR');", "CREATE TABLE S2 as SELECT ID, collect_list(value) as collected FROM test group by id;" ], + "post" : { + "sources" : [ { + "name" : "S2", + "type" : "TABLE", + "schema" : "`ID` BIGINT KEY, `COLLECTED` ARRAY>", + "keyFormat" : { + "format" : "KAFKA" + }, + "valueFormat" : "PROTOBUF_NOSR", + "keyFeatures" : [ ], + "valueFeatures" : [ ], + "isSource" : false + }, { + "name" : "TEST", + "type" : "STREAM", + "schema" : "`ID` BIGINT KEY, `VALUE` STRUCT<`A` STRING, `B` BIGINT>", + "keyFormat" : { + "format" : "KAFKA" + }, + "valueFormat" : "PROTOBUF_NOSR", + "keyFeatures" : [ ], + "valueFeatures" : [ ], + "isSource" : false + } ], + "topics" : { + "topics" : [ { + "name" : "_confluent-ksql-some.ksql.service.idquery_CTAS_S2_0-Aggregate-Aggregate-Materialize-changelog", + "keyFormat" : { + "format" : "KAFKA" + }, + "valueFormat" : { + "format" : "PROTOBUF_NOSR" + } + }, { + "name" : "test_topic", + "keyFormat" : { + "format" : "KAFKA" + }, + "valueFormat" : { + "format" : "PROTOBUF_NOSR" + }, + "partitions" : 4 + }, { + "name" : "S2", + "keyFormat" : { + "format" : "KAFKA" + }, + "valueFormat" : { + "format" : "PROTOBUF_NOSR" + }, + "partitions" : 4 + } ] + } + } + } +} \ No newline at end of file diff --git a/ksqldb-functional-tests/src/test/resources/historical_plans/collect-list_-_collect_list_struct_-_PROTOBUF_NOSR/7.3.0_1651069531802/topology b/ksqldb-functional-tests/src/test/resources/historical_plans/collect-list_-_collect_list_struct_-_PROTOBUF_NOSR/7.3.0_1651069531802/topology new file mode 100644 index 000000000000..fcfc8fb28665 --- /dev/null +++ b/ksqldb-functional-tests/src/test/resources/historical_plans/collect-list_-_collect_list_struct_-_PROTOBUF_NOSR/7.3.0_1651069531802/topology @@ -0,0 +1,25 @@ +Topologies: + Sub-topology: 0 + Source: KSTREAM-SOURCE-0000000000 (topics: [test_topic]) + --> KSTREAM-TRANSFORMVALUES-0000000001 + Processor: KSTREAM-TRANSFORMVALUES-0000000001 (stores: []) + --> Aggregate-Prepare + <-- KSTREAM-SOURCE-0000000000 + Processor: Aggregate-Prepare (stores: []) + --> KSTREAM-AGGREGATE-0000000003 + <-- KSTREAM-TRANSFORMVALUES-0000000001 + Processor: KSTREAM-AGGREGATE-0000000003 (stores: [Aggregate-Aggregate-Materialize]) + --> Aggregate-Aggregate-ToOutputSchema + <-- Aggregate-Prepare + Processor: Aggregate-Aggregate-ToOutputSchema (stores: []) + --> Aggregate-Project + <-- KSTREAM-AGGREGATE-0000000003 + Processor: Aggregate-Project (stores: []) + --> KTABLE-TOSTREAM-0000000006 + <-- Aggregate-Aggregate-ToOutputSchema + Processor: KTABLE-TOSTREAM-0000000006 (stores: []) + --> KSTREAM-SINK-0000000007 + <-- Aggregate-Project + Sink: KSTREAM-SINK-0000000007 (topic: S2) + <-- KTABLE-TOSTREAM-0000000006 + diff --git a/ksqldb-functional-tests/src/test/resources/historical_plans/collect-list_-_collect_list_time_map_-_PROTOBUF_NOSR/7.3.0_1651069548985/plan.json b/ksqldb-functional-tests/src/test/resources/historical_plans/collect-list_-_collect_list_time_map_-_PROTOBUF_NOSR/7.3.0_1651069548985/plan.json new file mode 100644 index 000000000000..f18ad6c3c394 --- /dev/null +++ b/ksqldb-functional-tests/src/test/resources/historical_plans/collect-list_-_collect_list_time_map_-_PROTOBUF_NOSR/7.3.0_1651069548985/plan.json @@ -0,0 +1,255 @@ +{ + "plan" : [ { + "@type" : "ksqlPlanV1", + "statementText" : "CREATE STREAM TEST (ID BIGINT KEY, NAME STRING, VALUE MAP) WITH (KAFKA_TOPIC='test_topic', KEY_FORMAT='KAFKA', VALUE_FORMAT='PROTOBUF_NOSR');", + "ddlCommand" : { + "@type" : "createStreamV1", + "sourceName" : "TEST", + "schema" : "`ID` BIGINT KEY, `NAME` STRING, `VALUE` MAP", + "timestampColumn" : null, + "topicName" : "test_topic", + "formats" : { + "keyFormat" : { + "format" : "KAFKA", + "properties" : { } + }, + "valueFormat" : { + "format" : "PROTOBUF_NOSR", + "properties" : { } + } + }, + "windowInfo" : null, + "orReplace" : false, + "isSource" : false + }, + "queryPlan" : null + }, { + "@type" : "ksqlPlanV1", + "statementText" : "CREATE TABLE S2 AS SELECT\n TEST.ID ID,\n COLLECT_LIST(TEST.VALUE['key1']) COLLECTED\nFROM TEST TEST\nGROUP BY TEST.ID\nEMIT CHANGES", + "ddlCommand" : { + "@type" : "createTableV1", + "sourceName" : "S2", + "schema" : "`ID` BIGINT KEY, `COLLECTED` ARRAY) WITH (KAFKA_TOPIC='test_topic', KEY_FORMAT='KAFKA', VALUE_FORMAT='PROTOBUF_NOSR');", + "ddlCommand" : { + "@type" : "createStreamV1", + "sourceName" : "TEST", + "schema" : "`ID` BIGINT KEY, `VALUE` STRUCT<`A` STRING, `B` BIGINT>", + "timestampColumn" : null, + "topicName" : "test_topic", + "formats" : { + "keyFormat" : { + "format" : "KAFKA", + "properties" : { } + }, + "valueFormat" : { + "format" : "PROTOBUF_NOSR", + "properties" : { } + } + }, + "windowInfo" : null, + "orReplace" : false, + "isSource" : false + }, + "queryPlan" : null + }, { + "@type" : "ksqlPlanV1", + "statementText" : "CREATE TABLE S2 AS SELECT\n TEST.ID ID,\n COLLECT_SET(TEST.VALUE) COLLECTED\nFROM TEST TEST\nGROUP BY TEST.ID\nEMIT CHANGES", + "ddlCommand" : { + "@type" : "createTableV1", + "sourceName" : "S2", + "schema" : "`ID` BIGINT KEY, `COLLECTED` ARRAY>", + "timestampColumn" : null, + "topicName" : "S2", + "formats" : { + "keyFormat" : { + "format" : "KAFKA", + "properties" : { } + }, + "valueFormat" : { + "format" : "PROTOBUF_NOSR", + "properties" : { } + } + }, + "windowInfo" : null, + "orReplace" : false, + "isSource" : false + }, + "queryPlan" : { + "sources" : [ "TEST" ], + "sink" : "S2", + "physicalPlan" : { + "@type" : "tableSinkV1", + "properties" : { + "queryContext" : "S2" + }, + "source" : { + "@type" : "tableSelectV1", + "properties" : { + "queryContext" : "Aggregate/Project" + }, + "source" : { + "@type" : "streamAggregateV1", + "properties" : { + "queryContext" : "Aggregate/Aggregate" + }, + "source" : { + "@type" : "streamGroupByKeyV1", + "properties" : { + "queryContext" : "Aggregate/GroupBy" + }, + "source" : { + "@type" : "streamSelectV1", + "properties" : { + "queryContext" : "Aggregate/Prepare" + }, + "source" : { + "@type" : "streamSourceV1", + "properties" : { + "queryContext" : "KsqlTopic/Source" + }, + "topicName" : "test_topic", + "formats" : { + "keyFormat" : { + "format" : "KAFKA", + "properties" : { } + }, + "valueFormat" : { + "format" : "PROTOBUF_NOSR", + "properties" : { } + } + }, + "timestampColumn" : null, + "sourceSchema" : "`ID` BIGINT KEY, `VALUE` STRUCT<`A` STRING, `B` BIGINT>", + "pseudoColumnVersion" : 1 + }, + "keyColumnNames" : [ "ID" ], + "selectedKeys" : null, + "selectExpressions" : [ "ID AS ID", "VALUE AS VALUE" ] + }, + "internalFormats" : { + "keyFormat" : { + "format" : "KAFKA", + "properties" : { } + }, + "valueFormat" : { + "format" : "PROTOBUF_NOSR", + "properties" : { } + } + } + }, + "internalFormats" : { + "keyFormat" : { + "format" : "KAFKA", + "properties" : { } + }, + "valueFormat" : { + "format" : "PROTOBUF_NOSR", + "properties" : { } + } + }, + "nonAggregateColumns" : [ "ID", "VALUE" ], + "aggregationFunctions" : [ "COLLECT_SET(VALUE)" ] + }, + "keyColumnNames" : [ "ID" ], + "selectExpressions" : [ "KSQL_AGG_VARIABLE_0 AS COLLECTED" ], + "internalFormats" : { + "keyFormat" : { + "format" : "KAFKA", + "properties" : { } + }, + "valueFormat" : { + "format" : "PROTOBUF_NOSR", + "properties" : { } + } + } + }, + "formats" : { + "keyFormat" : { + "format" : "KAFKA", + "properties" : { } + }, + "valueFormat" : { + "format" : "PROTOBUF_NOSR", + "properties" : { } + } + }, + "topicName" : "S2", + "timestampColumn" : null + }, + "queryId" : "CTAS_S2_0", + "runtimeId" : null + } + } ], + "configs" : { + "ksql.extension.dir" : "ext", + "ksql.streams.cache.max.bytes.buffering" : "0", + "metric.reporters" : "", + "ksql.query.status.running.threshold.seconds" : "300", + "ksql.connect.basic.auth.credentials.reload" : "false", + "ksql.output.topic.name.prefix" : "", + "ksql.query.pull.stream.enabled" : "true", + "ksql.query.push.v2.interpreter.enabled" : "true", + "ksql.queryanonymizer.logs_enabled" : "true", + "ksql.variable.substitution.enable" : "true", + "ksql.streams.shutdown.timeout.ms" : "300000", + "ksql.query.pull.max.allowed.offset.lag" : "9223372036854775807", + "ksql.query.pull.max.qps" : "2147483647", + "ksql.access.validator.enable" : "auto", + "ksql.streams.bootstrap.servers" : "localhost:0", + "ksql.query.pull.metrics.enabled" : "true", + "ksql.metrics.extension" : null, + "ksql.query.push.v2.alos.enabled" : "true", + "ksql.query.push.v2.max.hourly.bandwidth.megabytes" : "2147483647", + "ksql.query.pull.range.scan.enabled" : "true", + "ksql.transient.query.cleanup.service.initial.delay.seconds" : "600", + "ksql.hidden.topics" : "_confluent.*,__confluent.*,_schemas,__consumer_offsets,__transaction_state,connect-configs,connect-offsets,connect-status,connect-statuses", + "ksql.lambdas.enabled" : "true", + "ksql.source.table.materialization.enabled" : "true", + "ksql.query.pull.max.hourly.bandwidth.megabytes" : "2147483647", + "ksql.sink.window.change.log.additional.retention" : "1000000", + "ksql.query.persistent.active.limit" : "2147483647", + "ksql.persistence.wrap.single.values" : null, + "ksql.query.transient.max.bytes.buffering.total" : "-1", + "ksql.connect.basic.auth.credentials.source" : "NONE", + "ksql.schema.registry.url" : "schema_registry.url:0", + "ksql.properties.overrides.denylist" : "", + "ksql.service.id" : "some.ksql.service.id", + "ksql.query.push.v2.max.catchup.consumers" : "5", + "ksql.query.push.v2.enabled" : "false", + "ksql.transient.query.cleanup.service.enable" : "true", + "ksql.query.push.v2.metrics.enabled" : "true", + "ksql.rowpartition.rowoffset.enabled" : "true", + "ksql.streams.commit.interval.ms" : "2000", + "ksql.query.pull.table.scan.enabled" : "true", + "ksql.streams.auto.commit.interval.ms" : "0", + "ksql.streams.topology.optimization" : "all", + "ksql.endpoint.migrate.query" : "true", + "ksql.query.push.v2.registry.installed" : "false", + "ksql.streams.num.stream.threads" : "4", + "ksql.metrics.tags.custom" : "", + "ksql.query.push.v2.catchup.consumer.msg.window" : "50", + "ksql.runtime.feature.shared.enabled" : "false", + "ksql.udf.collect.metrics" : "false", + "ksql.new.query.planner.enabled" : "false", + "ksql.connect.request.headers.plugin" : null, + "ksql.security.extension.class" : null, + "ksql.transient.prefix" : "transient_", + "ksql.headers.columns.enabled" : "true", + "ksql.streams.default.deserialization.exception.handler" : "io.confluent.ksql.errors.LogMetricAndContinueExceptionHandler", + "ksql.connect.request.timeout.ms" : "5000", + "ksql.query.pull.enable.standby.reads" : "false", + "ksql.persistence.default.format.key" : "KAFKA", + "ksql.query.persistent.max.bytes.buffering.total" : "-1", + "ksql.query.error.max.queue.size" : "10", + "ksql.query.cleanup.shutdown.timeout.ms" : "30000", + "ksql.internal.topic.min.insync.replicas" : "1", + "ksql.internal.topic.replicas" : "1", + "ksql.insert.into.values.enabled" : "true", + "ksql.queryanonymizer.cluster_namespace" : null, + "ksql.create.or.replace.enabled" : "true", + "ksql.shared.runtimes.count" : "2", + "ksql.cast.strings.preserve.nulls" : "true", + "ksql.authorization.cache.max.entries" : "10000", + "ksql.pull.queries.enable" : "true", + "ksql.transient.query.cleanup.service.period.seconds" : "600", + "ksql.suppress.enabled" : "false", + "ksql.readonly.topics" : "_confluent.*,__confluent.*,_schemas,__consumer_offsets,__transaction_state,connect-configs,connect-offsets,connect-status,connect-statuses", + "ksql.connect.basic.auth.credentials.file" : "", + "ksql.authorization.cache.expiry.time.secs" : "30", + "ksql.query.retry.backoff.initial.ms" : "15000", + "ksql.query.pull.max.concurrent.requests" : "2147483647", + "ksql.streams.auto.offset.reset" : "earliest", + "ksql.connect.url" : "http://localhost:8083", + "ksql.query.push.v2.new.latest.delay.ms" : "5000", + "ksql.query.push.v2.latest.reset.age.ms" : "30000", + "ksql.streams.default.production.exception.handler" : "io.confluent.ksql.errors.ProductionExceptionHandlerUtil$LogAndFailProductionExceptionHandler", + "ksql.query.pull.interpreter.enabled" : "true", + "ksql.query.pull.limit.clause.enabled" : "true", + "ksql.query.pull.router.thread.pool.size" : "50", + "ksql.query.push.v2.continuation.tokens.enabled" : "false", + "ksql.query.retry.backoff.max.ms" : "900000", + "ksql.timestamp.throw.on.invalid" : "false", + "ksql.persistence.default.format.value" : null, + "ksql.udfs.enabled" : "true", + "ksql.udf.enable.security.manager" : "true", + "ksql.connect.worker.config" : "", + "ksql.nested.error.set.null" : "true", + "ksql.query.pull.thread.pool.size" : "50", + "ksql.persistent.prefix" : "query_", + "ksql.metastore.backup.location" : "", + "ksql.error.classifier.regex" : "", + "ksql.suppress.buffer.size.bytes" : "-1" + } +} \ No newline at end of file diff --git a/ksqldb-functional-tests/src/test/resources/historical_plans/collect-set_-_collect_set_struct_-_PROTOBUF_NOSR/7.3.0_1651069603680/spec.json b/ksqldb-functional-tests/src/test/resources/historical_plans/collect-set_-_collect_set_struct_-_PROTOBUF_NOSR/7.3.0_1651069603680/spec.json new file mode 100644 index 000000000000..435c718f1ae7 --- /dev/null +++ b/ksqldb-functional-tests/src/test/resources/historical_plans/collect-set_-_collect_set_struct_-_PROTOBUF_NOSR/7.3.0_1651069603680/spec.json @@ -0,0 +1,196 @@ +{ + "version" : "7.3.0", + "timestamp" : 1651069603680, + "path" : "query-validation-tests/collect-set.json", + "schemas" : { + "CTAS_S2_0.Aggregate.Aggregate.Materialize" : { + "schema" : "`ID` BIGINT KEY, `ID` BIGINT, `VALUE` STRUCT<`A` STRING, `B` BIGINT>, `KSQL_AGG_VARIABLE_0` ARRAY>", + "keyFormat" : { + "format" : "KAFKA" + }, + "valueFormat" : { + "format" : "PROTOBUF_NOSR" + } + }, + "CTAS_S2_0.Aggregate.Project" : { + "schema" : "`ID` BIGINT KEY, `COLLECTED` ARRAY>", + "keyFormat" : { + "format" : "KAFKA" + }, + "valueFormat" : { + "format" : "PROTOBUF_NOSR" + } + }, + "CTAS_S2_0.KsqlTopic.Source" : { + "schema" : "`ID` BIGINT KEY, `VALUE` STRUCT<`A` STRING, `B` BIGINT>", + "keyFormat" : { + "format" : "KAFKA" + }, + "valueFormat" : { + "format" : "PROTOBUF_NOSR" + } + }, + "CTAS_S2_0.S2" : { + "schema" : "`ID` BIGINT KEY, `COLLECTED` ARRAY>", + "keyFormat" : { + "format" : "KAFKA" + }, + "valueFormat" : { + "format" : "PROTOBUF_NOSR" + } + }, + "CTAS_S2_0.Aggregate.GroupBy" : { + "schema" : "`ID` BIGINT KEY, `ID` BIGINT, `VALUE` STRUCT<`A` STRING, `B` BIGINT>", + "keyFormat" : { + "format" : "KAFKA" + }, + "valueFormat" : { + "format" : "PROTOBUF_NOSR" + } + } + }, + "testCase" : { + "name" : "collect_set struct - PROTOBUF_NOSR", + "inputs" : [ { + "topic" : "test_topic", + "key" : 0, + "value" : { + "VALUE" : { + "A" : "Record0", + "B" : 1 + } + } + }, { + "topic" : "test_topic", + "key" : 0, + "value" : { + "VALUE" : { + "A" : "Record1", + "B" : 100 + } + } + }, { + "topic" : "test_topic", + "key" : 100, + "value" : { + "VALUE" : { + "A" : "Record0", + "B" : 100 + } + } + }, { + "topic" : "test_topic", + "key" : 100, + "value" : { + "VALUE" : { + "B" : 100, + "A" : "Record0" + } + } + } ], + "outputs" : [ { + "topic" : "S2", + "key" : 0, + "value" : { + "COLLECTED" : [ { + "A" : "Record0", + "B" : 1 + } ] + } + }, { + "topic" : "S2", + "key" : 0, + "value" : { + "COLLECTED" : [ { + "A" : "Record0", + "B" : 1 + }, { + "A" : "Record1", + "B" : 100 + } ] + } + }, { + "topic" : "S2", + "key" : 100, + "value" : { + "COLLECTED" : [ { + "A" : "Record0", + "B" : 100 + } ] + } + }, { + "topic" : "S2", + "key" : 100, + "value" : { + "COLLECTED" : [ { + "A" : "Record0", + "B" : 100 + } ] + } + } ], + "topics" : [ { + "name" : "test_topic", + "replicas" : 1, + "numPartitions" : 4 + }, { + "name" : "S2", + "replicas" : 1, + "numPartitions" : 4 + } ], + "statements" : [ "CREATE STREAM TEST (ID BIGINT KEY, VALUE STRUCT) WITH (kafka_topic='test_topic',value_format='PROTOBUF_NOSR');", "CREATE TABLE S2 as SELECT ID, collect_set(value) as collected FROM test group by id;" ], + "post" : { + "sources" : [ { + "name" : "S2", + "type" : "TABLE", + "schema" : "`ID` BIGINT KEY, `COLLECTED` ARRAY>", + "keyFormat" : { + "format" : "KAFKA" + }, + "valueFormat" : "PROTOBUF_NOSR", + "keyFeatures" : [ ], + "valueFeatures" : [ ], + "isSource" : false + }, { + "name" : "TEST", + "type" : "STREAM", + "schema" : "`ID` BIGINT KEY, `VALUE` STRUCT<`A` STRING, `B` BIGINT>", + "keyFormat" : { + "format" : "KAFKA" + }, + "valueFormat" : "PROTOBUF_NOSR", + "keyFeatures" : [ ], + "valueFeatures" : [ ], + "isSource" : false + } ], + "topics" : { + "topics" : [ { + "name" : "_confluent-ksql-some.ksql.service.idquery_CTAS_S2_0-Aggregate-Aggregate-Materialize-changelog", + "keyFormat" : { + "format" : "KAFKA" + }, + "valueFormat" : { + "format" : "PROTOBUF_NOSR" + } + }, { + "name" : "test_topic", + "keyFormat" : { + "format" : "KAFKA" + }, + "valueFormat" : { + "format" : "PROTOBUF_NOSR" + }, + "partitions" : 4 + }, { + "name" : "S2", + "keyFormat" : { + "format" : "KAFKA" + }, + "valueFormat" : { + "format" : "PROTOBUF_NOSR" + }, + "partitions" : 4 + } ] + } + } + } +} \ No newline at end of file diff --git a/ksqldb-functional-tests/src/test/resources/historical_plans/collect-set_-_collect_set_struct_-_PROTOBUF_NOSR/7.3.0_1651069603680/topology b/ksqldb-functional-tests/src/test/resources/historical_plans/collect-set_-_collect_set_struct_-_PROTOBUF_NOSR/7.3.0_1651069603680/topology new file mode 100644 index 000000000000..fcfc8fb28665 --- /dev/null +++ b/ksqldb-functional-tests/src/test/resources/historical_plans/collect-set_-_collect_set_struct_-_PROTOBUF_NOSR/7.3.0_1651069603680/topology @@ -0,0 +1,25 @@ +Topologies: + Sub-topology: 0 + Source: KSTREAM-SOURCE-0000000000 (topics: [test_topic]) + --> KSTREAM-TRANSFORMVALUES-0000000001 + Processor: KSTREAM-TRANSFORMVALUES-0000000001 (stores: []) + --> Aggregate-Prepare + <-- KSTREAM-SOURCE-0000000000 + Processor: Aggregate-Prepare (stores: []) + --> KSTREAM-AGGREGATE-0000000003 + <-- KSTREAM-TRANSFORMVALUES-0000000001 + Processor: KSTREAM-AGGREGATE-0000000003 (stores: [Aggregate-Aggregate-Materialize]) + --> Aggregate-Aggregate-ToOutputSchema + <-- Aggregate-Prepare + Processor: Aggregate-Aggregate-ToOutputSchema (stores: []) + --> Aggregate-Project + <-- KSTREAM-AGGREGATE-0000000003 + Processor: Aggregate-Project (stores: []) + --> KTABLE-TOSTREAM-0000000006 + <-- Aggregate-Aggregate-ToOutputSchema + Processor: KTABLE-TOSTREAM-0000000006 (stores: []) + --> KSTREAM-SINK-0000000007 + <-- Aggregate-Project + Sink: KSTREAM-SINK-0000000007 (topic: S2) + <-- KTABLE-TOSTREAM-0000000006 + diff --git a/ksqldb-functional-tests/src/test/resources/historical_plans/date_-_PROTOBUF_NOSR_in_out/7.3.0_1651069641154/plan.json b/ksqldb-functional-tests/src/test/resources/historical_plans/date_-_PROTOBUF_NOSR_in_out/7.3.0_1651069641154/plan.json new file mode 100644 index 000000000000..3607d35351ce --- /dev/null +++ b/ksqldb-functional-tests/src/test/resources/historical_plans/date_-_PROTOBUF_NOSR_in_out/7.3.0_1651069641154/plan.json @@ -0,0 +1,203 @@ +{ + "plan" : [ { + "@type" : "ksqlPlanV1", + "statementText" : "CREATE STREAM TEST (ID STRING KEY, DATE DATE) WITH (KAFKA_TOPIC='test', KEY_FORMAT='KAFKA', VALUE_FORMAT='PROTOBUF_NOSR');", + "ddlCommand" : { + "@type" : "createStreamV1", + "sourceName" : "TEST", + "schema" : "`ID` STRING KEY, `DATE` DATE", + "timestampColumn" : null, + "topicName" : "test", + "formats" : { + "keyFormat" : { + "format" : "KAFKA", + "properties" : { } + }, + "valueFormat" : { + "format" : "PROTOBUF_NOSR", + "properties" : { } + } + }, + "windowInfo" : null, + "orReplace" : false, + "isSource" : false + }, + "queryPlan" : null + }, { + "@type" : "ksqlPlanV1", + "statementText" : "CREATE STREAM TEST2 AS SELECT *\nFROM TEST TEST\nEMIT CHANGES", + "ddlCommand" : { + "@type" : "createStreamV1", + "sourceName" : "TEST2", + "schema" : "`ID` STRING KEY, `DATE` DATE", + "timestampColumn" : null, + "topicName" : "TEST2", + "formats" : { + "keyFormat" : { + "format" : "KAFKA", + "properties" : { } + }, + "valueFormat" : { + "format" : "PROTOBUF_NOSR", + "properties" : { } + } + }, + "windowInfo" : null, + "orReplace" : false, + "isSource" : false + }, + "queryPlan" : { + "sources" : [ "TEST" ], + "sink" : "TEST2", + "physicalPlan" : { + "@type" : "streamSinkV1", + "properties" : { + "queryContext" : "TEST2" + }, + "source" : { + "@type" : "streamSelectV1", + "properties" : { + "queryContext" : "Project" + }, + "source" : { + "@type" : "streamSourceV1", + "properties" : { + "queryContext" : "KsqlTopic/Source" + }, + "topicName" : "test", + "formats" : { + "keyFormat" : { + "format" : "KAFKA", + "properties" : { } + }, + "valueFormat" : { + "format" : "PROTOBUF_NOSR", + "properties" : { } + } + }, + "timestampColumn" : null, + "sourceSchema" : "`ID` STRING KEY, `DATE` DATE", + "pseudoColumnVersion" : 1 + }, + "keyColumnNames" : [ "ID" ], + "selectedKeys" : null, + "selectExpressions" : [ "DATE AS DATE" ] + }, + "formats" : { + "keyFormat" : { + "format" : "KAFKA", + "properties" : { } + }, + "valueFormat" : { + "format" : "PROTOBUF_NOSR", + "properties" : { } + } + }, + "topicName" : "TEST2", + "timestampColumn" : null + }, + "queryId" : "CSAS_TEST2_0", + "runtimeId" : null + } + } ], + "configs" : { + "ksql.extension.dir" : "ext", + "ksql.streams.cache.max.bytes.buffering" : "0", + "metric.reporters" : "", + "ksql.query.status.running.threshold.seconds" : "300", + "ksql.connect.basic.auth.credentials.reload" : "false", + "ksql.output.topic.name.prefix" : "", + "ksql.query.pull.stream.enabled" : "true", + "ksql.query.push.v2.interpreter.enabled" : "true", + "ksql.queryanonymizer.logs_enabled" : "true", + "ksql.variable.substitution.enable" : "true", + "ksql.streams.shutdown.timeout.ms" : "300000", + "ksql.query.pull.max.allowed.offset.lag" : "9223372036854775807", + "ksql.query.pull.max.qps" : "2147483647", + "ksql.access.validator.enable" : "auto", + "ksql.streams.bootstrap.servers" : "localhost:0", + "ksql.query.pull.metrics.enabled" : "true", + "ksql.metrics.extension" : null, + "ksql.query.push.v2.alos.enabled" : "true", + "ksql.query.push.v2.max.hourly.bandwidth.megabytes" : "2147483647", + "ksql.query.pull.range.scan.enabled" : "true", + "ksql.transient.query.cleanup.service.initial.delay.seconds" : "600", + "ksql.hidden.topics" : "_confluent.*,__confluent.*,_schemas,__consumer_offsets,__transaction_state,connect-configs,connect-offsets,connect-status,connect-statuses", + "ksql.lambdas.enabled" : "true", + "ksql.source.table.materialization.enabled" : "true", + "ksql.query.pull.max.hourly.bandwidth.megabytes" : "2147483647", + "ksql.sink.window.change.log.additional.retention" : "1000000", + "ksql.query.persistent.active.limit" : "2147483647", + "ksql.persistence.wrap.single.values" : null, + "ksql.query.transient.max.bytes.buffering.total" : "-1", + "ksql.connect.basic.auth.credentials.source" : "NONE", + "ksql.schema.registry.url" : "schema_registry.url:0", + "ksql.properties.overrides.denylist" : "", + "ksql.service.id" : "some.ksql.service.id", + "ksql.query.push.v2.max.catchup.consumers" : "5", + "ksql.query.push.v2.enabled" : "false", + "ksql.transient.query.cleanup.service.enable" : "true", + "ksql.query.push.v2.metrics.enabled" : "true", + "ksql.rowpartition.rowoffset.enabled" : "true", + "ksql.streams.commit.interval.ms" : "2000", + "ksql.query.pull.table.scan.enabled" : "true", + "ksql.streams.auto.commit.interval.ms" : "0", + "ksql.streams.topology.optimization" : "all", + "ksql.endpoint.migrate.query" : "true", + "ksql.query.push.v2.registry.installed" : "false", + "ksql.streams.num.stream.threads" : "4", + "ksql.metrics.tags.custom" : "", + "ksql.query.push.v2.catchup.consumer.msg.window" : "50", + "ksql.runtime.feature.shared.enabled" : "false", + "ksql.udf.collect.metrics" : "false", + "ksql.new.query.planner.enabled" : "false", + "ksql.connect.request.headers.plugin" : null, + "ksql.security.extension.class" : null, + "ksql.transient.prefix" : "transient_", + "ksql.headers.columns.enabled" : "true", + "ksql.streams.default.deserialization.exception.handler" : "io.confluent.ksql.errors.LogMetricAndContinueExceptionHandler", + "ksql.connect.request.timeout.ms" : "5000", + "ksql.query.pull.enable.standby.reads" : "false", + "ksql.persistence.default.format.key" : "KAFKA", + "ksql.query.persistent.max.bytes.buffering.total" : "-1", + "ksql.query.error.max.queue.size" : "10", + "ksql.query.cleanup.shutdown.timeout.ms" : "30000", + "ksql.internal.topic.min.insync.replicas" : "1", + "ksql.internal.topic.replicas" : "1", + "ksql.insert.into.values.enabled" : "true", + "ksql.queryanonymizer.cluster_namespace" : null, + "ksql.create.or.replace.enabled" : "true", + "ksql.shared.runtimes.count" : "2", + "ksql.cast.strings.preserve.nulls" : "true", + "ksql.authorization.cache.max.entries" : "10000", + "ksql.pull.queries.enable" : "true", + "ksql.transient.query.cleanup.service.period.seconds" : "600", + "ksql.suppress.enabled" : "false", + "ksql.readonly.topics" : "_confluent.*,__confluent.*,_schemas,__consumer_offsets,__transaction_state,connect-configs,connect-offsets,connect-status,connect-statuses", + "ksql.connect.basic.auth.credentials.file" : "", + "ksql.authorization.cache.expiry.time.secs" : "30", + "ksql.query.retry.backoff.initial.ms" : "15000", + "ksql.query.pull.max.concurrent.requests" : "2147483647", + "ksql.streams.auto.offset.reset" : "earliest", + "ksql.connect.url" : "http://localhost:8083", + "ksql.query.push.v2.new.latest.delay.ms" : "5000", + "ksql.query.push.v2.latest.reset.age.ms" : "30000", + "ksql.streams.default.production.exception.handler" : "io.confluent.ksql.errors.ProductionExceptionHandlerUtil$LogAndFailProductionExceptionHandler", + "ksql.query.pull.interpreter.enabled" : "true", + "ksql.query.pull.limit.clause.enabled" : "true", + "ksql.query.pull.router.thread.pool.size" : "50", + "ksql.query.push.v2.continuation.tokens.enabled" : "false", + "ksql.query.retry.backoff.max.ms" : "900000", + "ksql.timestamp.throw.on.invalid" : "false", + "ksql.persistence.default.format.value" : null, + "ksql.udfs.enabled" : "true", + "ksql.udf.enable.security.manager" : "true", + "ksql.connect.worker.config" : "", + "ksql.nested.error.set.null" : "true", + "ksql.query.pull.thread.pool.size" : "50", + "ksql.persistent.prefix" : "query_", + "ksql.metastore.backup.location" : "", + "ksql.error.classifier.regex" : "", + "ksql.suppress.buffer.size.bytes" : "-1" + } +} \ No newline at end of file diff --git a/ksqldb-functional-tests/src/test/resources/historical_plans/date_-_PROTOBUF_NOSR_in_out/7.3.0_1651069641154/spec.json b/ksqldb-functional-tests/src/test/resources/historical_plans/date_-_PROTOBUF_NOSR_in_out/7.3.0_1651069641154/spec.json new file mode 100644 index 000000000000..946f83e3b6d0 --- /dev/null +++ b/ksqldb-functional-tests/src/test/resources/historical_plans/date_-_PROTOBUF_NOSR_in_out/7.3.0_1651069641154/spec.json @@ -0,0 +1,110 @@ +{ + "version" : "7.3.0", + "timestamp" : 1651069641154, + "path" : "query-validation-tests/date.json", + "schemas" : { + "CSAS_TEST2_0.TEST2" : { + "schema" : "`ID` STRING KEY, `DATE` DATE", + "keyFormat" : { + "format" : "KAFKA" + }, + "valueFormat" : { + "format" : "PROTOBUF_NOSR" + } + }, + "CSAS_TEST2_0.KsqlTopic.Source" : { + "schema" : "`ID` STRING KEY, `DATE` DATE", + "keyFormat" : { + "format" : "KAFKA" + }, + "valueFormat" : { + "format" : "PROTOBUF_NOSR" + } + } + }, + "testCase" : { + "name" : "PROTOBUF_NOSR in/out", + "inputs" : [ { + "topic" : "test", + "key" : null, + "value" : { + "date" : 10 + } + }, { + "topic" : "test", + "key" : null, + "value" : { + "date" : -10 + } + } ], + "outputs" : [ { + "topic" : "TEST2", + "key" : null, + "value" : { + "DATE" : 10 + } + }, { + "topic" : "TEST2", + "key" : null, + "value" : { + "DATE" : -10 + } + } ], + "topics" : [ { + "name" : "TEST2", + "replicas" : 1, + "numPartitions" : 4 + }, { + "name" : "test", + "replicas" : 1, + "numPartitions" : 4 + } ], + "statements" : [ "CREATE STREAM TEST (ID STRING KEY, date DATE) WITH (kafka_topic='test', value_format='PROTOBUF_NOSR');", "CREATE STREAM TEST2 AS SELECT * FROM TEST;" ], + "post" : { + "sources" : [ { + "name" : "TEST", + "type" : "STREAM", + "schema" : "`ID` STRING KEY, `DATE` DATE", + "keyFormat" : { + "format" : "KAFKA" + }, + "valueFormat" : "PROTOBUF_NOSR", + "keyFeatures" : [ ], + "valueFeatures" : [ ], + "isSource" : false + }, { + "name" : "TEST2", + "type" : "STREAM", + "schema" : "`ID` STRING KEY, `DATE` DATE", + "keyFormat" : { + "format" : "KAFKA" + }, + "valueFormat" : "PROTOBUF_NOSR", + "keyFeatures" : [ ], + "valueFeatures" : [ ], + "isSource" : false + } ], + "topics" : { + "topics" : [ { + "name" : "test", + "keyFormat" : { + "format" : "KAFKA" + }, + "valueFormat" : { + "format" : "PROTOBUF_NOSR" + }, + "partitions" : 4 + }, { + "name" : "TEST2", + "keyFormat" : { + "format" : "KAFKA" + }, + "valueFormat" : { + "format" : "PROTOBUF_NOSR" + }, + "partitions" : 4 + } ] + } + } + } +} \ No newline at end of file diff --git a/ksqldb-functional-tests/src/test/resources/historical_plans/date_-_PROTOBUF_NOSR_in_out/7.3.0_1651069641154/topology b/ksqldb-functional-tests/src/test/resources/historical_plans/date_-_PROTOBUF_NOSR_in_out/7.3.0_1651069641154/topology new file mode 100644 index 000000000000..e080cd675668 --- /dev/null +++ b/ksqldb-functional-tests/src/test/resources/historical_plans/date_-_PROTOBUF_NOSR_in_out/7.3.0_1651069641154/topology @@ -0,0 +1,13 @@ +Topologies: + Sub-topology: 0 + Source: KSTREAM-SOURCE-0000000000 (topics: [test]) + --> KSTREAM-TRANSFORMVALUES-0000000001 + Processor: KSTREAM-TRANSFORMVALUES-0000000001 (stores: []) + --> Project + <-- KSTREAM-SOURCE-0000000000 + Processor: Project (stores: []) + --> KSTREAM-SINK-0000000003 + <-- KSTREAM-TRANSFORMVALUES-0000000001 + Sink: KSTREAM-SINK-0000000003 (topic: TEST2) + <-- Project + diff --git a/ksqldb-functional-tests/src/test/resources/historical_plans/decimal_-_PROTOBUF_NOSR_in_out/7.3.0_1651069641930/plan.json b/ksqldb-functional-tests/src/test/resources/historical_plans/decimal_-_PROTOBUF_NOSR_in_out/7.3.0_1651069641930/plan.json new file mode 100644 index 000000000000..ec4b12401767 --- /dev/null +++ b/ksqldb-functional-tests/src/test/resources/historical_plans/decimal_-_PROTOBUF_NOSR_in_out/7.3.0_1651069641930/plan.json @@ -0,0 +1,203 @@ +{ + "plan" : [ { + "@type" : "ksqlPlanV1", + "statementText" : "CREATE STREAM TEST (ID STRING KEY, DEC DECIMAL(21, 19)) WITH (KAFKA_TOPIC='test', KEY_FORMAT='KAFKA', VALUE_FORMAT='PROTOBUF_NOSR');", + "ddlCommand" : { + "@type" : "createStreamV1", + "sourceName" : "TEST", + "schema" : "`ID` STRING KEY, `DEC` DECIMAL(21, 19)", + "timestampColumn" : null, + "topicName" : "test", + "formats" : { + "keyFormat" : { + "format" : "KAFKA", + "properties" : { } + }, + "valueFormat" : { + "format" : "PROTOBUF_NOSR", + "properties" : { } + } + }, + "windowInfo" : null, + "orReplace" : false, + "isSource" : false + }, + "queryPlan" : null + }, { + "@type" : "ksqlPlanV1", + "statementText" : "CREATE STREAM TEST2 AS SELECT *\nFROM TEST TEST\nEMIT CHANGES", + "ddlCommand" : { + "@type" : "createStreamV1", + "sourceName" : "TEST2", + "schema" : "`ID` STRING KEY, `DEC` DECIMAL(21, 19)", + "timestampColumn" : null, + "topicName" : "TEST2", + "formats" : { + "keyFormat" : { + "format" : "KAFKA", + "properties" : { } + }, + "valueFormat" : { + "format" : "PROTOBUF_NOSR", + "properties" : { } + } + }, + "windowInfo" : null, + "orReplace" : false, + "isSource" : false + }, + "queryPlan" : { + "sources" : [ "TEST" ], + "sink" : "TEST2", + "physicalPlan" : { + "@type" : "streamSinkV1", + "properties" : { + "queryContext" : "TEST2" + }, + "source" : { + "@type" : "streamSelectV1", + "properties" : { + "queryContext" : "Project" + }, + "source" : { + "@type" : "streamSourceV1", + "properties" : { + "queryContext" : "KsqlTopic/Source" + }, + "topicName" : "test", + "formats" : { + "keyFormat" : { + "format" : "KAFKA", + "properties" : { } + }, + "valueFormat" : { + "format" : "PROTOBUF_NOSR", + "properties" : { } + } + }, + "timestampColumn" : null, + "sourceSchema" : "`ID` STRING KEY, `DEC` DECIMAL(21, 19)", + "pseudoColumnVersion" : 1 + }, + "keyColumnNames" : [ "ID" ], + "selectedKeys" : null, + "selectExpressions" : [ "DEC AS DEC" ] + }, + "formats" : { + "keyFormat" : { + "format" : "KAFKA", + "properties" : { } + }, + "valueFormat" : { + "format" : "PROTOBUF_NOSR", + "properties" : { } + } + }, + "topicName" : "TEST2", + "timestampColumn" : null + }, + "queryId" : "CSAS_TEST2_0", + "runtimeId" : null + } + } ], + "configs" : { + "ksql.extension.dir" : "ext", + "ksql.streams.cache.max.bytes.buffering" : "0", + "metric.reporters" : "", + "ksql.query.status.running.threshold.seconds" : "300", + "ksql.connect.basic.auth.credentials.reload" : "false", + "ksql.output.topic.name.prefix" : "", + "ksql.query.pull.stream.enabled" : "true", + "ksql.query.push.v2.interpreter.enabled" : "true", + "ksql.queryanonymizer.logs_enabled" : "true", + "ksql.variable.substitution.enable" : "true", + "ksql.streams.shutdown.timeout.ms" : "300000", + "ksql.query.pull.max.allowed.offset.lag" : "9223372036854775807", + "ksql.query.pull.max.qps" : "2147483647", + "ksql.access.validator.enable" : "auto", + "ksql.streams.bootstrap.servers" : "localhost:0", + "ksql.query.pull.metrics.enabled" : "true", + "ksql.metrics.extension" : null, + "ksql.query.push.v2.alos.enabled" : "true", + "ksql.query.push.v2.max.hourly.bandwidth.megabytes" : "2147483647", + "ksql.query.pull.range.scan.enabled" : "true", + "ksql.transient.query.cleanup.service.initial.delay.seconds" : "600", + "ksql.hidden.topics" : "_confluent.*,__confluent.*,_schemas,__consumer_offsets,__transaction_state,connect-configs,connect-offsets,connect-status,connect-statuses", + "ksql.lambdas.enabled" : "true", + "ksql.source.table.materialization.enabled" : "true", + "ksql.query.pull.max.hourly.bandwidth.megabytes" : "2147483647", + "ksql.sink.window.change.log.additional.retention" : "1000000", + "ksql.query.persistent.active.limit" : "2147483647", + "ksql.persistence.wrap.single.values" : null, + "ksql.query.transient.max.bytes.buffering.total" : "-1", + "ksql.connect.basic.auth.credentials.source" : "NONE", + "ksql.schema.registry.url" : "schema_registry.url:0", + "ksql.properties.overrides.denylist" : "", + "ksql.service.id" : "some.ksql.service.id", + "ksql.query.push.v2.max.catchup.consumers" : "5", + "ksql.query.push.v2.enabled" : "false", + "ksql.transient.query.cleanup.service.enable" : "true", + "ksql.query.push.v2.metrics.enabled" : "true", + "ksql.rowpartition.rowoffset.enabled" : "true", + "ksql.streams.commit.interval.ms" : "2000", + "ksql.query.pull.table.scan.enabled" : "true", + "ksql.streams.auto.commit.interval.ms" : "0", + "ksql.streams.topology.optimization" : "all", + "ksql.endpoint.migrate.query" : "true", + "ksql.query.push.v2.registry.installed" : "false", + "ksql.streams.num.stream.threads" : "4", + "ksql.metrics.tags.custom" : "", + "ksql.query.push.v2.catchup.consumer.msg.window" : "50", + "ksql.runtime.feature.shared.enabled" : "false", + "ksql.udf.collect.metrics" : "false", + "ksql.new.query.planner.enabled" : "false", + "ksql.connect.request.headers.plugin" : null, + "ksql.security.extension.class" : null, + "ksql.transient.prefix" : "transient_", + "ksql.headers.columns.enabled" : "true", + "ksql.streams.default.deserialization.exception.handler" : "io.confluent.ksql.errors.LogMetricAndContinueExceptionHandler", + "ksql.connect.request.timeout.ms" : "5000", + "ksql.query.pull.enable.standby.reads" : "false", + "ksql.persistence.default.format.key" : "KAFKA", + "ksql.query.persistent.max.bytes.buffering.total" : "-1", + "ksql.query.error.max.queue.size" : "10", + "ksql.query.cleanup.shutdown.timeout.ms" : "30000", + "ksql.internal.topic.min.insync.replicas" : "1", + "ksql.internal.topic.replicas" : "1", + "ksql.insert.into.values.enabled" : "true", + "ksql.queryanonymizer.cluster_namespace" : null, + "ksql.create.or.replace.enabled" : "true", + "ksql.shared.runtimes.count" : "2", + "ksql.cast.strings.preserve.nulls" : "true", + "ksql.authorization.cache.max.entries" : "10000", + "ksql.pull.queries.enable" : "true", + "ksql.transient.query.cleanup.service.period.seconds" : "600", + "ksql.suppress.enabled" : "false", + "ksql.readonly.topics" : "_confluent.*,__confluent.*,_schemas,__consumer_offsets,__transaction_state,connect-configs,connect-offsets,connect-status,connect-statuses", + "ksql.connect.basic.auth.credentials.file" : "", + "ksql.authorization.cache.expiry.time.secs" : "30", + "ksql.query.retry.backoff.initial.ms" : "15000", + "ksql.query.pull.max.concurrent.requests" : "2147483647", + "ksql.streams.auto.offset.reset" : "earliest", + "ksql.connect.url" : "http://localhost:8083", + "ksql.query.push.v2.new.latest.delay.ms" : "5000", + "ksql.query.push.v2.latest.reset.age.ms" : "30000", + "ksql.streams.default.production.exception.handler" : "io.confluent.ksql.errors.ProductionExceptionHandlerUtil$LogAndFailProductionExceptionHandler", + "ksql.query.pull.interpreter.enabled" : "true", + "ksql.query.pull.limit.clause.enabled" : "true", + "ksql.query.pull.router.thread.pool.size" : "50", + "ksql.query.push.v2.continuation.tokens.enabled" : "false", + "ksql.query.retry.backoff.max.ms" : "900000", + "ksql.timestamp.throw.on.invalid" : "false", + "ksql.persistence.default.format.value" : null, + "ksql.udfs.enabled" : "true", + "ksql.udf.enable.security.manager" : "true", + "ksql.connect.worker.config" : "", + "ksql.nested.error.set.null" : "true", + "ksql.query.pull.thread.pool.size" : "50", + "ksql.persistent.prefix" : "query_", + "ksql.metastore.backup.location" : "", + "ksql.error.classifier.regex" : "", + "ksql.suppress.buffer.size.bytes" : "-1" + } +} \ No newline at end of file diff --git a/ksqldb-functional-tests/src/test/resources/historical_plans/decimal_-_PROTOBUF_NOSR_in_out/7.3.0_1651069641930/spec.json b/ksqldb-functional-tests/src/test/resources/historical_plans/decimal_-_PROTOBUF_NOSR_in_out/7.3.0_1651069641930/spec.json new file mode 100644 index 000000000000..af6b3a2a43a6 --- /dev/null +++ b/ksqldb-functional-tests/src/test/resources/historical_plans/decimal_-_PROTOBUF_NOSR_in_out/7.3.0_1651069641930/spec.json @@ -0,0 +1,98 @@ +{ + "version" : "7.3.0", + "timestamp" : 1651069641930, + "path" : "query-validation-tests/decimal.json", + "schemas" : { + "CSAS_TEST2_0.TEST2" : { + "schema" : "`ID` STRING KEY, `DEC` DECIMAL(21, 19)", + "keyFormat" : { + "format" : "KAFKA" + }, + "valueFormat" : { + "format" : "PROTOBUF_NOSR" + } + }, + "CSAS_TEST2_0.KsqlTopic.Source" : { + "schema" : "`ID` STRING KEY, `DEC` DECIMAL(21, 19)", + "keyFormat" : { + "format" : "KAFKA" + }, + "valueFormat" : { + "format" : "PROTOBUF_NOSR" + } + } + }, + "testCase" : { + "name" : "PROTOBUF_NOSR in/out", + "inputs" : [ { + "topic" : "test", + "key" : null, + "value" : { + "DEC" : 10.1234512345123451234 + } + } ], + "outputs" : [ { + "topic" : "TEST2", + "key" : null, + "value" : { + "DEC" : 10.1234512345123451234 + } + } ], + "topics" : [ { + "name" : "TEST2", + "replicas" : 1, + "numPartitions" : 4 + }, { + "name" : "test", + "replicas" : 1, + "numPartitions" : 4 + } ], + "statements" : [ "CREATE STREAM TEST (ID STRING KEY, dec DECIMAL(21,19)) WITH (kafka_topic='test', value_format='PROTOBUF_NOSR');", "CREATE STREAM TEST2 AS SELECT * FROM TEST;" ], + "post" : { + "sources" : [ { + "name" : "TEST", + "type" : "STREAM", + "schema" : "`ID` STRING KEY, `DEC` DECIMAL(21, 19)", + "keyFormat" : { + "format" : "KAFKA" + }, + "valueFormat" : "PROTOBUF_NOSR", + "keyFeatures" : [ ], + "valueFeatures" : [ ], + "isSource" : false + }, { + "name" : "TEST2", + "type" : "STREAM", + "schema" : "`ID` STRING KEY, `DEC` DECIMAL(21, 19)", + "keyFormat" : { + "format" : "KAFKA" + }, + "valueFormat" : "PROTOBUF_NOSR", + "keyFeatures" : [ ], + "valueFeatures" : [ ], + "isSource" : false + } ], + "topics" : { + "topics" : [ { + "name" : "test", + "keyFormat" : { + "format" : "KAFKA" + }, + "valueFormat" : { + "format" : "PROTOBUF_NOSR" + }, + "partitions" : 4 + }, { + "name" : "TEST2", + "keyFormat" : { + "format" : "KAFKA" + }, + "valueFormat" : { + "format" : "PROTOBUF_NOSR" + }, + "partitions" : 4 + } ] + } + } + } +} \ No newline at end of file diff --git a/ksqldb-functional-tests/src/test/resources/historical_plans/decimal_-_PROTOBUF_NOSR_in_out/7.3.0_1651069641930/topology b/ksqldb-functional-tests/src/test/resources/historical_plans/decimal_-_PROTOBUF_NOSR_in_out/7.3.0_1651069641930/topology new file mode 100644 index 000000000000..e080cd675668 --- /dev/null +++ b/ksqldb-functional-tests/src/test/resources/historical_plans/decimal_-_PROTOBUF_NOSR_in_out/7.3.0_1651069641930/topology @@ -0,0 +1,13 @@ +Topologies: + Sub-topology: 0 + Source: KSTREAM-SOURCE-0000000000 (topics: [test]) + --> KSTREAM-TRANSFORMVALUES-0000000001 + Processor: KSTREAM-TRANSFORMVALUES-0000000001 (stores: []) + --> Project + <-- KSTREAM-SOURCE-0000000000 + Processor: Project (stores: []) + --> KSTREAM-SINK-0000000003 + <-- KSTREAM-TRANSFORMVALUES-0000000001 + Sink: KSTREAM-SINK-0000000003 (topic: TEST2) + <-- Project + diff --git a/ksqldb-functional-tests/src/test/resources/historical_plans/decimal_-_PROTOBUF_NOSR_should_not_trim_trailing_zeros/7.3.0_1651069642108/plan.json b/ksqldb-functional-tests/src/test/resources/historical_plans/decimal_-_PROTOBUF_NOSR_should_not_trim_trailing_zeros/7.3.0_1651069642108/plan.json new file mode 100644 index 000000000000..c39f35401695 --- /dev/null +++ b/ksqldb-functional-tests/src/test/resources/historical_plans/decimal_-_PROTOBUF_NOSR_should_not_trim_trailing_zeros/7.3.0_1651069642108/plan.json @@ -0,0 +1,203 @@ +{ + "plan" : [ { + "@type" : "ksqlPlanV1", + "statementText" : "CREATE STREAM INPUT (ID STRING KEY, DEC DECIMAL(6, 4)) WITH (KAFKA_TOPIC='test', KEY_FORMAT='KAFKA', VALUE_FORMAT='PROTOBUF_NOSR');", + "ddlCommand" : { + "@type" : "createStreamV1", + "sourceName" : "INPUT", + "schema" : "`ID` STRING KEY, `DEC` DECIMAL(6, 4)", + "timestampColumn" : null, + "topicName" : "test", + "formats" : { + "keyFormat" : { + "format" : "KAFKA", + "properties" : { } + }, + "valueFormat" : { + "format" : "PROTOBUF_NOSR", + "properties" : { } + } + }, + "windowInfo" : null, + "orReplace" : false, + "isSource" : false + }, + "queryPlan" : null + }, { + "@type" : "ksqlPlanV1", + "statementText" : "CREATE STREAM OUTPUT AS SELECT *\nFROM INPUT INPUT\nEMIT CHANGES", + "ddlCommand" : { + "@type" : "createStreamV1", + "sourceName" : "OUTPUT", + "schema" : "`ID` STRING KEY, `DEC` DECIMAL(6, 4)", + "timestampColumn" : null, + "topicName" : "OUTPUT", + "formats" : { + "keyFormat" : { + "format" : "KAFKA", + "properties" : { } + }, + "valueFormat" : { + "format" : "PROTOBUF_NOSR", + "properties" : { } + } + }, + "windowInfo" : null, + "orReplace" : false, + "isSource" : false + }, + "queryPlan" : { + "sources" : [ "INPUT" ], + "sink" : "OUTPUT", + "physicalPlan" : { + "@type" : "streamSinkV1", + "properties" : { + "queryContext" : "OUTPUT" + }, + "source" : { + "@type" : "streamSelectV1", + "properties" : { + "queryContext" : "Project" + }, + "source" : { + "@type" : "streamSourceV1", + "properties" : { + "queryContext" : "KsqlTopic/Source" + }, + "topicName" : "test", + "formats" : { + "keyFormat" : { + "format" : "KAFKA", + "properties" : { } + }, + "valueFormat" : { + "format" : "PROTOBUF_NOSR", + "properties" : { } + } + }, + "timestampColumn" : null, + "sourceSchema" : "`ID` STRING KEY, `DEC` DECIMAL(6, 4)", + "pseudoColumnVersion" : 1 + }, + "keyColumnNames" : [ "ID" ], + "selectedKeys" : null, + "selectExpressions" : [ "DEC AS DEC" ] + }, + "formats" : { + "keyFormat" : { + "format" : "KAFKA", + "properties" : { } + }, + "valueFormat" : { + "format" : "PROTOBUF_NOSR", + "properties" : { } + } + }, + "topicName" : "OUTPUT", + "timestampColumn" : null + }, + "queryId" : "CSAS_OUTPUT_0", + "runtimeId" : null + } + } ], + "configs" : { + "ksql.extension.dir" : "ext", + "ksql.streams.cache.max.bytes.buffering" : "0", + "metric.reporters" : "", + "ksql.query.status.running.threshold.seconds" : "300", + "ksql.connect.basic.auth.credentials.reload" : "false", + "ksql.output.topic.name.prefix" : "", + "ksql.query.pull.stream.enabled" : "true", + "ksql.query.push.v2.interpreter.enabled" : "true", + "ksql.queryanonymizer.logs_enabled" : "true", + "ksql.variable.substitution.enable" : "true", + "ksql.streams.shutdown.timeout.ms" : "300000", + "ksql.query.pull.max.allowed.offset.lag" : "9223372036854775807", + "ksql.query.pull.max.qps" : "2147483647", + "ksql.access.validator.enable" : "auto", + "ksql.streams.bootstrap.servers" : "localhost:0", + "ksql.query.pull.metrics.enabled" : "true", + "ksql.metrics.extension" : null, + "ksql.query.push.v2.alos.enabled" : "true", + "ksql.query.push.v2.max.hourly.bandwidth.megabytes" : "2147483647", + "ksql.query.pull.range.scan.enabled" : "true", + "ksql.transient.query.cleanup.service.initial.delay.seconds" : "600", + "ksql.hidden.topics" : "_confluent.*,__confluent.*,_schemas,__consumer_offsets,__transaction_state,connect-configs,connect-offsets,connect-status,connect-statuses", + "ksql.lambdas.enabled" : "true", + "ksql.source.table.materialization.enabled" : "true", + "ksql.query.pull.max.hourly.bandwidth.megabytes" : "2147483647", + "ksql.sink.window.change.log.additional.retention" : "1000000", + "ksql.query.persistent.active.limit" : "2147483647", + "ksql.persistence.wrap.single.values" : null, + "ksql.query.transient.max.bytes.buffering.total" : "-1", + "ksql.connect.basic.auth.credentials.source" : "NONE", + "ksql.schema.registry.url" : "schema_registry.url:0", + "ksql.properties.overrides.denylist" : "", + "ksql.service.id" : "some.ksql.service.id", + "ksql.query.push.v2.max.catchup.consumers" : "5", + "ksql.query.push.v2.enabled" : "false", + "ksql.transient.query.cleanup.service.enable" : "true", + "ksql.query.push.v2.metrics.enabled" : "true", + "ksql.rowpartition.rowoffset.enabled" : "true", + "ksql.streams.commit.interval.ms" : "2000", + "ksql.query.pull.table.scan.enabled" : "true", + "ksql.streams.auto.commit.interval.ms" : "0", + "ksql.streams.topology.optimization" : "all", + "ksql.endpoint.migrate.query" : "true", + "ksql.query.push.v2.registry.installed" : "false", + "ksql.streams.num.stream.threads" : "4", + "ksql.metrics.tags.custom" : "", + "ksql.query.push.v2.catchup.consumer.msg.window" : "50", + "ksql.runtime.feature.shared.enabled" : "false", + "ksql.udf.collect.metrics" : "false", + "ksql.new.query.planner.enabled" : "false", + "ksql.connect.request.headers.plugin" : null, + "ksql.security.extension.class" : null, + "ksql.transient.prefix" : "transient_", + "ksql.headers.columns.enabled" : "true", + "ksql.streams.default.deserialization.exception.handler" : "io.confluent.ksql.errors.LogMetricAndContinueExceptionHandler", + "ksql.connect.request.timeout.ms" : "5000", + "ksql.query.pull.enable.standby.reads" : "false", + "ksql.persistence.default.format.key" : "KAFKA", + "ksql.query.persistent.max.bytes.buffering.total" : "-1", + "ksql.query.error.max.queue.size" : "10", + "ksql.query.cleanup.shutdown.timeout.ms" : "30000", + "ksql.internal.topic.min.insync.replicas" : "1", + "ksql.internal.topic.replicas" : "1", + "ksql.insert.into.values.enabled" : "true", + "ksql.queryanonymizer.cluster_namespace" : null, + "ksql.create.or.replace.enabled" : "true", + "ksql.shared.runtimes.count" : "2", + "ksql.cast.strings.preserve.nulls" : "true", + "ksql.authorization.cache.max.entries" : "10000", + "ksql.pull.queries.enable" : "true", + "ksql.transient.query.cleanup.service.period.seconds" : "600", + "ksql.suppress.enabled" : "false", + "ksql.readonly.topics" : "_confluent.*,__confluent.*,_schemas,__consumer_offsets,__transaction_state,connect-configs,connect-offsets,connect-status,connect-statuses", + "ksql.connect.basic.auth.credentials.file" : "", + "ksql.authorization.cache.expiry.time.secs" : "30", + "ksql.query.retry.backoff.initial.ms" : "15000", + "ksql.query.pull.max.concurrent.requests" : "2147483647", + "ksql.streams.auto.offset.reset" : "earliest", + "ksql.connect.url" : "http://localhost:8083", + "ksql.query.push.v2.new.latest.delay.ms" : "5000", + "ksql.query.push.v2.latest.reset.age.ms" : "30000", + "ksql.streams.default.production.exception.handler" : "io.confluent.ksql.errors.ProductionExceptionHandlerUtil$LogAndFailProductionExceptionHandler", + "ksql.query.pull.interpreter.enabled" : "true", + "ksql.query.pull.limit.clause.enabled" : "true", + "ksql.query.pull.router.thread.pool.size" : "50", + "ksql.query.push.v2.continuation.tokens.enabled" : "false", + "ksql.query.retry.backoff.max.ms" : "900000", + "ksql.timestamp.throw.on.invalid" : "false", + "ksql.persistence.default.format.value" : null, + "ksql.udfs.enabled" : "true", + "ksql.udf.enable.security.manager" : "true", + "ksql.connect.worker.config" : "", + "ksql.nested.error.set.null" : "true", + "ksql.query.pull.thread.pool.size" : "50", + "ksql.persistent.prefix" : "query_", + "ksql.metastore.backup.location" : "", + "ksql.error.classifier.regex" : "", + "ksql.suppress.buffer.size.bytes" : "-1" + } +} \ No newline at end of file diff --git a/ksqldb-functional-tests/src/test/resources/historical_plans/decimal_-_PROTOBUF_NOSR_should_not_trim_trailing_zeros/7.3.0_1651069642108/spec.json b/ksqldb-functional-tests/src/test/resources/historical_plans/decimal_-_PROTOBUF_NOSR_should_not_trim_trailing_zeros/7.3.0_1651069642108/spec.json new file mode 100644 index 000000000000..5231f52c9fba --- /dev/null +++ b/ksqldb-functional-tests/src/test/resources/historical_plans/decimal_-_PROTOBUF_NOSR_should_not_trim_trailing_zeros/7.3.0_1651069642108/spec.json @@ -0,0 +1,110 @@ +{ + "version" : "7.3.0", + "timestamp" : 1651069642108, + "path" : "query-validation-tests/decimal.json", + "schemas" : { + "CSAS_OUTPUT_0.KsqlTopic.Source" : { + "schema" : "`ID` STRING KEY, `DEC` DECIMAL(6, 4)", + "keyFormat" : { + "format" : "KAFKA" + }, + "valueFormat" : { + "format" : "PROTOBUF_NOSR" + } + }, + "CSAS_OUTPUT_0.OUTPUT" : { + "schema" : "`ID` STRING KEY, `DEC` DECIMAL(6, 4)", + "keyFormat" : { + "format" : "KAFKA" + }, + "valueFormat" : { + "format" : "PROTOBUF_NOSR" + } + } + }, + "testCase" : { + "name" : "PROTOBUF_NOSR should not trim trailing zeros", + "inputs" : [ { + "topic" : "test", + "key" : null, + "value" : { + "DEC" : 10.0 + } + }, { + "topic" : "test", + "key" : null, + "value" : { + "DEC" : 1.0000 + } + } ], + "outputs" : [ { + "topic" : "OUTPUT", + "key" : null, + "value" : { + "DEC" : 10.0000 + } + }, { + "topic" : "OUTPUT", + "key" : null, + "value" : { + "DEC" : 1.0000 + } + } ], + "topics" : [ { + "name" : "test", + "replicas" : 1, + "numPartitions" : 4 + }, { + "name" : "OUTPUT", + "replicas" : 1, + "numPartitions" : 4 + } ], + "statements" : [ "CREATE STREAM INPUT (ID STRING KEY, dec DECIMAL(6,4)) WITH (kafka_topic='test', value_format='PROTOBUF_NOSR');", "CREATE STREAM OUTPUT AS SELECT * FROM INPUT;" ], + "post" : { + "sources" : [ { + "name" : "INPUT", + "type" : "STREAM", + "schema" : "`ID` STRING KEY, `DEC` DECIMAL(6, 4)", + "keyFormat" : { + "format" : "KAFKA" + }, + "valueFormat" : "PROTOBUF_NOSR", + "keyFeatures" : [ ], + "valueFeatures" : [ ], + "isSource" : false + }, { + "name" : "OUTPUT", + "type" : "STREAM", + "schema" : "`ID` STRING KEY, `DEC` DECIMAL(6, 4)", + "keyFormat" : { + "format" : "KAFKA" + }, + "valueFormat" : "PROTOBUF_NOSR", + "keyFeatures" : [ ], + "valueFeatures" : [ ], + "isSource" : false + } ], + "topics" : { + "topics" : [ { + "name" : "test", + "keyFormat" : { + "format" : "KAFKA" + }, + "valueFormat" : { + "format" : "PROTOBUF_NOSR" + }, + "partitions" : 4 + }, { + "name" : "OUTPUT", + "keyFormat" : { + "format" : "KAFKA" + }, + "valueFormat" : { + "format" : "PROTOBUF_NOSR" + }, + "partitions" : 4 + } ] + } + } + } +} \ No newline at end of file diff --git a/ksqldb-functional-tests/src/test/resources/historical_plans/decimal_-_PROTOBUF_NOSR_should_not_trim_trailing_zeros/7.3.0_1651069642108/topology b/ksqldb-functional-tests/src/test/resources/historical_plans/decimal_-_PROTOBUF_NOSR_should_not_trim_trailing_zeros/7.3.0_1651069642108/topology new file mode 100644 index 000000000000..9396b997ae94 --- /dev/null +++ b/ksqldb-functional-tests/src/test/resources/historical_plans/decimal_-_PROTOBUF_NOSR_should_not_trim_trailing_zeros/7.3.0_1651069642108/topology @@ -0,0 +1,13 @@ +Topologies: + Sub-topology: 0 + Source: KSTREAM-SOURCE-0000000000 (topics: [test]) + --> KSTREAM-TRANSFORMVALUES-0000000001 + Processor: KSTREAM-TRANSFORMVALUES-0000000001 (stores: []) + --> Project + <-- KSTREAM-SOURCE-0000000000 + Processor: Project (stores: []) + --> KSTREAM-SINK-0000000003 + <-- KSTREAM-TRANSFORMVALUES-0000000001 + Sink: KSTREAM-SINK-0000000003 (topic: OUTPUT) + <-- Project + diff --git a/ksqldb-functional-tests/src/test/resources/historical_plans/delimited_-_select_delimited_value_format_into_another_format_-_PROTOBUF_NOSR/7.3.0_1651069643541/plan.json b/ksqldb-functional-tests/src/test/resources/historical_plans/delimited_-_select_delimited_value_format_into_another_format_-_PROTOBUF_NOSR/7.3.0_1651069643541/plan.json new file mode 100644 index 000000000000..d87151508d68 --- /dev/null +++ b/ksqldb-functional-tests/src/test/resources/historical_plans/delimited_-_select_delimited_value_format_into_another_format_-_PROTOBUF_NOSR/7.3.0_1651069643541/plan.json @@ -0,0 +1,207 @@ +{ + "plan" : [ { + "@type" : "ksqlPlanV1", + "statementText" : "CREATE STREAM TEST (K STRING KEY, ID BIGINT, NAME STRING, VALUE INTEGER) WITH (KAFKA_TOPIC='test_topic', KEY_FORMAT='KAFKA', VALUE_DELIMITER=',', VALUE_FORMAT='DELIMITED');", + "ddlCommand" : { + "@type" : "createStreamV1", + "sourceName" : "TEST", + "schema" : "`K` STRING KEY, `ID` BIGINT, `NAME` STRING, `VALUE` INTEGER", + "timestampColumn" : null, + "topicName" : "test_topic", + "formats" : { + "keyFormat" : { + "format" : "KAFKA", + "properties" : { } + }, + "valueFormat" : { + "format" : "DELIMITED", + "properties" : { + "delimiter" : "," + } + } + }, + "windowInfo" : null, + "orReplace" : false, + "isSource" : false + }, + "queryPlan" : null + }, { + "@type" : "ksqlPlanV1", + "statementText" : "CREATE STREAM S2 WITH (VALUE_FORMAT='PROTOBUF_NOSR') AS SELECT\n TEST.K K,\n TEST.ID ID,\n TEST.NAME NAME,\n TEST.VALUE VALUE\nFROM TEST TEST\nEMIT CHANGES", + "ddlCommand" : { + "@type" : "createStreamV1", + "sourceName" : "S2", + "schema" : "`K` STRING KEY, `ID` BIGINT, `NAME` STRING, `VALUE` INTEGER", + "timestampColumn" : null, + "topicName" : "S2", + "formats" : { + "keyFormat" : { + "format" : "KAFKA", + "properties" : { } + }, + "valueFormat" : { + "format" : "PROTOBUF_NOSR", + "properties" : { } + } + }, + "windowInfo" : null, + "orReplace" : false, + "isSource" : false + }, + "queryPlan" : { + "sources" : [ "TEST" ], + "sink" : "S2", + "physicalPlan" : { + "@type" : "streamSinkV1", + "properties" : { + "queryContext" : "S2" + }, + "source" : { + "@type" : "streamSelectV1", + "properties" : { + "queryContext" : "Project" + }, + "source" : { + "@type" : "streamSourceV1", + "properties" : { + "queryContext" : "KsqlTopic/Source" + }, + "topicName" : "test_topic", + "formats" : { + "keyFormat" : { + "format" : "KAFKA", + "properties" : { } + }, + "valueFormat" : { + "format" : "DELIMITED", + "properties" : { + "delimiter" : "," + } + } + }, + "timestampColumn" : null, + "sourceSchema" : "`K` STRING KEY, `ID` BIGINT, `NAME` STRING, `VALUE` INTEGER", + "pseudoColumnVersion" : 1 + }, + "keyColumnNames" : [ "K" ], + "selectedKeys" : null, + "selectExpressions" : [ "ID AS ID", "NAME AS NAME", "VALUE AS VALUE" ] + }, + "formats" : { + "keyFormat" : { + "format" : "KAFKA", + "properties" : { } + }, + "valueFormat" : { + "format" : "PROTOBUF_NOSR", + "properties" : { } + } + }, + "topicName" : "S2", + "timestampColumn" : null + }, + "queryId" : "CSAS_S2_0", + "runtimeId" : null + } + } ], + "configs" : { + "ksql.extension.dir" : "ext", + "ksql.streams.cache.max.bytes.buffering" : "0", + "metric.reporters" : "", + "ksql.query.status.running.threshold.seconds" : "300", + "ksql.connect.basic.auth.credentials.reload" : "false", + "ksql.output.topic.name.prefix" : "", + "ksql.query.pull.stream.enabled" : "true", + "ksql.query.push.v2.interpreter.enabled" : "true", + "ksql.queryanonymizer.logs_enabled" : "true", + "ksql.variable.substitution.enable" : "true", + "ksql.streams.shutdown.timeout.ms" : "300000", + "ksql.query.pull.max.allowed.offset.lag" : "9223372036854775807", + "ksql.query.pull.max.qps" : "2147483647", + "ksql.access.validator.enable" : "auto", + "ksql.streams.bootstrap.servers" : "localhost:0", + "ksql.query.pull.metrics.enabled" : "true", + "ksql.metrics.extension" : null, + "ksql.query.push.v2.alos.enabled" : "true", + "ksql.query.push.v2.max.hourly.bandwidth.megabytes" : "2147483647", + "ksql.query.pull.range.scan.enabled" : "true", + "ksql.transient.query.cleanup.service.initial.delay.seconds" : "600", + "ksql.hidden.topics" : "_confluent.*,__confluent.*,_schemas,__consumer_offsets,__transaction_state,connect-configs,connect-offsets,connect-status,connect-statuses", + "ksql.lambdas.enabled" : "true", + "ksql.source.table.materialization.enabled" : "true", + "ksql.query.pull.max.hourly.bandwidth.megabytes" : "2147483647", + "ksql.sink.window.change.log.additional.retention" : "1000000", + "ksql.query.persistent.active.limit" : "2147483647", + "ksql.persistence.wrap.single.values" : null, + "ksql.query.transient.max.bytes.buffering.total" : "-1", + "ksql.connect.basic.auth.credentials.source" : "NONE", + "ksql.schema.registry.url" : "schema_registry.url:0", + "ksql.properties.overrides.denylist" : "", + "ksql.service.id" : "some.ksql.service.id", + "ksql.query.push.v2.max.catchup.consumers" : "5", + "ksql.query.push.v2.enabled" : "false", + "ksql.transient.query.cleanup.service.enable" : "true", + "ksql.query.push.v2.metrics.enabled" : "true", + "ksql.rowpartition.rowoffset.enabled" : "true", + "ksql.streams.commit.interval.ms" : "2000", + "ksql.query.pull.table.scan.enabled" : "true", + "ksql.streams.auto.commit.interval.ms" : "0", + "ksql.streams.topology.optimization" : "all", + "ksql.endpoint.migrate.query" : "true", + "ksql.query.push.v2.registry.installed" : "false", + "ksql.streams.num.stream.threads" : "4", + "ksql.metrics.tags.custom" : "", + "ksql.query.push.v2.catchup.consumer.msg.window" : "50", + "ksql.runtime.feature.shared.enabled" : "false", + "ksql.udf.collect.metrics" : "false", + "ksql.new.query.planner.enabled" : "false", + "ksql.connect.request.headers.plugin" : null, + "ksql.security.extension.class" : null, + "ksql.transient.prefix" : "transient_", + "ksql.headers.columns.enabled" : "true", + "ksql.streams.default.deserialization.exception.handler" : "io.confluent.ksql.errors.LogMetricAndContinueExceptionHandler", + "ksql.connect.request.timeout.ms" : "5000", + "ksql.query.pull.enable.standby.reads" : "false", + "ksql.persistence.default.format.key" : "KAFKA", + "ksql.query.persistent.max.bytes.buffering.total" : "-1", + "ksql.query.error.max.queue.size" : "10", + "ksql.query.cleanup.shutdown.timeout.ms" : "30000", + "ksql.internal.topic.min.insync.replicas" : "1", + "ksql.internal.topic.replicas" : "1", + "ksql.insert.into.values.enabled" : "true", + "ksql.queryanonymizer.cluster_namespace" : null, + "ksql.create.or.replace.enabled" : "true", + "ksql.shared.runtimes.count" : "2", + "ksql.cast.strings.preserve.nulls" : "true", + "ksql.authorization.cache.max.entries" : "10000", + "ksql.pull.queries.enable" : "true", + "ksql.transient.query.cleanup.service.period.seconds" : "600", + "ksql.suppress.enabled" : "false", + "ksql.readonly.topics" : "_confluent.*,__confluent.*,_schemas,__consumer_offsets,__transaction_state,connect-configs,connect-offsets,connect-status,connect-statuses", + "ksql.connect.basic.auth.credentials.file" : "", + "ksql.authorization.cache.expiry.time.secs" : "30", + "ksql.query.retry.backoff.initial.ms" : "15000", + "ksql.query.pull.max.concurrent.requests" : "2147483647", + "ksql.streams.auto.offset.reset" : "earliest", + "ksql.connect.url" : "http://localhost:8083", + "ksql.query.push.v2.new.latest.delay.ms" : "5000", + "ksql.query.push.v2.latest.reset.age.ms" : "30000", + "ksql.streams.default.production.exception.handler" : "io.confluent.ksql.errors.ProductionExceptionHandlerUtil$LogAndFailProductionExceptionHandler", + "ksql.query.pull.interpreter.enabled" : "true", + "ksql.query.pull.limit.clause.enabled" : "true", + "ksql.query.pull.router.thread.pool.size" : "50", + "ksql.query.push.v2.continuation.tokens.enabled" : "false", + "ksql.query.retry.backoff.max.ms" : "900000", + "ksql.timestamp.throw.on.invalid" : "false", + "ksql.persistence.default.format.value" : null, + "ksql.udfs.enabled" : "true", + "ksql.udf.enable.security.manager" : "true", + "ksql.connect.worker.config" : "", + "ksql.nested.error.set.null" : "true", + "ksql.query.pull.thread.pool.size" : "50", + "ksql.persistent.prefix" : "query_", + "ksql.metastore.backup.location" : "", + "ksql.error.classifier.regex" : "", + "ksql.suppress.buffer.size.bytes" : "-1" + } +} \ No newline at end of file diff --git a/ksqldb-functional-tests/src/test/resources/historical_plans/delimited_-_select_delimited_value_format_into_another_format_-_PROTOBUF_NOSR/7.3.0_1651069643541/spec.json b/ksqldb-functional-tests/src/test/resources/historical_plans/delimited_-_select_delimited_value_format_into_another_format_-_PROTOBUF_NOSR/7.3.0_1651069643541/spec.json new file mode 100644 index 000000000000..3b2989d7cdf4 --- /dev/null +++ b/ksqldb-functional-tests/src/test/resources/historical_plans/delimited_-_select_delimited_value_format_into_another_format_-_PROTOBUF_NOSR/7.3.0_1651069643541/spec.json @@ -0,0 +1,106 @@ +{ + "version" : "7.3.0", + "timestamp" : 1651069643541, + "path" : "query-validation-tests/delimited.json", + "schemas" : { + "CSAS_S2_0.S2" : { + "schema" : "`K` STRING KEY, `ID` BIGINT, `NAME` STRING, `VALUE` INTEGER", + "keyFormat" : { + "format" : "KAFKA" + }, + "valueFormat" : { + "format" : "PROTOBUF_NOSR" + } + }, + "CSAS_S2_0.KsqlTopic.Source" : { + "schema" : "`K` STRING KEY, `ID` BIGINT, `NAME` STRING, `VALUE` INTEGER", + "keyFormat" : { + "format" : "KAFKA" + }, + "valueFormat" : { + "format" : "DELIMITED", + "properties" : { + "delimiter" : "," + } + } + } + }, + "testCase" : { + "name" : "select delimited value_format into another format - PROTOBUF_NOSR", + "inputs" : [ { + "topic" : "test_topic", + "key" : "0", + "value" : "0,zero,0", + "timestamp" : 0 + } ], + "outputs" : [ { + "topic" : "S2", + "key" : "0", + "value" : { + "ID" : 0, + "NAME" : "zero", + "VALUE" : 0 + }, + "timestamp" : 0 + } ], + "topics" : [ { + "name" : "test_topic", + "replicas" : 1, + "numPartitions" : 4 + }, { + "name" : "S2", + "replicas" : 1, + "numPartitions" : 4 + } ], + "statements" : [ "CREATE STREAM TEST (K STRING KEY, ID bigint, NAME varchar, VALUE integer) WITH (kafka_topic='test_topic', value_format='DELIMITED', value_delimiter=',');", "CREATE STREAM S2 WITH(value_format='PROTOBUF_NOSR') as SELECT K, id, name, value FROM test;" ], + "post" : { + "sources" : [ { + "name" : "S2", + "type" : "STREAM", + "schema" : "`K` STRING KEY, `ID` BIGINT, `NAME` STRING, `VALUE` INTEGER", + "keyFormat" : { + "format" : "KAFKA" + }, + "valueFormat" : "PROTOBUF_NOSR", + "keyFeatures" : [ ], + "valueFeatures" : [ ], + "isSource" : false + }, { + "name" : "TEST", + "type" : "STREAM", + "schema" : "`K` STRING KEY, `ID` BIGINT, `NAME` STRING, `VALUE` INTEGER", + "keyFormat" : { + "format" : "KAFKA" + }, + "valueFormat" : "DELIMITED", + "keyFeatures" : [ ], + "valueFeatures" : [ ], + "isSource" : false + } ], + "topics" : { + "topics" : [ { + "name" : "test_topic", + "keyFormat" : { + "format" : "KAFKA" + }, + "valueFormat" : { + "format" : "DELIMITED", + "properties" : { + "delimiter" : "," + } + }, + "partitions" : 4 + }, { + "name" : "S2", + "keyFormat" : { + "format" : "KAFKA" + }, + "valueFormat" : { + "format" : "PROTOBUF_NOSR" + }, + "partitions" : 4 + } ] + } + } + } +} \ No newline at end of file diff --git a/ksqldb-functional-tests/src/test/resources/historical_plans/delimited_-_select_delimited_value_format_into_another_format_-_PROTOBUF_NOSR/7.3.0_1651069643541/topology b/ksqldb-functional-tests/src/test/resources/historical_plans/delimited_-_select_delimited_value_format_into_another_format_-_PROTOBUF_NOSR/7.3.0_1651069643541/topology new file mode 100644 index 000000000000..cdbc02e35648 --- /dev/null +++ b/ksqldb-functional-tests/src/test/resources/historical_plans/delimited_-_select_delimited_value_format_into_another_format_-_PROTOBUF_NOSR/7.3.0_1651069643541/topology @@ -0,0 +1,13 @@ +Topologies: + Sub-topology: 0 + Source: KSTREAM-SOURCE-0000000000 (topics: [test_topic]) + --> KSTREAM-TRANSFORMVALUES-0000000001 + Processor: KSTREAM-TRANSFORMVALUES-0000000001 (stores: []) + --> Project + <-- KSTREAM-SOURCE-0000000000 + Processor: Project (stores: []) + --> KSTREAM-SINK-0000000003 + <-- KSTREAM-TRANSFORMVALUES-0000000001 + Sink: KSTREAM-SINK-0000000003 (topic: S2) + <-- Project + diff --git a/ksqldb-functional-tests/src/test/resources/historical_plans/elements_-_validate_schema_id_and_schema_full_name_from_multiple_schema_definitions_OK_-_PROTOBUF/7.2.0_1650587925186/plan.json b/ksqldb-functional-tests/src/test/resources/historical_plans/elements_-_validate_schema_id_and_schema_full_name_from_multiple_schema_definitions_OK_-_PROTOBUF/7.2.0_1650587925186/plan.json new file mode 100644 index 000000000000..58be7822efc0 --- /dev/null +++ b/ksqldb-functional-tests/src/test/resources/historical_plans/elements_-_validate_schema_id_and_schema_full_name_from_multiple_schema_definitions_OK_-_PROTOBUF/7.2.0_1650587925186/plan.json @@ -0,0 +1,223 @@ +{ + "plan" : [ { + "@type" : "ksqlPlanV1", + "statementText" : "CREATE STREAM INPUT (K1 INTEGER KEY, C1 INTEGER) WITH (FORMAT='PROTOBUF', KAFKA_TOPIC='input', KEY_SCHEMA_FULL_NAME='KeySchema2', KEY_SCHEMA_ID=1, VALUE_SCHEMA_FULL_NAME='ValueSchema2', VALUE_SCHEMA_ID=2);", + "ddlCommand" : { + "@type" : "createStreamV1", + "sourceName" : "INPUT", + "schema" : "`K1` INTEGER KEY, `C1` INTEGER", + "topicName" : "input", + "formats" : { + "keyFormat" : { + "format" : "PROTOBUF", + "properties" : { + "fullSchemaName" : "KeySchema2", + "unwrapPrimitives" : "true", + "schemaId" : "1" + } + }, + "valueFormat" : { + "format" : "PROTOBUF", + "properties" : { + "fullSchemaName" : "ValueSchema2", + "unwrapPrimitives" : "true", + "schemaId" : "2" + } + } + }, + "orReplace" : false, + "isSource" : false + } + }, { + "@type" : "ksqlPlanV1", + "statementText" : "CREATE STREAM OUTPUT AS SELECT *\nFROM INPUT INPUT\nEMIT CHANGES", + "ddlCommand" : { + "@type" : "createStreamV1", + "sourceName" : "OUTPUT", + "schema" : "`K1` INTEGER KEY, `C1` INTEGER", + "topicName" : "OUTPUT", + "formats" : { + "keyFormat" : { + "format" : "PROTOBUF", + "properties" : { + "fullSchemaName" : "KeySchema2", + "unwrapPrimitives" : "true" + } + }, + "valueFormat" : { + "format" : "PROTOBUF", + "properties" : { + "fullSchemaName" : "ValueSchema2", + "unwrapPrimitives" : "true" + } + } + }, + "orReplace" : false, + "isSource" : false + }, + "queryPlan" : { + "sources" : [ "INPUT" ], + "sink" : "OUTPUT", + "physicalPlan" : { + "@type" : "streamSinkV1", + "properties" : { + "queryContext" : "OUTPUT" + }, + "source" : { + "@type" : "streamSelectV1", + "properties" : { + "queryContext" : "Project" + }, + "source" : { + "@type" : "streamSourceV1", + "properties" : { + "queryContext" : "KsqlTopic/Source" + }, + "topicName" : "input", + "formats" : { + "keyFormat" : { + "format" : "PROTOBUF", + "properties" : { + "fullSchemaName" : "KeySchema2", + "unwrapPrimitives" : "true", + "schemaId" : "1" + } + }, + "valueFormat" : { + "format" : "PROTOBUF", + "properties" : { + "fullSchemaName" : "ValueSchema2", + "unwrapPrimitives" : "true", + "schemaId" : "2" + } + } + }, + "sourceSchema" : "`K1` INTEGER KEY, `C1` INTEGER", + "pseudoColumnVersion" : 1 + }, + "keyColumnNames" : [ "K1" ], + "selectExpressions" : [ "C1 AS C1" ] + }, + "formats" : { + "keyFormat" : { + "format" : "PROTOBUF", + "properties" : { + "fullSchemaName" : "KeySchema2", + "unwrapPrimitives" : "true" + } + }, + "valueFormat" : { + "format" : "PROTOBUF", + "properties" : { + "fullSchemaName" : "ValueSchema2", + "unwrapPrimitives" : "true" + } + } + }, + "topicName" : "OUTPUT" + }, + "queryId" : "CSAS_OUTPUT_0" + } + } ], + "configs" : { + "ksql.extension.dir" : "ext", + "ksql.streams.cache.max.bytes.buffering" : "0", + "metric.reporters" : "", + "ksql.query.status.running.threshold.seconds" : "300", + "ksql.connect.basic.auth.credentials.reload" : "false", + "ksql.output.topic.name.prefix" : "", + "ksql.query.pull.stream.enabled" : "true", + "ksql.query.push.v2.interpreter.enabled" : "true", + "ksql.queryanonymizer.logs_enabled" : "true", + "ksql.variable.substitution.enable" : "true", + "ksql.streams.shutdown.timeout.ms" : "300000", + "ksql.query.pull.max.allowed.offset.lag" : "9223372036854775807", + "ksql.query.pull.max.qps" : "2147483647", + "ksql.access.validator.enable" : "auto", + "ksql.streams.bootstrap.servers" : "localhost:0", + "ksql.query.pull.metrics.enabled" : "true", + "ksql.metrics.extension" : null, + "ksql.query.push.v2.alos.enabled" : "true", + "ksql.query.push.v2.max.hourly.bandwidth.megabytes" : "2147483647", + "ksql.query.pull.range.scan.enabled" : "true", + "ksql.transient.query.cleanup.service.initial.delay.seconds" : "600", + "ksql.hidden.topics" : "_confluent.*,__confluent.*,_schemas,__consumer_offsets,__transaction_state,connect-configs,connect-offsets,connect-status,connect-statuses", + "ksql.lambdas.enabled" : "true", + "ksql.source.table.materialization.enabled" : "true", + "ksql.query.pull.max.hourly.bandwidth.megabytes" : "2147483647", + "ksql.sink.window.change.log.additional.retention" : "1000000", + "ksql.query.persistent.active.limit" : "2147483647", + "ksql.persistence.wrap.single.values" : null, + "ksql.query.transient.max.bytes.buffering.total" : "-1", + "ksql.connect.basic.auth.credentials.source" : "NONE", + "ksql.schema.registry.url" : "schema_registry.url:0", + "ksql.properties.overrides.denylist" : "", + "ksql.service.id" : "some.ksql.service.id", + "ksql.query.push.v2.max.catchup.consumers" : "5", + "ksql.query.push.v2.enabled" : "false", + "ksql.transient.query.cleanup.service.enable" : "true", + "ksql.query.push.v2.metrics.enabled" : "true", + "ksql.rowpartition.rowoffset.enabled" : "true", + "ksql.streams.commit.interval.ms" : "2000", + "ksql.query.pull.table.scan.enabled" : "true", + "ksql.streams.auto.commit.interval.ms" : "0", + "ksql.streams.topology.optimization" : "all", + "ksql.endpoint.migrate.query" : "true", + "ksql.query.push.v2.registry.installed" : "false", + "ksql.streams.num.stream.threads" : "4", + "ksql.metrics.tags.custom" : "", + "ksql.query.push.v2.catchup.consumer.msg.window" : "50", + "ksql.runtime.feature.shared.enabled" : "false", + "ksql.udf.collect.metrics" : "false", + "ksql.new.query.planner.enabled" : "false", + "ksql.connect.request.headers.plugin" : null, + "ksql.security.extension.class" : null, + "ksql.transient.prefix" : "transient_", + "ksql.headers.columns.enabled" : "true", + "ksql.streams.default.deserialization.exception.handler" : "io.confluent.ksql.errors.LogMetricAndContinueExceptionHandler", + "ksql.connect.request.timeout.ms" : "5000", + "ksql.query.pull.enable.standby.reads" : "false", + "ksql.persistence.default.format.key" : "KAFKA", + "ksql.query.persistent.max.bytes.buffering.total" : "-1", + "ksql.query.error.max.queue.size" : "10", + "ksql.query.cleanup.shutdown.timeout.ms" : "30000", + "ksql.internal.topic.min.insync.replicas" : "1", + "ksql.internal.topic.replicas" : "1", + "ksql.insert.into.values.enabled" : "true", + "ksql.queryanonymizer.cluster_namespace" : null, + "ksql.create.or.replace.enabled" : "true", + "ksql.shared.runtimes.count" : "2", + "ksql.cast.strings.preserve.nulls" : "true", + "ksql.authorization.cache.max.entries" : "10000", + "ksql.pull.queries.enable" : "true", + "ksql.transient.query.cleanup.service.period.seconds" : "600", + "ksql.suppress.enabled" : "false", + "ksql.readonly.topics" : "_confluent.*,__confluent.*,_schemas,__consumer_offsets,__transaction_state,connect-configs,connect-offsets,connect-status,connect-statuses", + "ksql.connect.basic.auth.credentials.file" : "", + "ksql.authorization.cache.expiry.time.secs" : "30", + "ksql.query.retry.backoff.initial.ms" : "15000", + "ksql.query.pull.max.concurrent.requests" : "2147483647", + "ksql.streams.auto.offset.reset" : "earliest", + "ksql.connect.url" : "http://localhost:8083", + "ksql.query.push.v2.new.latest.delay.ms" : "5000", + "ksql.query.push.v2.latest.reset.age.ms" : "30000", + "ksql.streams.default.production.exception.handler" : "io.confluent.ksql.errors.ProductionExceptionHandlerUtil$LogAndFailProductionExceptionHandler", + "ksql.query.pull.interpreter.enabled" : "true", + "ksql.query.pull.limit.clause.enabled" : "true", + "ksql.connect.error.handler" : null, + "ksql.query.pull.router.thread.pool.size" : "50", + "ksql.query.push.v2.continuation.tokens.enabled" : "false", + "ksql.query.retry.backoff.max.ms" : "900000", + "ksql.timestamp.throw.on.invalid" : "false", + "ksql.persistence.default.format.value" : null, + "ksql.udfs.enabled" : "true", + "ksql.udf.enable.security.manager" : "true", + "ksql.connect.worker.config" : "", + "ksql.nested.error.set.null" : "true", + "ksql.query.pull.thread.pool.size" : "50", + "ksql.persistent.prefix" : "query_", + "ksql.metastore.backup.location" : "", + "ksql.error.classifier.regex" : "", + "ksql.suppress.buffer.size.bytes" : "-1" + } +} \ No newline at end of file diff --git a/ksqldb-functional-tests/src/test/resources/historical_plans/elements_-_validate_schema_id_and_schema_full_name_from_multiple_schema_definitions_OK_-_PROTOBUF/7.2.0_1650587925186/spec.json b/ksqldb-functional-tests/src/test/resources/historical_plans/elements_-_validate_schema_id_and_schema_full_name_from_multiple_schema_definitions_OK_-_PROTOBUF/7.2.0_1650587925186/spec.json new file mode 100644 index 000000000000..1f1a86d7211b --- /dev/null +++ b/ksqldb-functional-tests/src/test/resources/historical_plans/elements_-_validate_schema_id_and_schema_full_name_from_multiple_schema_definitions_OK_-_PROTOBUF/7.2.0_1650587925186/spec.json @@ -0,0 +1,146 @@ +{ + "version" : "7.2.0", + "timestamp" : 1650587925186, + "path" : "query-validation-tests/elements.json", + "schemas" : { + "CSAS_OUTPUT_0.KsqlTopic.Source" : { + "schema" : "`K1` INTEGER KEY, `C1` INTEGER", + "keyFormat" : { + "format" : "PROTOBUF", + "properties" : { + "fullSchemaName" : "KeySchema2", + "unwrapPrimitives" : "true", + "schemaId" : "1" + } + }, + "valueFormat" : { + "format" : "PROTOBUF", + "properties" : { + "fullSchemaName" : "ValueSchema2", + "unwrapPrimitives" : "true", + "schemaId" : "2" + } + } + }, + "CSAS_OUTPUT_0.OUTPUT" : { + "schema" : "`K1` INTEGER KEY, `C1` INTEGER", + "keyFormat" : { + "format" : "PROTOBUF", + "properties" : { + "fullSchemaName" : "KeySchema2", + "unwrapPrimitives" : "true" + } + }, + "valueFormat" : { + "format" : "PROTOBUF", + "properties" : { + "fullSchemaName" : "ValueSchema2", + "unwrapPrimitives" : "true" + } + } + } + }, + "testCase" : { + "name" : "validate schema id and schema full name from multiple schema definitions OK - PROTOBUF", + "inputs" : [ { + "topic" : "input", + "key" : { + "k1" : 42 + }, + "value" : { + "c1" : 4 + } + } ], + "outputs" : [ { + "topic" : "OUTPUT", + "key" : { + "k1" : 42 + }, + "value" : { + "c1" : 4 + } + } ], + "topics" : [ { + "name" : "input", + "keySchema" : "syntax = \"proto3\";\n\nmessage KeySchema1 {\n int32 K0 = 1;\n}\nmessage KeySchema2 {\n int32 K1 = 1;\n}\n", + "valueSchema" : "syntax = \"proto3\";\n\nmessage ValueSchema1 {\n int32 C0 = 1;\n}\nmessage ValueSchema2 {\n int32 C1 = 1;\n}\n", + "keyFormat" : "PROTOBUF", + "valueFormat" : "PROTOBUF", + "replicas" : 1, + "numPartitions" : 1 + }, { + "name" : "OUTPUT", + "replicas" : 1, + "numPartitions" : 1 + } ], + "statements" : [ "CREATE STREAM INPUT WITH (kafka_topic='input', format='PROTOBUF', key_schema_id=1, key_schema_full_name='KeySchema2', value_schema_id=2, value_schema_full_name='ValueSchema2');", "CREATE STREAM OUTPUT AS SELECT * FROM input;" ], + "post" : { + "sources" : [ { + "name" : "INPUT", + "type" : "STREAM", + "schema" : "`K1` INTEGER KEY, `C1` INTEGER", + "keyFormat" : { + "format" : "PROTOBUF" + }, + "valueFormat" : "PROTOBUF", + "keyFeatures" : [ ], + "valueFeatures" : [ ], + "isSource" : false + }, { + "name" : "OUTPUT", + "type" : "STREAM", + "schema" : "`K1` INTEGER KEY, `C1` INTEGER", + "keyFormat" : { + "format" : "PROTOBUF" + }, + "valueFormat" : "PROTOBUF", + "keyFeatures" : [ ], + "valueFeatures" : [ ], + "isSource" : false + } ], + "topics" : { + "topics" : [ { + "name" : "input", + "keyFormat" : { + "format" : "PROTOBUF", + "properties" : { + "fullSchemaName" : "KeySchema2", + "unwrapPrimitives" : "true", + "schemaId" : "1" + } + }, + "valueFormat" : { + "format" : "PROTOBUF", + "properties" : { + "fullSchemaName" : "ValueSchema2", + "unwrapPrimitives" : "true", + "schemaId" : "2" + } + }, + "partitions" : 1, + "keySchema" : "syntax = \"proto3\";\n\nmessage KeySchema2 {\n int32 K1 = 1;\n}\n", + "valueSchema" : "syntax = \"proto3\";\n\nmessage ValueSchema2 {\n int32 C1 = 1;\n}\n" + }, { + "name" : "OUTPUT", + "keyFormat" : { + "format" : "PROTOBUF", + "properties" : { + "fullSchemaName" : "KeySchema2", + "unwrapPrimitives" : "true" + } + }, + "valueFormat" : { + "format" : "PROTOBUF", + "properties" : { + "fullSchemaName" : "ValueSchema2", + "unwrapPrimitives" : "true" + } + }, + "partitions" : 4, + "keySchema" : "syntax = \"proto3\";\n\nmessage KeySchema2 {\n int32 K1 = 1;\n}\n", + "valueSchema" : "syntax = \"proto3\";\n\nmessage ValueSchema2 {\n int32 C1 = 1;\n}\n" + } ] + } + } + } +} \ No newline at end of file diff --git a/ksqldb-functional-tests/src/test/resources/historical_plans/elements_-_validate_schema_id_and_schema_full_name_from_multiple_schema_definitions_OK_-_PROTOBUF/7.2.0_1650587925186/topology b/ksqldb-functional-tests/src/test/resources/historical_plans/elements_-_validate_schema_id_and_schema_full_name_from_multiple_schema_definitions_OK_-_PROTOBUF/7.2.0_1650587925186/topology new file mode 100644 index 000000000000..a70a4e91e301 --- /dev/null +++ b/ksqldb-functional-tests/src/test/resources/historical_plans/elements_-_validate_schema_id_and_schema_full_name_from_multiple_schema_definitions_OK_-_PROTOBUF/7.2.0_1650587925186/topology @@ -0,0 +1,13 @@ +Topologies: + Sub-topology: 0 + Source: KSTREAM-SOURCE-0000000000 (topics: [input]) + --> KSTREAM-TRANSFORMVALUES-0000000001 + Processor: KSTREAM-TRANSFORMVALUES-0000000001 (stores: []) + --> Project + <-- KSTREAM-SOURCE-0000000000 + Processor: Project (stores: []) + --> KSTREAM-SINK-0000000003 + <-- KSTREAM-TRANSFORMVALUES-0000000001 + Sink: KSTREAM-SINK-0000000003 (topic: OUTPUT) + <-- Project + diff --git a/ksqldb-functional-tests/src/test/resources/historical_plans/elements_-_validate_schema_id_and_schema_full_name_from_multiple_schema_definitions_OK_-_PROTOBUF/7.3.0_1650556180294/plan.json b/ksqldb-functional-tests/src/test/resources/historical_plans/elements_-_validate_schema_id_and_schema_full_name_from_multiple_schema_definitions_OK_-_PROTOBUF/7.3.0_1650556180294/plan.json new file mode 100644 index 000000000000..c2f90f7c5399 --- /dev/null +++ b/ksqldb-functional-tests/src/test/resources/historical_plans/elements_-_validate_schema_id_and_schema_full_name_from_multiple_schema_definitions_OK_-_PROTOBUF/7.3.0_1650556180294/plan.json @@ -0,0 +1,231 @@ +{ + "plan" : [ { + "@type" : "ksqlPlanV1", + "statementText" : "CREATE STREAM INPUT (K1 INTEGER KEY, C1 INTEGER) WITH (FORMAT='PROTOBUF', KAFKA_TOPIC='input', KEY_SCHEMA_FULL_NAME='KeySchema2', KEY_SCHEMA_ID=1, VALUE_SCHEMA_FULL_NAME='ValueSchema2', VALUE_SCHEMA_ID=2);", + "ddlCommand" : { + "@type" : "createStreamV1", + "sourceName" : "INPUT", + "schema" : "`K1` INTEGER KEY, `C1` INTEGER", + "timestampColumn" : null, + "topicName" : "input", + "formats" : { + "keyFormat" : { + "format" : "PROTOBUF", + "properties" : { + "fullSchemaName" : "KeySchema2", + "unwrapPrimitives" : "true", + "schemaId" : "1" + } + }, + "valueFormat" : { + "format" : "PROTOBUF", + "properties" : { + "fullSchemaName" : "ValueSchema2", + "unwrapPrimitives" : "true", + "schemaId" : "2" + } + } + }, + "windowInfo" : null, + "orReplace" : false, + "isSource" : false + }, + "queryPlan" : null + }, { + "@type" : "ksqlPlanV1", + "statementText" : "CREATE STREAM OUTPUT AS SELECT *\nFROM INPUT INPUT\nEMIT CHANGES", + "ddlCommand" : { + "@type" : "createStreamV1", + "sourceName" : "OUTPUT", + "schema" : "`K1` INTEGER KEY, `C1` INTEGER", + "timestampColumn" : null, + "topicName" : "OUTPUT", + "formats" : { + "keyFormat" : { + "format" : "PROTOBUF", + "properties" : { + "fullSchemaName" : "KeySchema2", + "unwrapPrimitives" : "true" + } + }, + "valueFormat" : { + "format" : "PROTOBUF", + "properties" : { + "fullSchemaName" : "ValueSchema2", + "unwrapPrimitives" : "true" + } + } + }, + "windowInfo" : null, + "orReplace" : false, + "isSource" : false + }, + "queryPlan" : { + "sources" : [ "INPUT" ], + "sink" : "OUTPUT", + "physicalPlan" : { + "@type" : "streamSinkV1", + "properties" : { + "queryContext" : "OUTPUT" + }, + "source" : { + "@type" : "streamSelectV1", + "properties" : { + "queryContext" : "Project" + }, + "source" : { + "@type" : "streamSourceV1", + "properties" : { + "queryContext" : "KsqlTopic/Source" + }, + "topicName" : "input", + "formats" : { + "keyFormat" : { + "format" : "PROTOBUF", + "properties" : { + "fullSchemaName" : "KeySchema2", + "unwrapPrimitives" : "true", + "schemaId" : "1" + } + }, + "valueFormat" : { + "format" : "PROTOBUF", + "properties" : { + "fullSchemaName" : "ValueSchema2", + "unwrapPrimitives" : "true", + "schemaId" : "2" + } + } + }, + "timestampColumn" : null, + "sourceSchema" : "`K1` INTEGER KEY, `C1` INTEGER", + "pseudoColumnVersion" : 1 + }, + "keyColumnNames" : [ "K1" ], + "selectedKeys" : null, + "selectExpressions" : [ "C1 AS C1" ] + }, + "formats" : { + "keyFormat" : { + "format" : "PROTOBUF", + "properties" : { + "fullSchemaName" : "KeySchema2", + "unwrapPrimitives" : "true" + } + }, + "valueFormat" : { + "format" : "PROTOBUF", + "properties" : { + "fullSchemaName" : "ValueSchema2", + "unwrapPrimitives" : "true" + } + } + }, + "topicName" : "OUTPUT", + "timestampColumn" : null + }, + "queryId" : "CSAS_OUTPUT_0", + "runtimeId" : null + } + } ], + "configs" : { + "ksql.extension.dir" : "ext", + "ksql.streams.cache.max.bytes.buffering" : "0", + "metric.reporters" : "", + "ksql.query.status.running.threshold.seconds" : "300", + "ksql.connect.basic.auth.credentials.reload" : "false", + "ksql.output.topic.name.prefix" : "", + "ksql.query.pull.stream.enabled" : "true", + "ksql.query.push.v2.interpreter.enabled" : "true", + "ksql.queryanonymizer.logs_enabled" : "true", + "ksql.variable.substitution.enable" : "true", + "ksql.streams.shutdown.timeout.ms" : "300000", + "ksql.query.pull.max.allowed.offset.lag" : "9223372036854775807", + "ksql.query.pull.max.qps" : "2147483647", + "ksql.access.validator.enable" : "auto", + "ksql.streams.bootstrap.servers" : "localhost:0", + "ksql.query.pull.metrics.enabled" : "true", + "ksql.metrics.extension" : null, + "ksql.query.push.v2.alos.enabled" : "true", + "ksql.query.push.v2.max.hourly.bandwidth.megabytes" : "2147483647", + "ksql.query.pull.range.scan.enabled" : "true", + "ksql.transient.query.cleanup.service.initial.delay.seconds" : "600", + "ksql.hidden.topics" : "_confluent.*,__confluent.*,_schemas,__consumer_offsets,__transaction_state,connect-configs,connect-offsets,connect-status,connect-statuses", + "ksql.lambdas.enabled" : "true", + "ksql.source.table.materialization.enabled" : "true", + "ksql.query.pull.max.hourly.bandwidth.megabytes" : "2147483647", + "ksql.sink.window.change.log.additional.retention" : "1000000", + "ksql.query.persistent.active.limit" : "2147483647", + "ksql.persistence.wrap.single.values" : null, + "ksql.query.transient.max.bytes.buffering.total" : "-1", + "ksql.connect.basic.auth.credentials.source" : "NONE", + "ksql.schema.registry.url" : "schema_registry.url:0", + "ksql.properties.overrides.denylist" : "", + "ksql.service.id" : "some.ksql.service.id", + "ksql.query.push.v2.max.catchup.consumers" : "5", + "ksql.query.push.v2.enabled" : "false", + "ksql.transient.query.cleanup.service.enable" : "true", + "ksql.query.push.v2.metrics.enabled" : "true", + "ksql.rowpartition.rowoffset.enabled" : "true", + "ksql.streams.commit.interval.ms" : "2000", + "ksql.query.pull.table.scan.enabled" : "true", + "ksql.streams.auto.commit.interval.ms" : "0", + "ksql.streams.topology.optimization" : "all", + "ksql.endpoint.migrate.query" : "true", + "ksql.query.push.v2.registry.installed" : "false", + "ksql.streams.num.stream.threads" : "4", + "ksql.metrics.tags.custom" : "", + "ksql.query.push.v2.catchup.consumer.msg.window" : "50", + "ksql.runtime.feature.shared.enabled" : "false", + "ksql.udf.collect.metrics" : "false", + "ksql.new.query.planner.enabled" : "false", + "ksql.connect.request.headers.plugin" : null, + "ksql.security.extension.class" : null, + "ksql.transient.prefix" : "transient_", + "ksql.headers.columns.enabled" : "true", + "ksql.streams.default.deserialization.exception.handler" : "io.confluent.ksql.errors.LogMetricAndContinueExceptionHandler", + "ksql.connect.request.timeout.ms" : "5000", + "ksql.query.pull.enable.standby.reads" : "false", + "ksql.persistence.default.format.key" : "KAFKA", + "ksql.query.persistent.max.bytes.buffering.total" : "-1", + "ksql.query.error.max.queue.size" : "10", + "ksql.query.cleanup.shutdown.timeout.ms" : "30000", + "ksql.internal.topic.min.insync.replicas" : "1", + "ksql.internal.topic.replicas" : "1", + "ksql.insert.into.values.enabled" : "true", + "ksql.queryanonymizer.cluster_namespace" : null, + "ksql.create.or.replace.enabled" : "true", + "ksql.shared.runtimes.count" : "2", + "ksql.cast.strings.preserve.nulls" : "true", + "ksql.authorization.cache.max.entries" : "10000", + "ksql.pull.queries.enable" : "true", + "ksql.transient.query.cleanup.service.period.seconds" : "600", + "ksql.suppress.enabled" : "false", + "ksql.readonly.topics" : "_confluent.*,__confluent.*,_schemas,__consumer_offsets,__transaction_state,connect-configs,connect-offsets,connect-status,connect-statuses", + "ksql.connect.basic.auth.credentials.file" : "", + "ksql.authorization.cache.expiry.time.secs" : "30", + "ksql.query.retry.backoff.initial.ms" : "15000", + "ksql.query.pull.max.concurrent.requests" : "2147483647", + "ksql.streams.auto.offset.reset" : "earliest", + "ksql.connect.url" : "http://localhost:8083", + "ksql.query.push.v2.new.latest.delay.ms" : "5000", + "ksql.query.push.v2.latest.reset.age.ms" : "30000", + "ksql.streams.default.production.exception.handler" : "io.confluent.ksql.errors.ProductionExceptionHandlerUtil$LogAndFailProductionExceptionHandler", + "ksql.query.pull.interpreter.enabled" : "true", + "ksql.query.pull.limit.clause.enabled" : "true", + "ksql.query.pull.router.thread.pool.size" : "50", + "ksql.query.push.v2.continuation.tokens.enabled" : "false", + "ksql.query.retry.backoff.max.ms" : "900000", + "ksql.timestamp.throw.on.invalid" : "false", + "ksql.persistence.default.format.value" : null, + "ksql.udfs.enabled" : "true", + "ksql.udf.enable.security.manager" : "true", + "ksql.connect.worker.config" : "", + "ksql.nested.error.set.null" : "true", + "ksql.query.pull.thread.pool.size" : "50", + "ksql.persistent.prefix" : "query_", + "ksql.metastore.backup.location" : "", + "ksql.error.classifier.regex" : "", + "ksql.suppress.buffer.size.bytes" : "-1" + } +} \ No newline at end of file diff --git a/ksqldb-functional-tests/src/test/resources/historical_plans/elements_-_validate_schema_id_and_schema_full_name_from_multiple_schema_definitions_OK_-_PROTOBUF/7.3.0_1650556180294/spec.json b/ksqldb-functional-tests/src/test/resources/historical_plans/elements_-_validate_schema_id_and_schema_full_name_from_multiple_schema_definitions_OK_-_PROTOBUF/7.3.0_1650556180294/spec.json new file mode 100644 index 000000000000..b108b4f57a38 --- /dev/null +++ b/ksqldb-functional-tests/src/test/resources/historical_plans/elements_-_validate_schema_id_and_schema_full_name_from_multiple_schema_definitions_OK_-_PROTOBUF/7.3.0_1650556180294/spec.json @@ -0,0 +1,146 @@ +{ + "version" : "7.3.0", + "timestamp" : 1650556180294, + "path" : "query-validation-tests/elements.json", + "schemas" : { + "CSAS_OUTPUT_0.KsqlTopic.Source" : { + "schema" : "`K1` INTEGER KEY, `C1` INTEGER", + "keyFormat" : { + "format" : "PROTOBUF", + "properties" : { + "fullSchemaName" : "KeySchema2", + "unwrapPrimitives" : "true", + "schemaId" : "1" + } + }, + "valueFormat" : { + "format" : "PROTOBUF", + "properties" : { + "fullSchemaName" : "ValueSchema2", + "unwrapPrimitives" : "true", + "schemaId" : "2" + } + } + }, + "CSAS_OUTPUT_0.OUTPUT" : { + "schema" : "`K1` INTEGER KEY, `C1` INTEGER", + "keyFormat" : { + "format" : "PROTOBUF", + "properties" : { + "fullSchemaName" : "KeySchema2", + "unwrapPrimitives" : "true" + } + }, + "valueFormat" : { + "format" : "PROTOBUF", + "properties" : { + "fullSchemaName" : "ValueSchema2", + "unwrapPrimitives" : "true" + } + } + } + }, + "testCase" : { + "name" : "validate schema id and schema full name from multiple schema definitions OK - PROTOBUF", + "inputs" : [ { + "topic" : "input", + "key" : { + "k1" : 42 + }, + "value" : { + "c1" : 4 + } + } ], + "outputs" : [ { + "topic" : "OUTPUT", + "key" : { + "k1" : 42 + }, + "value" : { + "c1" : 4 + } + } ], + "topics" : [ { + "name" : "input", + "keySchema" : "syntax = \"proto3\";\n\nmessage KeySchema1 {\n int32 K0 = 1;\n}\nmessage KeySchema2 {\n int32 K1 = 1;\n}\n", + "valueSchema" : "syntax = \"proto3\";\n\nmessage ValueSchema1 {\n int32 C0 = 1;\n}\nmessage ValueSchema2 {\n int32 C1 = 1;\n}\n", + "keyFormat" : "PROTOBUF", + "valueFormat" : "PROTOBUF", + "replicas" : 1, + "numPartitions" : 1 + }, { + "name" : "OUTPUT", + "replicas" : 1, + "numPartitions" : 1 + } ], + "statements" : [ "CREATE STREAM INPUT WITH (kafka_topic='input', format='PROTOBUF', key_schema_id=1, key_schema_full_name='KeySchema2', value_schema_id=2, value_schema_full_name='ValueSchema2');", "CREATE STREAM OUTPUT AS SELECT * FROM input;" ], + "post" : { + "sources" : [ { + "name" : "INPUT", + "type" : "STREAM", + "schema" : "`K1` INTEGER KEY, `C1` INTEGER", + "keyFormat" : { + "format" : "PROTOBUF" + }, + "valueFormat" : "PROTOBUF", + "keyFeatures" : [ ], + "valueFeatures" : [ ], + "isSource" : false + }, { + "name" : "OUTPUT", + "type" : "STREAM", + "schema" : "`K1` INTEGER KEY, `C1` INTEGER", + "keyFormat" : { + "format" : "PROTOBUF" + }, + "valueFormat" : "PROTOBUF", + "keyFeatures" : [ ], + "valueFeatures" : [ ], + "isSource" : false + } ], + "topics" : { + "topics" : [ { + "name" : "input", + "keyFormat" : { + "format" : "PROTOBUF", + "properties" : { + "fullSchemaName" : "KeySchema2", + "unwrapPrimitives" : "true", + "schemaId" : "1" + } + }, + "valueFormat" : { + "format" : "PROTOBUF", + "properties" : { + "fullSchemaName" : "ValueSchema2", + "unwrapPrimitives" : "true", + "schemaId" : "2" + } + }, + "partitions" : 1, + "keySchema" : "syntax = \"proto3\";\n\nmessage KeySchema2 {\n int32 K1 = 1;\n}\n", + "valueSchema" : "syntax = \"proto3\";\n\nmessage ValueSchema2 {\n int32 C1 = 1;\n}\n" + }, { + "name" : "OUTPUT", + "keyFormat" : { + "format" : "PROTOBUF", + "properties" : { + "fullSchemaName" : "KeySchema2", + "unwrapPrimitives" : "true" + } + }, + "valueFormat" : { + "format" : "PROTOBUF", + "properties" : { + "fullSchemaName" : "ValueSchema2", + "unwrapPrimitives" : "true" + } + }, + "partitions" : 4, + "keySchema" : "syntax = \"proto3\";\n\nmessage KeySchema2 {\n int32 K1 = 1;\n}\n", + "valueSchema" : "syntax = \"proto3\";\n\nmessage ValueSchema2 {\n int32 C1 = 1;\n}\n" + } ] + } + } + } +} \ No newline at end of file diff --git a/ksqldb-functional-tests/src/test/resources/historical_plans/elements_-_validate_schema_id_and_schema_full_name_from_multiple_schema_definitions_OK_-_PROTOBUF/7.3.0_1650556180294/topology b/ksqldb-functional-tests/src/test/resources/historical_plans/elements_-_validate_schema_id_and_schema_full_name_from_multiple_schema_definitions_OK_-_PROTOBUF/7.3.0_1650556180294/topology new file mode 100644 index 000000000000..a70a4e91e301 --- /dev/null +++ b/ksqldb-functional-tests/src/test/resources/historical_plans/elements_-_validate_schema_id_and_schema_full_name_from_multiple_schema_definitions_OK_-_PROTOBUF/7.3.0_1650556180294/topology @@ -0,0 +1,13 @@ +Topologies: + Sub-topology: 0 + Source: KSTREAM-SOURCE-0000000000 (topics: [input]) + --> KSTREAM-TRANSFORMVALUES-0000000001 + Processor: KSTREAM-TRANSFORMVALUES-0000000001 (stores: []) + --> Project + <-- KSTREAM-SOURCE-0000000000 + Processor: Project (stores: []) + --> KSTREAM-SINK-0000000003 + <-- KSTREAM-TRANSFORMVALUES-0000000001 + Sink: KSTREAM-SINK-0000000003 (topic: OUTPUT) + <-- Project + diff --git a/ksqldb-functional-tests/src/test/resources/historical_plans/elements_-_validate_schema_id_without_elements_with_non-default_schema_name_OK_-_PROTOBUF/7.2.0_1650587924557/plan.json b/ksqldb-functional-tests/src/test/resources/historical_plans/elements_-_validate_schema_id_without_elements_with_non-default_schema_name_OK_-_PROTOBUF/7.2.0_1650587924557/plan.json new file mode 100644 index 000000000000..447430400b39 --- /dev/null +++ b/ksqldb-functional-tests/src/test/resources/historical_plans/elements_-_validate_schema_id_without_elements_with_non-default_schema_name_OK_-_PROTOBUF/7.2.0_1650587924557/plan.json @@ -0,0 +1,204 @@ +{ + "plan" : [ { + "@type" : "ksqlPlanV1", + "statementText" : "CREATE STREAM INPUT (`c1` INTEGER) WITH (KAFKA_TOPIC='input', KEY_FORMAT='KAFKA', VALUE_FORMAT='PROTOBUF', VALUE_SCHEMA_FULL_NAME='ValueName', VALUE_SCHEMA_ID=1);", + "ddlCommand" : { + "@type" : "createStreamV1", + "sourceName" : "INPUT", + "schema" : "`c1` INTEGER", + "topicName" : "input", + "formats" : { + "keyFormat" : { + "format" : "KAFKA" + }, + "valueFormat" : { + "format" : "PROTOBUF", + "properties" : { + "fullSchemaName" : "ValueName", + "unwrapPrimitives" : "true", + "schemaId" : "1" + } + } + }, + "orReplace" : false, + "isSource" : false + } + }, { + "@type" : "ksqlPlanV1", + "statementText" : "CREATE STREAM OUTPUT WITH (PARTITIONS=4) AS SELECT *\nFROM INPUT INPUT\nEMIT CHANGES", + "ddlCommand" : { + "@type" : "createStreamV1", + "sourceName" : "OUTPUT", + "schema" : "`c1` INTEGER", + "topicName" : "OUTPUT", + "formats" : { + "keyFormat" : { + "format" : "KAFKA" + }, + "valueFormat" : { + "format" : "PROTOBUF", + "properties" : { + "fullSchemaName" : "ValueName", + "unwrapPrimitives" : "true" + } + } + }, + "orReplace" : false, + "isSource" : false + }, + "queryPlan" : { + "sources" : [ "INPUT" ], + "sink" : "OUTPUT", + "physicalPlan" : { + "@type" : "streamSinkV1", + "properties" : { + "queryContext" : "OUTPUT" + }, + "source" : { + "@type" : "streamSelectV1", + "properties" : { + "queryContext" : "Project" + }, + "source" : { + "@type" : "streamSourceV1", + "properties" : { + "queryContext" : "KsqlTopic/Source" + }, + "topicName" : "input", + "formats" : { + "keyFormat" : { + "format" : "KAFKA" + }, + "valueFormat" : { + "format" : "PROTOBUF", + "properties" : { + "fullSchemaName" : "ValueName", + "unwrapPrimitives" : "true", + "schemaId" : "1" + } + } + }, + "sourceSchema" : "`c1` INTEGER", + "pseudoColumnVersion" : 1 + }, + "selectExpressions" : [ "`c1` AS `c1`" ] + }, + "formats" : { + "keyFormat" : { + "format" : "KAFKA" + }, + "valueFormat" : { + "format" : "PROTOBUF", + "properties" : { + "fullSchemaName" : "ValueName", + "unwrapPrimitives" : "true" + } + } + }, + "topicName" : "OUTPUT" + }, + "queryId" : "CSAS_OUTPUT_0" + } + } ], + "configs" : { + "ksql.extension.dir" : "ext", + "ksql.streams.cache.max.bytes.buffering" : "0", + "metric.reporters" : "", + "ksql.query.status.running.threshold.seconds" : "300", + "ksql.connect.basic.auth.credentials.reload" : "false", + "ksql.output.topic.name.prefix" : "", + "ksql.query.pull.stream.enabled" : "true", + "ksql.query.push.v2.interpreter.enabled" : "true", + "ksql.queryanonymizer.logs_enabled" : "true", + "ksql.variable.substitution.enable" : "true", + "ksql.streams.shutdown.timeout.ms" : "300000", + "ksql.query.pull.max.allowed.offset.lag" : "9223372036854775807", + "ksql.query.pull.max.qps" : "2147483647", + "ksql.access.validator.enable" : "auto", + "ksql.streams.bootstrap.servers" : "localhost:0", + "ksql.query.pull.metrics.enabled" : "true", + "ksql.metrics.extension" : null, + "ksql.query.push.v2.alos.enabled" : "true", + "ksql.query.push.v2.max.hourly.bandwidth.megabytes" : "2147483647", + "ksql.query.pull.range.scan.enabled" : "true", + "ksql.transient.query.cleanup.service.initial.delay.seconds" : "600", + "ksql.hidden.topics" : "_confluent.*,__confluent.*,_schemas,__consumer_offsets,__transaction_state,connect-configs,connect-offsets,connect-status,connect-statuses", + "ksql.lambdas.enabled" : "true", + "ksql.source.table.materialization.enabled" : "true", + "ksql.query.pull.max.hourly.bandwidth.megabytes" : "2147483647", + "ksql.sink.window.change.log.additional.retention" : "1000000", + "ksql.query.persistent.active.limit" : "2147483647", + "ksql.persistence.wrap.single.values" : null, + "ksql.query.transient.max.bytes.buffering.total" : "-1", + "ksql.connect.basic.auth.credentials.source" : "NONE", + "ksql.schema.registry.url" : "schema_registry.url:0", + "ksql.properties.overrides.denylist" : "", + "ksql.service.id" : "some.ksql.service.id", + "ksql.query.push.v2.max.catchup.consumers" : "5", + "ksql.query.push.v2.enabled" : "false", + "ksql.transient.query.cleanup.service.enable" : "true", + "ksql.query.push.v2.metrics.enabled" : "true", + "ksql.rowpartition.rowoffset.enabled" : "true", + "ksql.streams.commit.interval.ms" : "2000", + "ksql.query.pull.table.scan.enabled" : "true", + "ksql.streams.auto.commit.interval.ms" : "0", + "ksql.streams.topology.optimization" : "all", + "ksql.endpoint.migrate.query" : "true", + "ksql.query.push.v2.registry.installed" : "false", + "ksql.streams.num.stream.threads" : "4", + "ksql.metrics.tags.custom" : "", + "ksql.query.push.v2.catchup.consumer.msg.window" : "50", + "ksql.runtime.feature.shared.enabled" : "false", + "ksql.udf.collect.metrics" : "false", + "ksql.new.query.planner.enabled" : "false", + "ksql.connect.request.headers.plugin" : null, + "ksql.security.extension.class" : null, + "ksql.transient.prefix" : "transient_", + "ksql.headers.columns.enabled" : "true", + "ksql.streams.default.deserialization.exception.handler" : "io.confluent.ksql.errors.LogMetricAndContinueExceptionHandler", + "ksql.connect.request.timeout.ms" : "5000", + "ksql.query.pull.enable.standby.reads" : "false", + "ksql.persistence.default.format.key" : "KAFKA", + "ksql.query.persistent.max.bytes.buffering.total" : "-1", + "ksql.query.error.max.queue.size" : "10", + "ksql.query.cleanup.shutdown.timeout.ms" : "30000", + "ksql.internal.topic.min.insync.replicas" : "1", + "ksql.internal.topic.replicas" : "1", + "ksql.insert.into.values.enabled" : "true", + "ksql.queryanonymizer.cluster_namespace" : null, + "ksql.create.or.replace.enabled" : "true", + "ksql.shared.runtimes.count" : "2", + "ksql.cast.strings.preserve.nulls" : "true", + "ksql.authorization.cache.max.entries" : "10000", + "ksql.pull.queries.enable" : "true", + "ksql.transient.query.cleanup.service.period.seconds" : "600", + "ksql.suppress.enabled" : "false", + "ksql.readonly.topics" : "_confluent.*,__confluent.*,_schemas,__consumer_offsets,__transaction_state,connect-configs,connect-offsets,connect-status,connect-statuses", + "ksql.connect.basic.auth.credentials.file" : "", + "ksql.authorization.cache.expiry.time.secs" : "30", + "ksql.query.retry.backoff.initial.ms" : "15000", + "ksql.query.pull.max.concurrent.requests" : "2147483647", + "ksql.streams.auto.offset.reset" : "earliest", + "ksql.connect.url" : "http://localhost:8083", + "ksql.query.push.v2.new.latest.delay.ms" : "5000", + "ksql.query.push.v2.latest.reset.age.ms" : "30000", + "ksql.streams.default.production.exception.handler" : "io.confluent.ksql.errors.ProductionExceptionHandlerUtil$LogAndFailProductionExceptionHandler", + "ksql.query.pull.interpreter.enabled" : "true", + "ksql.query.pull.limit.clause.enabled" : "true", + "ksql.connect.error.handler" : null, + "ksql.query.pull.router.thread.pool.size" : "50", + "ksql.query.push.v2.continuation.tokens.enabled" : "false", + "ksql.query.retry.backoff.max.ms" : "900000", + "ksql.timestamp.throw.on.invalid" : "false", + "ksql.persistence.default.format.value" : null, + "ksql.udfs.enabled" : "true", + "ksql.udf.enable.security.manager" : "true", + "ksql.connect.worker.config" : "", + "ksql.nested.error.set.null" : "true", + "ksql.query.pull.thread.pool.size" : "50", + "ksql.persistent.prefix" : "query_", + "ksql.metastore.backup.location" : "", + "ksql.error.classifier.regex" : "", + "ksql.suppress.buffer.size.bytes" : "-1" + } +} \ No newline at end of file diff --git a/ksqldb-functional-tests/src/test/resources/historical_plans/elements_-_validate_schema_id_without_elements_with_non-default_schema_name_OK_-_PROTOBUF/7.2.0_1650587924557/spec.json b/ksqldb-functional-tests/src/test/resources/historical_plans/elements_-_validate_schema_id_without_elements_with_non-default_schema_name_OK_-_PROTOBUF/7.2.0_1650587924557/spec.json new file mode 100644 index 000000000000..8c1ef7dafdba --- /dev/null +++ b/ksqldb-functional-tests/src/test/resources/historical_plans/elements_-_validate_schema_id_without_elements_with_non-default_schema_name_OK_-_PROTOBUF/7.2.0_1650587924557/spec.json @@ -0,0 +1,120 @@ +{ + "version" : "7.2.0", + "timestamp" : 1650587924557, + "path" : "query-validation-tests/elements.json", + "schemas" : { + "CSAS_OUTPUT_0.KsqlTopic.Source" : { + "schema" : "`c1` INTEGER", + "keyFormat" : { + "format" : "KAFKA" + }, + "valueFormat" : { + "format" : "PROTOBUF", + "properties" : { + "fullSchemaName" : "ValueName", + "unwrapPrimitives" : "true", + "schemaId" : "1" + } + } + }, + "CSAS_OUTPUT_0.OUTPUT" : { + "schema" : "`c1` INTEGER", + "keyFormat" : { + "format" : "KAFKA" + }, + "valueFormat" : { + "format" : "PROTOBUF", + "properties" : { + "fullSchemaName" : "ValueName", + "unwrapPrimitives" : "true" + } + } + } + }, + "testCase" : { + "name" : "validate schema id without elements with non-default schema name OK - PROTOBUF", + "inputs" : [ { + "topic" : "input", + "key" : null, + "value" : { + "c1" : 4 + } + } ], + "outputs" : [ { + "topic" : "OUTPUT", + "key" : null, + "value" : { + "c1" : 4 + } + } ], + "topics" : [ { + "name" : "input", + "valueSchema" : "syntax = \"proto3\";\n\nmessage ValueName {\n int32 c1 = 1;\n}\n", + "valueFormat" : "PROTOBUF", + "replicas" : 1, + "numPartitions" : 1 + }, { + "name" : "OUTPUT", + "replicas" : 1, + "numPartitions" : 4 + } ], + "statements" : [ "CREATE STREAM INPUT WITH (kafka_topic='input', value_format='PROTOBUF', value_schema_id=1);", "CREATE STREAM OUTPUT WITH(PARTITIONS = 4) as SELECT * FROM input;" ], + "post" : { + "sources" : [ { + "name" : "INPUT", + "type" : "STREAM", + "schema" : "`c1` INTEGER", + "keyFormat" : { + "format" : "KAFKA" + }, + "valueFormat" : "PROTOBUF", + "keyFeatures" : [ ], + "valueFeatures" : [ ], + "isSource" : false + }, { + "name" : "OUTPUT", + "type" : "STREAM", + "schema" : "`c1` INTEGER", + "keyFormat" : { + "format" : "KAFKA" + }, + "valueFormat" : "PROTOBUF", + "keyFeatures" : [ ], + "valueFeatures" : [ ], + "isSource" : false + } ], + "topics" : { + "topics" : [ { + "name" : "input", + "keyFormat" : { + "format" : "KAFKA" + }, + "valueFormat" : { + "format" : "PROTOBUF", + "properties" : { + "fullSchemaName" : "ValueName", + "unwrapPrimitives" : "true", + "schemaId" : "1" + } + }, + "partitions" : 1, + "valueSchema" : "syntax = \"proto3\";\n\nmessage ValueName {\n int32 c1 = 1;\n}\n" + }, { + "name" : "OUTPUT", + "keyFormat" : { + "format" : "KAFKA" + }, + "valueFormat" : { + "format" : "PROTOBUF", + "properties" : { + "fullSchemaName" : "ValueName", + "unwrapPrimitives" : "true" + } + }, + "partitions" : 4, + "valueSchema" : "syntax = \"proto3\";\n\nmessage ValueName {\n int32 c1 = 1;\n}\n" + } ] + } + } + } +} \ No newline at end of file diff --git a/ksqldb-functional-tests/src/test/resources/historical_plans/elements_-_validate_schema_id_without_elements_with_non-default_schema_name_OK_-_PROTOBUF/7.2.0_1650587924557/topology b/ksqldb-functional-tests/src/test/resources/historical_plans/elements_-_validate_schema_id_without_elements_with_non-default_schema_name_OK_-_PROTOBUF/7.2.0_1650587924557/topology new file mode 100644 index 000000000000..a70a4e91e301 --- /dev/null +++ b/ksqldb-functional-tests/src/test/resources/historical_plans/elements_-_validate_schema_id_without_elements_with_non-default_schema_name_OK_-_PROTOBUF/7.2.0_1650587924557/topology @@ -0,0 +1,13 @@ +Topologies: + Sub-topology: 0 + Source: KSTREAM-SOURCE-0000000000 (topics: [input]) + --> KSTREAM-TRANSFORMVALUES-0000000001 + Processor: KSTREAM-TRANSFORMVALUES-0000000001 (stores: []) + --> Project + <-- KSTREAM-SOURCE-0000000000 + Processor: Project (stores: []) + --> KSTREAM-SINK-0000000003 + <-- KSTREAM-TRANSFORMVALUES-0000000001 + Sink: KSTREAM-SINK-0000000003 (topic: OUTPUT) + <-- Project + diff --git a/ksqldb-functional-tests/src/test/resources/historical_plans/elements_-_validate_schema_id_without_elements_with_non-default_schema_name_OK_-_PROTOBUF/7.3.0_1650556179734/plan.json b/ksqldb-functional-tests/src/test/resources/historical_plans/elements_-_validate_schema_id_without_elements_with_non-default_schema_name_OK_-_PROTOBUF/7.3.0_1650556179734/plan.json new file mode 100644 index 000000000000..26a36e01405b --- /dev/null +++ b/ksqldb-functional-tests/src/test/resources/historical_plans/elements_-_validate_schema_id_without_elements_with_non-default_schema_name_OK_-_PROTOBUF/7.3.0_1650556179734/plan.json @@ -0,0 +1,217 @@ +{ + "plan" : [ { + "@type" : "ksqlPlanV1", + "statementText" : "CREATE STREAM INPUT (`c1` INTEGER) WITH (KAFKA_TOPIC='input', KEY_FORMAT='KAFKA', VALUE_FORMAT='PROTOBUF', VALUE_SCHEMA_FULL_NAME='ValueName', VALUE_SCHEMA_ID=1);", + "ddlCommand" : { + "@type" : "createStreamV1", + "sourceName" : "INPUT", + "schema" : "`c1` INTEGER", + "timestampColumn" : null, + "topicName" : "input", + "formats" : { + "keyFormat" : { + "format" : "KAFKA", + "properties" : { } + }, + "valueFormat" : { + "format" : "PROTOBUF", + "properties" : { + "fullSchemaName" : "ValueName", + "unwrapPrimitives" : "true", + "schemaId" : "1" + } + } + }, + "windowInfo" : null, + "orReplace" : false, + "isSource" : false + }, + "queryPlan" : null + }, { + "@type" : "ksqlPlanV1", + "statementText" : "CREATE STREAM OUTPUT WITH (PARTITIONS=4) AS SELECT *\nFROM INPUT INPUT\nEMIT CHANGES", + "ddlCommand" : { + "@type" : "createStreamV1", + "sourceName" : "OUTPUT", + "schema" : "`c1` INTEGER", + "timestampColumn" : null, + "topicName" : "OUTPUT", + "formats" : { + "keyFormat" : { + "format" : "KAFKA", + "properties" : { } + }, + "valueFormat" : { + "format" : "PROTOBUF", + "properties" : { + "fullSchemaName" : "ValueName", + "unwrapPrimitives" : "true" + } + } + }, + "windowInfo" : null, + "orReplace" : false, + "isSource" : false + }, + "queryPlan" : { + "sources" : [ "INPUT" ], + "sink" : "OUTPUT", + "physicalPlan" : { + "@type" : "streamSinkV1", + "properties" : { + "queryContext" : "OUTPUT" + }, + "source" : { + "@type" : "streamSelectV1", + "properties" : { + "queryContext" : "Project" + }, + "source" : { + "@type" : "streamSourceV1", + "properties" : { + "queryContext" : "KsqlTopic/Source" + }, + "topicName" : "input", + "formats" : { + "keyFormat" : { + "format" : "KAFKA", + "properties" : { } + }, + "valueFormat" : { + "format" : "PROTOBUF", + "properties" : { + "fullSchemaName" : "ValueName", + "unwrapPrimitives" : "true", + "schemaId" : "1" + } + } + }, + "timestampColumn" : null, + "sourceSchema" : "`c1` INTEGER", + "pseudoColumnVersion" : 1 + }, + "keyColumnNames" : [ ], + "selectedKeys" : null, + "selectExpressions" : [ "`c1` AS `c1`" ] + }, + "formats" : { + "keyFormat" : { + "format" : "KAFKA", + "properties" : { } + }, + "valueFormat" : { + "format" : "PROTOBUF", + "properties" : { + "fullSchemaName" : "ValueName", + "unwrapPrimitives" : "true" + } + } + }, + "topicName" : "OUTPUT", + "timestampColumn" : null + }, + "queryId" : "CSAS_OUTPUT_0", + "runtimeId" : null + } + } ], + "configs" : { + "ksql.extension.dir" : "ext", + "ksql.streams.cache.max.bytes.buffering" : "0", + "metric.reporters" : "", + "ksql.query.status.running.threshold.seconds" : "300", + "ksql.connect.basic.auth.credentials.reload" : "false", + "ksql.output.topic.name.prefix" : "", + "ksql.query.pull.stream.enabled" : "true", + "ksql.query.push.v2.interpreter.enabled" : "true", + "ksql.queryanonymizer.logs_enabled" : "true", + "ksql.variable.substitution.enable" : "true", + "ksql.streams.shutdown.timeout.ms" : "300000", + "ksql.query.pull.max.allowed.offset.lag" : "9223372036854775807", + "ksql.query.pull.max.qps" : "2147483647", + "ksql.access.validator.enable" : "auto", + "ksql.streams.bootstrap.servers" : "localhost:0", + "ksql.query.pull.metrics.enabled" : "true", + "ksql.metrics.extension" : null, + "ksql.query.push.v2.alos.enabled" : "true", + "ksql.query.push.v2.max.hourly.bandwidth.megabytes" : "2147483647", + "ksql.query.pull.range.scan.enabled" : "true", + "ksql.transient.query.cleanup.service.initial.delay.seconds" : "600", + "ksql.hidden.topics" : "_confluent.*,__confluent.*,_schemas,__consumer_offsets,__transaction_state,connect-configs,connect-offsets,connect-status,connect-statuses", + "ksql.lambdas.enabled" : "true", + "ksql.source.table.materialization.enabled" : "true", + "ksql.query.pull.max.hourly.bandwidth.megabytes" : "2147483647", + "ksql.sink.window.change.log.additional.retention" : "1000000", + "ksql.query.persistent.active.limit" : "2147483647", + "ksql.persistence.wrap.single.values" : null, + "ksql.query.transient.max.bytes.buffering.total" : "-1", + "ksql.connect.basic.auth.credentials.source" : "NONE", + "ksql.schema.registry.url" : "schema_registry.url:0", + "ksql.properties.overrides.denylist" : "", + "ksql.service.id" : "some.ksql.service.id", + "ksql.query.push.v2.max.catchup.consumers" : "5", + "ksql.query.push.v2.enabled" : "false", + "ksql.transient.query.cleanup.service.enable" : "true", + "ksql.query.push.v2.metrics.enabled" : "true", + "ksql.rowpartition.rowoffset.enabled" : "true", + "ksql.streams.commit.interval.ms" : "2000", + "ksql.query.pull.table.scan.enabled" : "true", + "ksql.streams.auto.commit.interval.ms" : "0", + "ksql.streams.topology.optimization" : "all", + "ksql.endpoint.migrate.query" : "true", + "ksql.query.push.v2.registry.installed" : "false", + "ksql.streams.num.stream.threads" : "4", + "ksql.metrics.tags.custom" : "", + "ksql.query.push.v2.catchup.consumer.msg.window" : "50", + "ksql.runtime.feature.shared.enabled" : "false", + "ksql.udf.collect.metrics" : "false", + "ksql.new.query.planner.enabled" : "false", + "ksql.connect.request.headers.plugin" : null, + "ksql.security.extension.class" : null, + "ksql.transient.prefix" : "transient_", + "ksql.headers.columns.enabled" : "true", + "ksql.streams.default.deserialization.exception.handler" : "io.confluent.ksql.errors.LogMetricAndContinueExceptionHandler", + "ksql.connect.request.timeout.ms" : "5000", + "ksql.query.pull.enable.standby.reads" : "false", + "ksql.persistence.default.format.key" : "KAFKA", + "ksql.query.persistent.max.bytes.buffering.total" : "-1", + "ksql.query.error.max.queue.size" : "10", + "ksql.query.cleanup.shutdown.timeout.ms" : "30000", + "ksql.internal.topic.min.insync.replicas" : "1", + "ksql.internal.topic.replicas" : "1", + "ksql.insert.into.values.enabled" : "true", + "ksql.queryanonymizer.cluster_namespace" : null, + "ksql.create.or.replace.enabled" : "true", + "ksql.shared.runtimes.count" : "2", + "ksql.cast.strings.preserve.nulls" : "true", + "ksql.authorization.cache.max.entries" : "10000", + "ksql.pull.queries.enable" : "true", + "ksql.transient.query.cleanup.service.period.seconds" : "600", + "ksql.suppress.enabled" : "false", + "ksql.readonly.topics" : "_confluent.*,__confluent.*,_schemas,__consumer_offsets,__transaction_state,connect-configs,connect-offsets,connect-status,connect-statuses", + "ksql.connect.basic.auth.credentials.file" : "", + "ksql.authorization.cache.expiry.time.secs" : "30", + "ksql.query.retry.backoff.initial.ms" : "15000", + "ksql.query.pull.max.concurrent.requests" : "2147483647", + "ksql.streams.auto.offset.reset" : "earliest", + "ksql.connect.url" : "http://localhost:8083", + "ksql.query.push.v2.new.latest.delay.ms" : "5000", + "ksql.query.push.v2.latest.reset.age.ms" : "30000", + "ksql.streams.default.production.exception.handler" : "io.confluent.ksql.errors.ProductionExceptionHandlerUtil$LogAndFailProductionExceptionHandler", + "ksql.query.pull.interpreter.enabled" : "true", + "ksql.query.pull.limit.clause.enabled" : "true", + "ksql.query.pull.router.thread.pool.size" : "50", + "ksql.query.push.v2.continuation.tokens.enabled" : "false", + "ksql.query.retry.backoff.max.ms" : "900000", + "ksql.timestamp.throw.on.invalid" : "false", + "ksql.persistence.default.format.value" : null, + "ksql.udfs.enabled" : "true", + "ksql.udf.enable.security.manager" : "true", + "ksql.connect.worker.config" : "", + "ksql.nested.error.set.null" : "true", + "ksql.query.pull.thread.pool.size" : "50", + "ksql.persistent.prefix" : "query_", + "ksql.metastore.backup.location" : "", + "ksql.error.classifier.regex" : "", + "ksql.suppress.buffer.size.bytes" : "-1" + } +} \ No newline at end of file diff --git a/ksqldb-functional-tests/src/test/resources/historical_plans/elements_-_validate_schema_id_without_elements_with_non-default_schema_name_OK_-_PROTOBUF/7.3.0_1650556179734/spec.json b/ksqldb-functional-tests/src/test/resources/historical_plans/elements_-_validate_schema_id_without_elements_with_non-default_schema_name_OK_-_PROTOBUF/7.3.0_1650556179734/spec.json new file mode 100644 index 000000000000..e2e0cbf6e228 --- /dev/null +++ b/ksqldb-functional-tests/src/test/resources/historical_plans/elements_-_validate_schema_id_without_elements_with_non-default_schema_name_OK_-_PROTOBUF/7.3.0_1650556179734/spec.json @@ -0,0 +1,120 @@ +{ + "version" : "7.3.0", + "timestamp" : 1650556179734, + "path" : "query-validation-tests/elements.json", + "schemas" : { + "CSAS_OUTPUT_0.KsqlTopic.Source" : { + "schema" : "`c1` INTEGER", + "keyFormat" : { + "format" : "KAFKA" + }, + "valueFormat" : { + "format" : "PROTOBUF", + "properties" : { + "fullSchemaName" : "ValueName", + "unwrapPrimitives" : "true", + "schemaId" : "1" + } + } + }, + "CSAS_OUTPUT_0.OUTPUT" : { + "schema" : "`c1` INTEGER", + "keyFormat" : { + "format" : "KAFKA" + }, + "valueFormat" : { + "format" : "PROTOBUF", + "properties" : { + "fullSchemaName" : "ValueName", + "unwrapPrimitives" : "true" + } + } + } + }, + "testCase" : { + "name" : "validate schema id without elements with non-default schema name OK - PROTOBUF", + "inputs" : [ { + "topic" : "input", + "key" : null, + "value" : { + "c1" : 4 + } + } ], + "outputs" : [ { + "topic" : "OUTPUT", + "key" : null, + "value" : { + "c1" : 4 + } + } ], + "topics" : [ { + "name" : "input", + "valueSchema" : "syntax = \"proto3\";\n\nmessage ValueName {\n int32 c1 = 1;\n}\n", + "valueFormat" : "PROTOBUF", + "replicas" : 1, + "numPartitions" : 1 + }, { + "name" : "OUTPUT", + "replicas" : 1, + "numPartitions" : 4 + } ], + "statements" : [ "CREATE STREAM INPUT WITH (kafka_topic='input', value_format='PROTOBUF', value_schema_id=1);", "CREATE STREAM OUTPUT WITH(PARTITIONS = 4) as SELECT * FROM input;" ], + "post" : { + "sources" : [ { + "name" : "INPUT", + "type" : "STREAM", + "schema" : "`c1` INTEGER", + "keyFormat" : { + "format" : "KAFKA" + }, + "valueFormat" : "PROTOBUF", + "keyFeatures" : [ ], + "valueFeatures" : [ ], + "isSource" : false + }, { + "name" : "OUTPUT", + "type" : "STREAM", + "schema" : "`c1` INTEGER", + "keyFormat" : { + "format" : "KAFKA" + }, + "valueFormat" : "PROTOBUF", + "keyFeatures" : [ ], + "valueFeatures" : [ ], + "isSource" : false + } ], + "topics" : { + "topics" : [ { + "name" : "input", + "keyFormat" : { + "format" : "KAFKA" + }, + "valueFormat" : { + "format" : "PROTOBUF", + "properties" : { + "fullSchemaName" : "ValueName", + "unwrapPrimitives" : "true", + "schemaId" : "1" + } + }, + "partitions" : 1, + "valueSchema" : "syntax = \"proto3\";\n\nmessage ValueName {\n int32 c1 = 1;\n}\n" + }, { + "name" : "OUTPUT", + "keyFormat" : { + "format" : "KAFKA" + }, + "valueFormat" : { + "format" : "PROTOBUF", + "properties" : { + "fullSchemaName" : "ValueName", + "unwrapPrimitives" : "true" + } + }, + "partitions" : 4, + "valueSchema" : "syntax = \"proto3\";\n\nmessage ValueName {\n int32 c1 = 1;\n}\n" + } ] + } + } + } +} \ No newline at end of file diff --git a/ksqldb-functional-tests/src/test/resources/historical_plans/elements_-_validate_schema_id_without_elements_with_non-default_schema_name_OK_-_PROTOBUF/7.3.0_1650556179734/topology b/ksqldb-functional-tests/src/test/resources/historical_plans/elements_-_validate_schema_id_without_elements_with_non-default_schema_name_OK_-_PROTOBUF/7.3.0_1650556179734/topology new file mode 100644 index 000000000000..a70a4e91e301 --- /dev/null +++ b/ksqldb-functional-tests/src/test/resources/historical_plans/elements_-_validate_schema_id_without_elements_with_non-default_schema_name_OK_-_PROTOBUF/7.3.0_1650556179734/topology @@ -0,0 +1,13 @@ +Topologies: + Sub-topology: 0 + Source: KSTREAM-SOURCE-0000000000 (topics: [input]) + --> KSTREAM-TRANSFORMVALUES-0000000001 + Processor: KSTREAM-TRANSFORMVALUES-0000000001 (stores: []) + --> Project + <-- KSTREAM-SOURCE-0000000000 + Processor: Project (stores: []) + --> KSTREAM-SINK-0000000003 + <-- KSTREAM-TRANSFORMVALUES-0000000001 + Sink: KSTREAM-SINK-0000000003 (topic: OUTPUT) + <-- Project + diff --git a/ksqldb-functional-tests/src/test/resources/historical_plans/elements_-_validate_without_elements_with_non-default_schema_name_OK_-_PROTOBUF/7.2.0_1650587924391/plan.json b/ksqldb-functional-tests/src/test/resources/historical_plans/elements_-_validate_without_elements_with_non-default_schema_name_OK_-_PROTOBUF/7.2.0_1650587924391/plan.json new file mode 100644 index 000000000000..c6a4e07f677c --- /dev/null +++ b/ksqldb-functional-tests/src/test/resources/historical_plans/elements_-_validate_without_elements_with_non-default_schema_name_OK_-_PROTOBUF/7.2.0_1650587924391/plan.json @@ -0,0 +1,198 @@ +{ + "plan" : [ { + "@type" : "ksqlPlanV1", + "statementText" : "CREATE STREAM INPUT (C1 INTEGER) WITH (KAFKA_TOPIC='input', KEY_FORMAT='KAFKA', VALUE_FORMAT='PROTOBUF');", + "ddlCommand" : { + "@type" : "createStreamV1", + "sourceName" : "INPUT", + "schema" : "`C1` INTEGER", + "topicName" : "input", + "formats" : { + "keyFormat" : { + "format" : "KAFKA" + }, + "valueFormat" : { + "format" : "PROTOBUF", + "properties" : { + "unwrapPrimitives" : "true" + } + } + }, + "orReplace" : false, + "isSource" : false + } + }, { + "@type" : "ksqlPlanV1", + "statementText" : "CREATE STREAM OUTPUT WITH (PARTITIONS=4) AS SELECT *\nFROM INPUT INPUT\nEMIT CHANGES", + "ddlCommand" : { + "@type" : "createStreamV1", + "sourceName" : "OUTPUT", + "schema" : "`C1` INTEGER", + "topicName" : "OUTPUT", + "formats" : { + "keyFormat" : { + "format" : "KAFKA" + }, + "valueFormat" : { + "format" : "PROTOBUF", + "properties" : { + "unwrapPrimitives" : "true" + } + } + }, + "orReplace" : false, + "isSource" : false + }, + "queryPlan" : { + "sources" : [ "INPUT" ], + "sink" : "OUTPUT", + "physicalPlan" : { + "@type" : "streamSinkV1", + "properties" : { + "queryContext" : "OUTPUT" + }, + "source" : { + "@type" : "streamSelectV1", + "properties" : { + "queryContext" : "Project" + }, + "source" : { + "@type" : "streamSourceV1", + "properties" : { + "queryContext" : "KsqlTopic/Source" + }, + "topicName" : "input", + "formats" : { + "keyFormat" : { + "format" : "KAFKA" + }, + "valueFormat" : { + "format" : "PROTOBUF", + "properties" : { + "unwrapPrimitives" : "true" + } + } + }, + "sourceSchema" : "`C1` INTEGER", + "pseudoColumnVersion" : 1 + }, + "selectExpressions" : [ "C1 AS C1" ] + }, + "formats" : { + "keyFormat" : { + "format" : "KAFKA" + }, + "valueFormat" : { + "format" : "PROTOBUF", + "properties" : { + "unwrapPrimitives" : "true" + } + } + }, + "topicName" : "OUTPUT" + }, + "queryId" : "CSAS_OUTPUT_0" + } + } ], + "configs" : { + "ksql.extension.dir" : "ext", + "ksql.streams.cache.max.bytes.buffering" : "0", + "metric.reporters" : "", + "ksql.query.status.running.threshold.seconds" : "300", + "ksql.connect.basic.auth.credentials.reload" : "false", + "ksql.output.topic.name.prefix" : "", + "ksql.query.pull.stream.enabled" : "true", + "ksql.query.push.v2.interpreter.enabled" : "true", + "ksql.queryanonymizer.logs_enabled" : "true", + "ksql.variable.substitution.enable" : "true", + "ksql.streams.shutdown.timeout.ms" : "300000", + "ksql.query.pull.max.allowed.offset.lag" : "9223372036854775807", + "ksql.query.pull.max.qps" : "2147483647", + "ksql.access.validator.enable" : "auto", + "ksql.streams.bootstrap.servers" : "localhost:0", + "ksql.query.pull.metrics.enabled" : "true", + "ksql.metrics.extension" : null, + "ksql.query.push.v2.alos.enabled" : "true", + "ksql.query.push.v2.max.hourly.bandwidth.megabytes" : "2147483647", + "ksql.query.pull.range.scan.enabled" : "true", + "ksql.transient.query.cleanup.service.initial.delay.seconds" : "600", + "ksql.hidden.topics" : "_confluent.*,__confluent.*,_schemas,__consumer_offsets,__transaction_state,connect-configs,connect-offsets,connect-status,connect-statuses", + "ksql.lambdas.enabled" : "true", + "ksql.source.table.materialization.enabled" : "true", + "ksql.query.pull.max.hourly.bandwidth.megabytes" : "2147483647", + "ksql.sink.window.change.log.additional.retention" : "1000000", + "ksql.query.persistent.active.limit" : "2147483647", + "ksql.persistence.wrap.single.values" : null, + "ksql.query.transient.max.bytes.buffering.total" : "-1", + "ksql.connect.basic.auth.credentials.source" : "NONE", + "ksql.schema.registry.url" : "schema_registry.url:0", + "ksql.properties.overrides.denylist" : "", + "ksql.service.id" : "some.ksql.service.id", + "ksql.query.push.v2.max.catchup.consumers" : "5", + "ksql.query.push.v2.enabled" : "false", + "ksql.transient.query.cleanup.service.enable" : "true", + "ksql.query.push.v2.metrics.enabled" : "true", + "ksql.rowpartition.rowoffset.enabled" : "true", + "ksql.streams.commit.interval.ms" : "2000", + "ksql.query.pull.table.scan.enabled" : "true", + "ksql.streams.auto.commit.interval.ms" : "0", + "ksql.streams.topology.optimization" : "all", + "ksql.endpoint.migrate.query" : "true", + "ksql.query.push.v2.registry.installed" : "false", + "ksql.streams.num.stream.threads" : "4", + "ksql.metrics.tags.custom" : "", + "ksql.query.push.v2.catchup.consumer.msg.window" : "50", + "ksql.runtime.feature.shared.enabled" : "false", + "ksql.udf.collect.metrics" : "false", + "ksql.new.query.planner.enabled" : "false", + "ksql.connect.request.headers.plugin" : null, + "ksql.security.extension.class" : null, + "ksql.transient.prefix" : "transient_", + "ksql.headers.columns.enabled" : "true", + "ksql.streams.default.deserialization.exception.handler" : "io.confluent.ksql.errors.LogMetricAndContinueExceptionHandler", + "ksql.connect.request.timeout.ms" : "5000", + "ksql.query.pull.enable.standby.reads" : "false", + "ksql.persistence.default.format.key" : "KAFKA", + "ksql.query.persistent.max.bytes.buffering.total" : "-1", + "ksql.query.error.max.queue.size" : "10", + "ksql.query.cleanup.shutdown.timeout.ms" : "30000", + "ksql.internal.topic.min.insync.replicas" : "1", + "ksql.internal.topic.replicas" : "1", + "ksql.insert.into.values.enabled" : "true", + "ksql.queryanonymizer.cluster_namespace" : null, + "ksql.create.or.replace.enabled" : "true", + "ksql.shared.runtimes.count" : "2", + "ksql.cast.strings.preserve.nulls" : "true", + "ksql.authorization.cache.max.entries" : "10000", + "ksql.pull.queries.enable" : "true", + "ksql.transient.query.cleanup.service.period.seconds" : "600", + "ksql.suppress.enabled" : "false", + "ksql.readonly.topics" : "_confluent.*,__confluent.*,_schemas,__consumer_offsets,__transaction_state,connect-configs,connect-offsets,connect-status,connect-statuses", + "ksql.connect.basic.auth.credentials.file" : "", + "ksql.authorization.cache.expiry.time.secs" : "30", + "ksql.query.retry.backoff.initial.ms" : "15000", + "ksql.query.pull.max.concurrent.requests" : "2147483647", + "ksql.streams.auto.offset.reset" : "earliest", + "ksql.connect.url" : "http://localhost:8083", + "ksql.query.push.v2.new.latest.delay.ms" : "5000", + "ksql.query.push.v2.latest.reset.age.ms" : "30000", + "ksql.streams.default.production.exception.handler" : "io.confluent.ksql.errors.ProductionExceptionHandlerUtil$LogAndFailProductionExceptionHandler", + "ksql.query.pull.interpreter.enabled" : "true", + "ksql.query.pull.limit.clause.enabled" : "true", + "ksql.connect.error.handler" : null, + "ksql.query.pull.router.thread.pool.size" : "50", + "ksql.query.push.v2.continuation.tokens.enabled" : "false", + "ksql.query.retry.backoff.max.ms" : "900000", + "ksql.timestamp.throw.on.invalid" : "false", + "ksql.persistence.default.format.value" : null, + "ksql.udfs.enabled" : "true", + "ksql.udf.enable.security.manager" : "true", + "ksql.connect.worker.config" : "", + "ksql.nested.error.set.null" : "true", + "ksql.query.pull.thread.pool.size" : "50", + "ksql.persistent.prefix" : "query_", + "ksql.metastore.backup.location" : "", + "ksql.error.classifier.regex" : "", + "ksql.suppress.buffer.size.bytes" : "-1" + } +} \ No newline at end of file diff --git a/ksqldb-functional-tests/src/test/resources/historical_plans/elements_-_validate_without_elements_with_non-default_schema_name_OK_-_PROTOBUF/7.2.0_1650587924391/spec.json b/ksqldb-functional-tests/src/test/resources/historical_plans/elements_-_validate_without_elements_with_non-default_schema_name_OK_-_PROTOBUF/7.2.0_1650587924391/spec.json new file mode 100644 index 000000000000..1152addeb0a2 --- /dev/null +++ b/ksqldb-functional-tests/src/test/resources/historical_plans/elements_-_validate_without_elements_with_non-default_schema_name_OK_-_PROTOBUF/7.2.0_1650587924391/spec.json @@ -0,0 +1,114 @@ +{ + "version" : "7.2.0", + "timestamp" : 1650587924391, + "path" : "query-validation-tests/elements.json", + "schemas" : { + "CSAS_OUTPUT_0.KsqlTopic.Source" : { + "schema" : "`C1` INTEGER", + "keyFormat" : { + "format" : "KAFKA" + }, + "valueFormat" : { + "format" : "PROTOBUF", + "properties" : { + "unwrapPrimitives" : "true" + } + } + }, + "CSAS_OUTPUT_0.OUTPUT" : { + "schema" : "`C1` INTEGER", + "keyFormat" : { + "format" : "KAFKA" + }, + "valueFormat" : { + "format" : "PROTOBUF", + "properties" : { + "unwrapPrimitives" : "true" + } + } + } + }, + "testCase" : { + "name" : "validate without elements with non-default schema name OK - PROTOBUF", + "inputs" : [ { + "topic" : "input", + "key" : null, + "value" : { + "c1" : 4 + } + } ], + "outputs" : [ { + "topic" : "OUTPUT", + "key" : null, + "value" : { + "C1" : 4 + } + } ], + "topics" : [ { + "name" : "input", + "valueSchema" : "syntax = \"proto3\";\n\nmessage ValueName {\n int32 c1 = 1;\n}\n", + "valueFormat" : "PROTOBUF", + "replicas" : 1, + "numPartitions" : 1 + }, { + "name" : "OUTPUT", + "replicas" : 1, + "numPartitions" : 4 + } ], + "statements" : [ "CREATE STREAM INPUT WITH (kafka_topic='input', value_format='PROTOBUF');", "CREATE STREAM OUTPUT WITH(PARTITIONS = 4) as SELECT * FROM input;" ], + "post" : { + "sources" : [ { + "name" : "INPUT", + "type" : "STREAM", + "schema" : "`C1` INTEGER", + "keyFormat" : { + "format" : "KAFKA" + }, + "valueFormat" : "PROTOBUF", + "keyFeatures" : [ ], + "valueFeatures" : [ ], + "isSource" : false + }, { + "name" : "OUTPUT", + "type" : "STREAM", + "schema" : "`C1` INTEGER", + "keyFormat" : { + "format" : "KAFKA" + }, + "valueFormat" : "PROTOBUF", + "keyFeatures" : [ ], + "valueFeatures" : [ ], + "isSource" : false + } ], + "topics" : { + "topics" : [ { + "name" : "input", + "keyFormat" : { + "format" : "KAFKA" + }, + "valueFormat" : { + "format" : "PROTOBUF", + "properties" : { + "unwrapPrimitives" : "true" + } + }, + "partitions" : 1, + "valueSchema" : "syntax = \"proto3\";\n\nmessage ValueName {\n int32 c1 = 1;\n}\n" + }, { + "name" : "OUTPUT", + "keyFormat" : { + "format" : "KAFKA" + }, + "valueFormat" : { + "format" : "PROTOBUF", + "properties" : { + "unwrapPrimitives" : "true" + } + }, + "partitions" : 4, + "valueSchema" : "syntax = \"proto3\";\n\nmessage ConnectDefault1 {\n int32 C1 = 1;\n}\n" + } ] + } + } + } +} \ No newline at end of file diff --git a/ksqldb-functional-tests/src/test/resources/historical_plans/elements_-_validate_without_elements_with_non-default_schema_name_OK_-_PROTOBUF/7.2.0_1650587924391/topology b/ksqldb-functional-tests/src/test/resources/historical_plans/elements_-_validate_without_elements_with_non-default_schema_name_OK_-_PROTOBUF/7.2.0_1650587924391/topology new file mode 100644 index 000000000000..a70a4e91e301 --- /dev/null +++ b/ksqldb-functional-tests/src/test/resources/historical_plans/elements_-_validate_without_elements_with_non-default_schema_name_OK_-_PROTOBUF/7.2.0_1650587924391/topology @@ -0,0 +1,13 @@ +Topologies: + Sub-topology: 0 + Source: KSTREAM-SOURCE-0000000000 (topics: [input]) + --> KSTREAM-TRANSFORMVALUES-0000000001 + Processor: KSTREAM-TRANSFORMVALUES-0000000001 (stores: []) + --> Project + <-- KSTREAM-SOURCE-0000000000 + Processor: Project (stores: []) + --> KSTREAM-SINK-0000000003 + <-- KSTREAM-TRANSFORMVALUES-0000000001 + Sink: KSTREAM-SINK-0000000003 (topic: OUTPUT) + <-- Project + diff --git a/ksqldb-functional-tests/src/test/resources/historical_plans/elements_-_validate_without_elements_with_non-default_schema_name_OK_-_PROTOBUF/7.3.0_1650556179599/plan.json b/ksqldb-functional-tests/src/test/resources/historical_plans/elements_-_validate_without_elements_with_non-default_schema_name_OK_-_PROTOBUF/7.3.0_1650556179599/plan.json new file mode 100644 index 000000000000..c340c7dde0e1 --- /dev/null +++ b/ksqldb-functional-tests/src/test/resources/historical_plans/elements_-_validate_without_elements_with_non-default_schema_name_OK_-_PROTOBUF/7.3.0_1650556179599/plan.json @@ -0,0 +1,211 @@ +{ + "plan" : [ { + "@type" : "ksqlPlanV1", + "statementText" : "CREATE STREAM INPUT (C1 INTEGER) WITH (KAFKA_TOPIC='input', KEY_FORMAT='KAFKA', VALUE_FORMAT='PROTOBUF');", + "ddlCommand" : { + "@type" : "createStreamV1", + "sourceName" : "INPUT", + "schema" : "`C1` INTEGER", + "timestampColumn" : null, + "topicName" : "input", + "formats" : { + "keyFormat" : { + "format" : "KAFKA", + "properties" : { } + }, + "valueFormat" : { + "format" : "PROTOBUF", + "properties" : { + "unwrapPrimitives" : "true" + } + } + }, + "windowInfo" : null, + "orReplace" : false, + "isSource" : false + }, + "queryPlan" : null + }, { + "@type" : "ksqlPlanV1", + "statementText" : "CREATE STREAM OUTPUT WITH (PARTITIONS=4) AS SELECT *\nFROM INPUT INPUT\nEMIT CHANGES", + "ddlCommand" : { + "@type" : "createStreamV1", + "sourceName" : "OUTPUT", + "schema" : "`C1` INTEGER", + "timestampColumn" : null, + "topicName" : "OUTPUT", + "formats" : { + "keyFormat" : { + "format" : "KAFKA", + "properties" : { } + }, + "valueFormat" : { + "format" : "PROTOBUF", + "properties" : { + "unwrapPrimitives" : "true" + } + } + }, + "windowInfo" : null, + "orReplace" : false, + "isSource" : false + }, + "queryPlan" : { + "sources" : [ "INPUT" ], + "sink" : "OUTPUT", + "physicalPlan" : { + "@type" : "streamSinkV1", + "properties" : { + "queryContext" : "OUTPUT" + }, + "source" : { + "@type" : "streamSelectV1", + "properties" : { + "queryContext" : "Project" + }, + "source" : { + "@type" : "streamSourceV1", + "properties" : { + "queryContext" : "KsqlTopic/Source" + }, + "topicName" : "input", + "formats" : { + "keyFormat" : { + "format" : "KAFKA", + "properties" : { } + }, + "valueFormat" : { + "format" : "PROTOBUF", + "properties" : { + "unwrapPrimitives" : "true" + } + } + }, + "timestampColumn" : null, + "sourceSchema" : "`C1` INTEGER", + "pseudoColumnVersion" : 1 + }, + "keyColumnNames" : [ ], + "selectedKeys" : null, + "selectExpressions" : [ "C1 AS C1" ] + }, + "formats" : { + "keyFormat" : { + "format" : "KAFKA", + "properties" : { } + }, + "valueFormat" : { + "format" : "PROTOBUF", + "properties" : { + "unwrapPrimitives" : "true" + } + } + }, + "topicName" : "OUTPUT", + "timestampColumn" : null + }, + "queryId" : "CSAS_OUTPUT_0", + "runtimeId" : null + } + } ], + "configs" : { + "ksql.extension.dir" : "ext", + "ksql.streams.cache.max.bytes.buffering" : "0", + "metric.reporters" : "", + "ksql.query.status.running.threshold.seconds" : "300", + "ksql.connect.basic.auth.credentials.reload" : "false", + "ksql.output.topic.name.prefix" : "", + "ksql.query.pull.stream.enabled" : "true", + "ksql.query.push.v2.interpreter.enabled" : "true", + "ksql.queryanonymizer.logs_enabled" : "true", + "ksql.variable.substitution.enable" : "true", + "ksql.streams.shutdown.timeout.ms" : "300000", + "ksql.query.pull.max.allowed.offset.lag" : "9223372036854775807", + "ksql.query.pull.max.qps" : "2147483647", + "ksql.access.validator.enable" : "auto", + "ksql.streams.bootstrap.servers" : "localhost:0", + "ksql.query.pull.metrics.enabled" : "true", + "ksql.metrics.extension" : null, + "ksql.query.push.v2.alos.enabled" : "true", + "ksql.query.push.v2.max.hourly.bandwidth.megabytes" : "2147483647", + "ksql.query.pull.range.scan.enabled" : "true", + "ksql.transient.query.cleanup.service.initial.delay.seconds" : "600", + "ksql.hidden.topics" : "_confluent.*,__confluent.*,_schemas,__consumer_offsets,__transaction_state,connect-configs,connect-offsets,connect-status,connect-statuses", + "ksql.lambdas.enabled" : "true", + "ksql.source.table.materialization.enabled" : "true", + "ksql.query.pull.max.hourly.bandwidth.megabytes" : "2147483647", + "ksql.sink.window.change.log.additional.retention" : "1000000", + "ksql.query.persistent.active.limit" : "2147483647", + "ksql.persistence.wrap.single.values" : null, + "ksql.query.transient.max.bytes.buffering.total" : "-1", + "ksql.connect.basic.auth.credentials.source" : "NONE", + "ksql.schema.registry.url" : "schema_registry.url:0", + "ksql.properties.overrides.denylist" : "", + "ksql.service.id" : "some.ksql.service.id", + "ksql.query.push.v2.max.catchup.consumers" : "5", + "ksql.query.push.v2.enabled" : "false", + "ksql.transient.query.cleanup.service.enable" : "true", + "ksql.query.push.v2.metrics.enabled" : "true", + "ksql.rowpartition.rowoffset.enabled" : "true", + "ksql.streams.commit.interval.ms" : "2000", + "ksql.query.pull.table.scan.enabled" : "true", + "ksql.streams.auto.commit.interval.ms" : "0", + "ksql.streams.topology.optimization" : "all", + "ksql.endpoint.migrate.query" : "true", + "ksql.query.push.v2.registry.installed" : "false", + "ksql.streams.num.stream.threads" : "4", + "ksql.metrics.tags.custom" : "", + "ksql.query.push.v2.catchup.consumer.msg.window" : "50", + "ksql.runtime.feature.shared.enabled" : "false", + "ksql.udf.collect.metrics" : "false", + "ksql.new.query.planner.enabled" : "false", + "ksql.connect.request.headers.plugin" : null, + "ksql.security.extension.class" : null, + "ksql.transient.prefix" : "transient_", + "ksql.headers.columns.enabled" : "true", + "ksql.streams.default.deserialization.exception.handler" : "io.confluent.ksql.errors.LogMetricAndContinueExceptionHandler", + "ksql.connect.request.timeout.ms" : "5000", + "ksql.query.pull.enable.standby.reads" : "false", + "ksql.persistence.default.format.key" : "KAFKA", + "ksql.query.persistent.max.bytes.buffering.total" : "-1", + "ksql.query.error.max.queue.size" : "10", + "ksql.query.cleanup.shutdown.timeout.ms" : "30000", + "ksql.internal.topic.min.insync.replicas" : "1", + "ksql.internal.topic.replicas" : "1", + "ksql.insert.into.values.enabled" : "true", + "ksql.queryanonymizer.cluster_namespace" : null, + "ksql.create.or.replace.enabled" : "true", + "ksql.shared.runtimes.count" : "2", + "ksql.cast.strings.preserve.nulls" : "true", + "ksql.authorization.cache.max.entries" : "10000", + "ksql.pull.queries.enable" : "true", + "ksql.transient.query.cleanup.service.period.seconds" : "600", + "ksql.suppress.enabled" : "false", + "ksql.readonly.topics" : "_confluent.*,__confluent.*,_schemas,__consumer_offsets,__transaction_state,connect-configs,connect-offsets,connect-status,connect-statuses", + "ksql.connect.basic.auth.credentials.file" : "", + "ksql.authorization.cache.expiry.time.secs" : "30", + "ksql.query.retry.backoff.initial.ms" : "15000", + "ksql.query.pull.max.concurrent.requests" : "2147483647", + "ksql.streams.auto.offset.reset" : "earliest", + "ksql.connect.url" : "http://localhost:8083", + "ksql.query.push.v2.new.latest.delay.ms" : "5000", + "ksql.query.push.v2.latest.reset.age.ms" : "30000", + "ksql.streams.default.production.exception.handler" : "io.confluent.ksql.errors.ProductionExceptionHandlerUtil$LogAndFailProductionExceptionHandler", + "ksql.query.pull.interpreter.enabled" : "true", + "ksql.query.pull.limit.clause.enabled" : "true", + "ksql.query.pull.router.thread.pool.size" : "50", + "ksql.query.push.v2.continuation.tokens.enabled" : "false", + "ksql.query.retry.backoff.max.ms" : "900000", + "ksql.timestamp.throw.on.invalid" : "false", + "ksql.persistence.default.format.value" : null, + "ksql.udfs.enabled" : "true", + "ksql.udf.enable.security.manager" : "true", + "ksql.connect.worker.config" : "", + "ksql.nested.error.set.null" : "true", + "ksql.query.pull.thread.pool.size" : "50", + "ksql.persistent.prefix" : "query_", + "ksql.metastore.backup.location" : "", + "ksql.error.classifier.regex" : "", + "ksql.suppress.buffer.size.bytes" : "-1" + } +} \ No newline at end of file diff --git a/ksqldb-functional-tests/src/test/resources/historical_plans/elements_-_validate_without_elements_with_non-default_schema_name_OK_-_PROTOBUF/7.3.0_1650556179599/spec.json b/ksqldb-functional-tests/src/test/resources/historical_plans/elements_-_validate_without_elements_with_non-default_schema_name_OK_-_PROTOBUF/7.3.0_1650556179599/spec.json new file mode 100644 index 000000000000..07edeb4ca20a --- /dev/null +++ b/ksqldb-functional-tests/src/test/resources/historical_plans/elements_-_validate_without_elements_with_non-default_schema_name_OK_-_PROTOBUF/7.3.0_1650556179599/spec.json @@ -0,0 +1,114 @@ +{ + "version" : "7.3.0", + "timestamp" : 1650556179599, + "path" : "query-validation-tests/elements.json", + "schemas" : { + "CSAS_OUTPUT_0.KsqlTopic.Source" : { + "schema" : "`C1` INTEGER", + "keyFormat" : { + "format" : "KAFKA" + }, + "valueFormat" : { + "format" : "PROTOBUF", + "properties" : { + "unwrapPrimitives" : "true" + } + } + }, + "CSAS_OUTPUT_0.OUTPUT" : { + "schema" : "`C1` INTEGER", + "keyFormat" : { + "format" : "KAFKA" + }, + "valueFormat" : { + "format" : "PROTOBUF", + "properties" : { + "unwrapPrimitives" : "true" + } + } + } + }, + "testCase" : { + "name" : "validate without elements with non-default schema name OK - PROTOBUF", + "inputs" : [ { + "topic" : "input", + "key" : null, + "value" : { + "c1" : 4 + } + } ], + "outputs" : [ { + "topic" : "OUTPUT", + "key" : null, + "value" : { + "C1" : 4 + } + } ], + "topics" : [ { + "name" : "input", + "valueSchema" : "syntax = \"proto3\";\n\nmessage ValueName {\n int32 c1 = 1;\n}\n", + "valueFormat" : "PROTOBUF", + "replicas" : 1, + "numPartitions" : 1 + }, { + "name" : "OUTPUT", + "replicas" : 1, + "numPartitions" : 4 + } ], + "statements" : [ "CREATE STREAM INPUT WITH (kafka_topic='input', value_format='PROTOBUF');", "CREATE STREAM OUTPUT WITH(PARTITIONS = 4) as SELECT * FROM input;" ], + "post" : { + "sources" : [ { + "name" : "INPUT", + "type" : "STREAM", + "schema" : "`C1` INTEGER", + "keyFormat" : { + "format" : "KAFKA" + }, + "valueFormat" : "PROTOBUF", + "keyFeatures" : [ ], + "valueFeatures" : [ ], + "isSource" : false + }, { + "name" : "OUTPUT", + "type" : "STREAM", + "schema" : "`C1` INTEGER", + "keyFormat" : { + "format" : "KAFKA" + }, + "valueFormat" : "PROTOBUF", + "keyFeatures" : [ ], + "valueFeatures" : [ ], + "isSource" : false + } ], + "topics" : { + "topics" : [ { + "name" : "input", + "keyFormat" : { + "format" : "KAFKA" + }, + "valueFormat" : { + "format" : "PROTOBUF", + "properties" : { + "unwrapPrimitives" : "true" + } + }, + "partitions" : 1, + "valueSchema" : "syntax = \"proto3\";\n\nmessage ValueName {\n int32 c1 = 1;\n}\n" + }, { + "name" : "OUTPUT", + "keyFormat" : { + "format" : "KAFKA" + }, + "valueFormat" : { + "format" : "PROTOBUF", + "properties" : { + "unwrapPrimitives" : "true" + } + }, + "partitions" : 4, + "valueSchema" : "syntax = \"proto3\";\n\nmessage ConnectDefault1 {\n int32 C1 = 1;\n}\n" + } ] + } + } + } +} \ No newline at end of file diff --git a/ksqldb-functional-tests/src/test/resources/historical_plans/elements_-_validate_without_elements_with_non-default_schema_name_OK_-_PROTOBUF/7.3.0_1650556179599/topology b/ksqldb-functional-tests/src/test/resources/historical_plans/elements_-_validate_without_elements_with_non-default_schema_name_OK_-_PROTOBUF/7.3.0_1650556179599/topology new file mode 100644 index 000000000000..a70a4e91e301 --- /dev/null +++ b/ksqldb-functional-tests/src/test/resources/historical_plans/elements_-_validate_without_elements_with_non-default_schema_name_OK_-_PROTOBUF/7.3.0_1650556179599/topology @@ -0,0 +1,13 @@ +Topologies: + Sub-topology: 0 + Source: KSTREAM-SOURCE-0000000000 (topics: [input]) + --> KSTREAM-TRANSFORMVALUES-0000000001 + Processor: KSTREAM-TRANSFORMVALUES-0000000001 (stores: []) + --> Project + <-- KSTREAM-SOURCE-0000000000 + Processor: Project (stores: []) + --> KSTREAM-SINK-0000000003 + <-- KSTREAM-TRANSFORMVALUES-0000000001 + Sink: KSTREAM-SINK-0000000003 (topic: OUTPUT) + <-- Project + diff --git a/ksqldb-functional-tests/src/test/resources/historical_plans/histogram_-_histogram_on_a_table_-_PROTOBUF_NOSR/7.3.0_1651069869084/plan.json b/ksqldb-functional-tests/src/test/resources/historical_plans/histogram_-_histogram_on_a_table_-_PROTOBUF_NOSR/7.3.0_1651069869084/plan.json new file mode 100644 index 000000000000..14eea76acd69 --- /dev/null +++ b/ksqldb-functional-tests/src/test/resources/historical_plans/histogram_-_histogram_on_a_table_-_PROTOBUF_NOSR/7.3.0_1651069869084/plan.json @@ -0,0 +1,275 @@ +{ + "plan" : [ { + "@type" : "ksqlPlanV1", + "statementText" : "CREATE TABLE TEST (K STRING PRIMARY KEY, ID BIGINT, NAME STRING, REGION STRING) WITH (KAFKA_TOPIC='test_topic', KEY_FORMAT='KAFKA', VALUE_FORMAT='PROTOBUF_NOSR');", + "ddlCommand" : { + "@type" : "createTableV1", + "sourceName" : "TEST", + "schema" : "`K` STRING KEY, `ID` BIGINT, `NAME` STRING, `REGION` STRING", + "timestampColumn" : null, + "topicName" : "test_topic", + "formats" : { + "keyFormat" : { + "format" : "KAFKA", + "properties" : { } + }, + "valueFormat" : { + "format" : "PROTOBUF_NOSR", + "properties" : { } + } + }, + "windowInfo" : null, + "orReplace" : false, + "isSource" : false + }, + "queryPlan" : null + }, { + "@type" : "ksqlPlanV1", + "statementText" : "CREATE TABLE COUNT_BY_REGION AS SELECT\n TEST.REGION REGION,\n HISTOGRAM(TEST.NAME) COUNTS\nFROM TEST TEST\nGROUP BY TEST.REGION\nEMIT CHANGES", + "ddlCommand" : { + "@type" : "createTableV1", + "sourceName" : "COUNT_BY_REGION", + "schema" : "`REGION` STRING KEY, `COUNTS` MAP", + "timestampColumn" : null, + "topicName" : "COUNT_BY_REGION", + "formats" : { + "keyFormat" : { + "format" : "KAFKA", + "properties" : { } + }, + "valueFormat" : { + "format" : "PROTOBUF_NOSR", + "properties" : { } + } + }, + "windowInfo" : null, + "orReplace" : false, + "isSource" : false + }, + "queryPlan" : { + "sources" : [ "TEST" ], + "sink" : "COUNT_BY_REGION", + "physicalPlan" : { + "@type" : "tableSinkV1", + "properties" : { + "queryContext" : "COUNT_BY_REGION" + }, + "source" : { + "@type" : "tableSelectV1", + "properties" : { + "queryContext" : "Aggregate/Project" + }, + "source" : { + "@type" : "tableAggregateV1", + "properties" : { + "queryContext" : "Aggregate/Aggregate" + }, + "source" : { + "@type" : "tableGroupByV2", + "properties" : { + "queryContext" : "Aggregate/GroupBy" + }, + "source" : { + "@type" : "tableSelectV1", + "properties" : { + "queryContext" : "Aggregate/Prepare" + }, + "source" : { + "@type" : "tableSourceV2", + "properties" : { + "queryContext" : "KsqlTopic/Source" + }, + "topicName" : "test_topic", + "formats" : { + "keyFormat" : { + "format" : "KAFKA", + "properties" : { } + }, + "valueFormat" : { + "format" : "PROTOBUF_NOSR", + "properties" : { } + } + }, + "timestampColumn" : null, + "sourceSchema" : "`K` STRING KEY, `ID` BIGINT, `NAME` STRING, `REGION` STRING", + "pseudoColumnVersion" : 1, + "stateStoreFormats" : { + "keyFormat" : { + "format" : "KAFKA", + "properties" : { } + }, + "valueFormat" : { + "format" : "PROTOBUF_NOSR", + "properties" : { } + } + } + }, + "keyColumnNames" : [ "K" ], + "selectExpressions" : [ "REGION AS REGION", "NAME AS NAME" ], + "internalFormats" : { + "keyFormat" : { + "format" : "KAFKA", + "properties" : { } + }, + "valueFormat" : { + "format" : "PROTOBUF_NOSR", + "properties" : { } + } + } + }, + "internalFormats" : { + "keyFormat" : { + "format" : "KAFKA", + "properties" : { } + }, + "valueFormat" : { + "format" : "PROTOBUF_NOSR", + "properties" : { } + } + }, + "groupByExpressions" : [ "REGION" ] + }, + "internalFormats" : { + "keyFormat" : { + "format" : "KAFKA", + "properties" : { } + }, + "valueFormat" : { + "format" : "PROTOBUF_NOSR", + "properties" : { } + } + }, + "nonAggregateColumns" : [ "REGION", "NAME" ], + "aggregationFunctions" : [ "HISTOGRAM(NAME)" ] + }, + "keyColumnNames" : [ "REGION" ], + "selectExpressions" : [ "KSQL_AGG_VARIABLE_0 AS COUNTS" ], + "internalFormats" : { + "keyFormat" : { + "format" : "KAFKA", + "properties" : { } + }, + "valueFormat" : { + "format" : "PROTOBUF_NOSR", + "properties" : { } + } + } + }, + "formats" : { + "keyFormat" : { + "format" : "KAFKA", + "properties" : { } + }, + "valueFormat" : { + "format" : "PROTOBUF_NOSR", + "properties" : { } + } + }, + "topicName" : "COUNT_BY_REGION", + "timestampColumn" : null + }, + "queryId" : "CTAS_COUNT_BY_REGION_0", + "runtimeId" : null + } + } ], + "configs" : { + "ksql.extension.dir" : "ext", + "ksql.streams.cache.max.bytes.buffering" : "0", + "metric.reporters" : "", + "ksql.query.status.running.threshold.seconds" : "300", + "ksql.connect.basic.auth.credentials.reload" : "false", + "ksql.output.topic.name.prefix" : "", + "ksql.query.pull.stream.enabled" : "true", + "ksql.query.push.v2.interpreter.enabled" : "true", + "ksql.queryanonymizer.logs_enabled" : "true", + "ksql.variable.substitution.enable" : "true", + "ksql.streams.shutdown.timeout.ms" : "300000", + "ksql.query.pull.max.allowed.offset.lag" : "9223372036854775807", + "ksql.query.pull.max.qps" : "2147483647", + "ksql.access.validator.enable" : "auto", + "ksql.streams.bootstrap.servers" : "localhost:0", + "ksql.query.pull.metrics.enabled" : "true", + "ksql.metrics.extension" : null, + "ksql.query.push.v2.alos.enabled" : "true", + "ksql.query.push.v2.max.hourly.bandwidth.megabytes" : "2147483647", + "ksql.query.pull.range.scan.enabled" : "true", + "ksql.transient.query.cleanup.service.initial.delay.seconds" : "600", + "ksql.hidden.topics" : "_confluent.*,__confluent.*,_schemas,__consumer_offsets,__transaction_state,connect-configs,connect-offsets,connect-status,connect-statuses", + "ksql.lambdas.enabled" : "true", + "ksql.source.table.materialization.enabled" : "true", + "ksql.query.pull.max.hourly.bandwidth.megabytes" : "2147483647", + "ksql.sink.window.change.log.additional.retention" : "1000000", + "ksql.query.persistent.active.limit" : "2147483647", + "ksql.persistence.wrap.single.values" : null, + "ksql.query.transient.max.bytes.buffering.total" : "-1", + "ksql.connect.basic.auth.credentials.source" : "NONE", + "ksql.schema.registry.url" : "schema_registry.url:0", + "ksql.properties.overrides.denylist" : "", + "ksql.service.id" : "some.ksql.service.id", + "ksql.query.push.v2.max.catchup.consumers" : "5", + "ksql.query.push.v2.enabled" : "false", + "ksql.transient.query.cleanup.service.enable" : "true", + "ksql.query.push.v2.metrics.enabled" : "true", + "ksql.rowpartition.rowoffset.enabled" : "true", + "ksql.streams.commit.interval.ms" : "2000", + "ksql.query.pull.table.scan.enabled" : "true", + "ksql.streams.auto.commit.interval.ms" : "0", + "ksql.streams.topology.optimization" : "all", + "ksql.endpoint.migrate.query" : "true", + "ksql.query.push.v2.registry.installed" : "false", + "ksql.streams.num.stream.threads" : "4", + "ksql.metrics.tags.custom" : "", + "ksql.query.push.v2.catchup.consumer.msg.window" : "50", + "ksql.runtime.feature.shared.enabled" : "false", + "ksql.udf.collect.metrics" : "false", + "ksql.new.query.planner.enabled" : "false", + "ksql.connect.request.headers.plugin" : null, + "ksql.security.extension.class" : null, + "ksql.transient.prefix" : "transient_", + "ksql.headers.columns.enabled" : "true", + "ksql.streams.default.deserialization.exception.handler" : "io.confluent.ksql.errors.LogMetricAndContinueExceptionHandler", + "ksql.connect.request.timeout.ms" : "5000", + "ksql.query.pull.enable.standby.reads" : "false", + "ksql.persistence.default.format.key" : "KAFKA", + "ksql.query.persistent.max.bytes.buffering.total" : "-1", + "ksql.query.error.max.queue.size" : "10", + "ksql.query.cleanup.shutdown.timeout.ms" : "30000", + "ksql.internal.topic.min.insync.replicas" : "1", + "ksql.internal.topic.replicas" : "1", + "ksql.insert.into.values.enabled" : "true", + "ksql.queryanonymizer.cluster_namespace" : null, + "ksql.create.or.replace.enabled" : "true", + "ksql.shared.runtimes.count" : "2", + "ksql.cast.strings.preserve.nulls" : "true", + "ksql.authorization.cache.max.entries" : "10000", + "ksql.pull.queries.enable" : "true", + "ksql.transient.query.cleanup.service.period.seconds" : "600", + "ksql.suppress.enabled" : "false", + "ksql.readonly.topics" : "_confluent.*,__confluent.*,_schemas,__consumer_offsets,__transaction_state,connect-configs,connect-offsets,connect-status,connect-statuses", + "ksql.connect.basic.auth.credentials.file" : "", + "ksql.authorization.cache.expiry.time.secs" : "30", + "ksql.query.retry.backoff.initial.ms" : "15000", + "ksql.query.pull.max.concurrent.requests" : "2147483647", + "ksql.streams.auto.offset.reset" : "earliest", + "ksql.connect.url" : "http://localhost:8083", + "ksql.query.push.v2.new.latest.delay.ms" : "5000", + "ksql.query.push.v2.latest.reset.age.ms" : "30000", + "ksql.streams.default.production.exception.handler" : "io.confluent.ksql.errors.ProductionExceptionHandlerUtil$LogAndFailProductionExceptionHandler", + "ksql.query.pull.interpreter.enabled" : "true", + "ksql.query.pull.limit.clause.enabled" : "true", + "ksql.query.pull.router.thread.pool.size" : "50", + "ksql.query.push.v2.continuation.tokens.enabled" : "false", + "ksql.query.retry.backoff.max.ms" : "900000", + "ksql.timestamp.throw.on.invalid" : "false", + "ksql.persistence.default.format.value" : null, + "ksql.udfs.enabled" : "true", + "ksql.udf.enable.security.manager" : "true", + "ksql.connect.worker.config" : "", + "ksql.nested.error.set.null" : "true", + "ksql.query.pull.thread.pool.size" : "50", + "ksql.persistent.prefix" : "query_", + "ksql.metastore.backup.location" : "", + "ksql.error.classifier.regex" : "", + "ksql.suppress.buffer.size.bytes" : "-1" + } +} \ No newline at end of file diff --git a/ksqldb-functional-tests/src/test/resources/historical_plans/histogram_-_histogram_on_a_table_-_PROTOBUF_NOSR/7.3.0_1651069869084/spec.json b/ksqldb-functional-tests/src/test/resources/historical_plans/histogram_-_histogram_on_a_table_-_PROTOBUF_NOSR/7.3.0_1651069869084/spec.json new file mode 100644 index 000000000000..6bda8a986763 --- /dev/null +++ b/ksqldb-functional-tests/src/test/resources/historical_plans/histogram_-_histogram_on_a_table_-_PROTOBUF_NOSR/7.3.0_1651069869084/spec.json @@ -0,0 +1,260 @@ +{ + "version" : "7.3.0", + "timestamp" : 1651069869084, + "path" : "query-validation-tests/histogram.json", + "schemas" : { + "CTAS_COUNT_BY_REGION_0.Aggregate.Project" : { + "schema" : "`REGION` STRING KEY, `COUNTS` MAP", + "keyFormat" : { + "format" : "KAFKA" + }, + "valueFormat" : { + "format" : "PROTOBUF_NOSR" + } + }, + "CTAS_COUNT_BY_REGION_0.Aggregate.GroupBy" : { + "schema" : "`REGION` STRING KEY, `REGION` STRING, `NAME` STRING", + "keyFormat" : { + "format" : "KAFKA" + }, + "valueFormat" : { + "format" : "PROTOBUF_NOSR" + } + }, + "CTAS_COUNT_BY_REGION_0.COUNT_BY_REGION" : { + "schema" : "`REGION` STRING KEY, `COUNTS` MAP", + "keyFormat" : { + "format" : "KAFKA" + }, + "valueFormat" : { + "format" : "PROTOBUF_NOSR" + } + }, + "CTAS_COUNT_BY_REGION_0.Aggregate.Prepare" : { + "schema" : "`K` STRING KEY, `REGION` STRING, `NAME` STRING", + "keyFormat" : { + "format" : "KAFKA" + }, + "valueFormat" : { + "format" : "PROTOBUF_NOSR" + } + }, + "CTAS_COUNT_BY_REGION_0.Aggregate.Aggregate.Materialize" : { + "schema" : "`REGION` STRING KEY, `REGION` STRING, `NAME` STRING, `KSQL_AGG_VARIABLE_0` MAP", + "keyFormat" : { + "format" : "KAFKA" + }, + "valueFormat" : { + "format" : "PROTOBUF_NOSR" + } + }, + "CTAS_COUNT_BY_REGION_0.KsqlTopic.Source" : { + "schema" : "`K` STRING KEY, `ID` BIGINT, `NAME` STRING, `REGION` STRING", + "keyFormat" : { + "format" : "KAFKA" + }, + "valueFormat" : { + "format" : "PROTOBUF_NOSR" + } + }, + "CTAS_COUNT_BY_REGION_0.KsqlTopic.Source.Materialized" : { + "schema" : "`K` STRING KEY, `ID` BIGINT, `NAME` STRING, `REGION` STRING, `ROWPARTITION` INTEGER, `ROWOFFSET` BIGINT", + "keyFormat" : { + "format" : "KAFKA" + }, + "valueFormat" : { + "format" : "PROTOBUF_NOSR" + } + } + }, + "testCase" : { + "name" : "histogram on a table - PROTOBUF_NOSR", + "inputs" : [ { + "topic" : "test_topic", + "key" : "0", + "value" : { + "ID" : 0, + "NAME" : "alice", + "REGION" : "east" + } + }, { + "topic" : "test_topic", + "key" : "1", + "value" : { + "ID" : 1, + "NAME" : "bob", + "REGION" : "east" + } + }, { + "topic" : "test_topic", + "key" : "2", + "value" : { + "ID" : 2, + "NAME" : "carol", + "REGION" : "west" + } + }, { + "topic" : "test_topic", + "key" : "3", + "value" : { + "ID" : 3, + "NAME" : "dave", + "REGION" : "west" + } + }, { + "topic" : "test_topic", + "key" : "1", + "value" : { + "ID" : 1, + "NAME" : "bob", + "REGION" : "west" + } + }, { + "topic" : "test_topic", + "key" : "1", + "value" : null + } ], + "outputs" : [ { + "topic" : "COUNT_BY_REGION", + "key" : "east", + "value" : { + "COUNTS" : { + "alice" : 1 + } + } + }, { + "topic" : "COUNT_BY_REGION", + "key" : "east", + "value" : { + "COUNTS" : { + "alice" : 1, + "bob" : 1 + } + } + }, { + "topic" : "COUNT_BY_REGION", + "key" : "west", + "value" : { + "COUNTS" : { + "carol" : 1 + } + } + }, { + "topic" : "COUNT_BY_REGION", + "key" : "west", + "value" : { + "COUNTS" : { + "carol" : 1, + "dave" : 1 + } + } + }, { + "topic" : "COUNT_BY_REGION", + "key" : "east", + "value" : { + "COUNTS" : { + "alice" : 1 + } + } + }, { + "topic" : "COUNT_BY_REGION", + "key" : "west", + "value" : { + "COUNTS" : { + "carol" : 1, + "dave" : 1, + "bob" : 1 + } + } + }, { + "topic" : "COUNT_BY_REGION", + "key" : "west", + "value" : { + "COUNTS" : { + "carol" : 1, + "dave" : 1 + } + } + } ], + "topics" : [ { + "name" : "test_topic", + "replicas" : 1, + "numPartitions" : 4 + }, { + "name" : "COUNT_BY_REGION", + "replicas" : 1, + "numPartitions" : 4 + } ], + "statements" : [ "CREATE TABLE TEST (K STRING PRIMARY KEY, ID bigint, NAME varchar, REGION string) WITH (kafka_topic='test_topic', value_format='PROTOBUF_NOSR');", "CREATE TABLE COUNT_BY_REGION AS SELECT region, histogram(name) AS COUNTS FROM TEST GROUP BY region;" ], + "post" : { + "sources" : [ { + "name" : "COUNT_BY_REGION", + "type" : "TABLE", + "schema" : "`REGION` STRING KEY, `COUNTS` MAP", + "keyFormat" : { + "format" : "KAFKA" + }, + "valueFormat" : "PROTOBUF_NOSR", + "keyFeatures" : [ ], + "valueFeatures" : [ ], + "isSource" : false + }, { + "name" : "TEST", + "type" : "TABLE", + "schema" : "`K` STRING KEY, `ID` BIGINT, `NAME` STRING, `REGION` STRING", + "keyFormat" : { + "format" : "KAFKA" + }, + "valueFormat" : "PROTOBUF_NOSR", + "keyFeatures" : [ ], + "valueFeatures" : [ ], + "isSource" : false + } ], + "topics" : { + "topics" : [ { + "name" : "_confluent-ksql-some.ksql.service.idquery_CTAS_COUNT_BY_REGION_0-Aggregate-GroupBy-repartition", + "keyFormat" : { + "format" : "KAFKA" + }, + "valueFormat" : { + "format" : "PROTOBUF_NOSR" + } + }, { + "name" : "test_topic", + "keyFormat" : { + "format" : "KAFKA" + }, + "valueFormat" : { + "format" : "PROTOBUF_NOSR" + }, + "partitions" : 4 + }, { + "name" : "COUNT_BY_REGION", + "keyFormat" : { + "format" : "KAFKA" + }, + "valueFormat" : { + "format" : "PROTOBUF_NOSR" + }, + "partitions" : 4 + }, { + "name" : "_confluent-ksql-some.ksql.service.idquery_CTAS_COUNT_BY_REGION_0-KsqlTopic-Reduce-changelog", + "keyFormat" : { + "format" : "KAFKA" + }, + "valueFormat" : { + "format" : "PROTOBUF_NOSR" + } + }, { + "name" : "_confluent-ksql-some.ksql.service.idquery_CTAS_COUNT_BY_REGION_0-Aggregate-Aggregate-Materialize-changelog", + "keyFormat" : { + "format" : "KAFKA" + }, + "valueFormat" : { + "format" : "PROTOBUF_NOSR" + } + } ] + } + } + } +} \ No newline at end of file diff --git a/ksqldb-functional-tests/src/test/resources/historical_plans/histogram_-_histogram_on_a_table_-_PROTOBUF_NOSR/7.3.0_1651069869084/topology b/ksqldb-functional-tests/src/test/resources/historical_plans/histogram_-_histogram_on_a_table_-_PROTOBUF_NOSR/7.3.0_1651069869084/topology new file mode 100644 index 000000000000..e720854b02aa --- /dev/null +++ b/ksqldb-functional-tests/src/test/resources/historical_plans/histogram_-_histogram_on_a_table_-_PROTOBUF_NOSR/7.3.0_1651069869084/topology @@ -0,0 +1,43 @@ +Topologies: + Sub-topology: 0 + Source: KSTREAM-SOURCE-0000000001 (topics: [test_topic]) + --> KTABLE-SOURCE-0000000002 + Processor: KTABLE-SOURCE-0000000002 (stores: []) + --> KTABLE-TRANSFORMVALUES-0000000003 + <-- KSTREAM-SOURCE-0000000001 + Processor: KTABLE-TRANSFORMVALUES-0000000003 (stores: [KsqlTopic-Reduce]) + --> KTABLE-TRANSFORMVALUES-0000000004 + <-- KTABLE-SOURCE-0000000002 + Processor: KTABLE-TRANSFORMVALUES-0000000004 (stores: []) + --> Aggregate-Prepare + <-- KTABLE-TRANSFORMVALUES-0000000003 + Processor: Aggregate-Prepare (stores: []) + --> KTABLE-FILTER-0000000006 + <-- KTABLE-TRANSFORMVALUES-0000000004 + Processor: KTABLE-FILTER-0000000006 (stores: []) + --> Aggregate-GroupBy + <-- Aggregate-Prepare + Processor: Aggregate-GroupBy (stores: []) + --> KSTREAM-SINK-0000000008 + <-- KTABLE-FILTER-0000000006 + Sink: KSTREAM-SINK-0000000008 (topic: Aggregate-GroupBy-repartition) + <-- Aggregate-GroupBy + + Sub-topology: 1 + Source: KSTREAM-SOURCE-0000000009 (topics: [Aggregate-GroupBy-repartition]) + --> KTABLE-AGGREGATE-0000000010 + Processor: KTABLE-AGGREGATE-0000000010 (stores: [Aggregate-Aggregate-Materialize]) + --> Aggregate-Aggregate-ToOutputSchema + <-- KSTREAM-SOURCE-0000000009 + Processor: Aggregate-Aggregate-ToOutputSchema (stores: []) + --> Aggregate-Project + <-- KTABLE-AGGREGATE-0000000010 + Processor: Aggregate-Project (stores: []) + --> KTABLE-TOSTREAM-0000000013 + <-- Aggregate-Aggregate-ToOutputSchema + Processor: KTABLE-TOSTREAM-0000000013 (stores: []) + --> KSTREAM-SINK-0000000014 + <-- Aggregate-Project + Sink: KSTREAM-SINK-0000000014 (topic: COUNT_BY_REGION) + <-- KTABLE-TOSTREAM-0000000013 + diff --git a/ksqldb-functional-tests/src/test/resources/historical_plans/histogram_-_histogram_string_-_PROTOBUF_NOSR/7.3.0_1651069862592/plan.json b/ksqldb-functional-tests/src/test/resources/historical_plans/histogram_-_histogram_string_-_PROTOBUF_NOSR/7.3.0_1651069862592/plan.json new file mode 100644 index 000000000000..6f81547c2e67 --- /dev/null +++ b/ksqldb-functional-tests/src/test/resources/historical_plans/histogram_-_histogram_string_-_PROTOBUF_NOSR/7.3.0_1651069862592/plan.json @@ -0,0 +1,255 @@ +{ + "plan" : [ { + "@type" : "ksqlPlanV1", + "statementText" : "CREATE STREAM TEST (ID BIGINT KEY, VALUE STRING) WITH (KAFKA_TOPIC='test_topic', KEY_FORMAT='KAFKA', VALUE_FORMAT='PROTOBUF_NOSR');", + "ddlCommand" : { + "@type" : "createStreamV1", + "sourceName" : "TEST", + "schema" : "`ID` BIGINT KEY, `VALUE` STRING", + "timestampColumn" : null, + "topicName" : "test_topic", + "formats" : { + "keyFormat" : { + "format" : "KAFKA", + "properties" : { } + }, + "valueFormat" : { + "format" : "PROTOBUF_NOSR", + "properties" : { } + } + }, + "windowInfo" : null, + "orReplace" : false, + "isSource" : false + }, + "queryPlan" : null + }, { + "@type" : "ksqlPlanV1", + "statementText" : "CREATE TABLE S2 AS SELECT\n TEST.ID ID,\n HISTOGRAM(TEST.VALUE) COUNTS\nFROM TEST TEST\nGROUP BY TEST.ID\nEMIT CHANGES", + "ddlCommand" : { + "@type" : "createTableV1", + "sourceName" : "S2", + "schema" : "`ID` BIGINT KEY, `COUNTS` MAP", + "timestampColumn" : null, + "topicName" : "S2", + "formats" : { + "keyFormat" : { + "format" : "KAFKA", + "properties" : { } + }, + "valueFormat" : { + "format" : "PROTOBUF_NOSR", + "properties" : { } + } + }, + "windowInfo" : null, + "orReplace" : false, + "isSource" : false + }, + "queryPlan" : { + "sources" : [ "TEST" ], + "sink" : "S2", + "physicalPlan" : { + "@type" : "tableSinkV1", + "properties" : { + "queryContext" : "S2" + }, + "source" : { + "@type" : "tableSelectV1", + "properties" : { + "queryContext" : "Aggregate/Project" + }, + "source" : { + "@type" : "streamAggregateV1", + "properties" : { + "queryContext" : "Aggregate/Aggregate" + }, + "source" : { + "@type" : "streamGroupByKeyV1", + "properties" : { + "queryContext" : "Aggregate/GroupBy" + }, + "source" : { + "@type" : "streamSelectV1", + "properties" : { + "queryContext" : "Aggregate/Prepare" + }, + "source" : { + "@type" : "streamSourceV1", + "properties" : { + "queryContext" : "KsqlTopic/Source" + }, + "topicName" : "test_topic", + "formats" : { + "keyFormat" : { + "format" : "KAFKA", + "properties" : { } + }, + "valueFormat" : { + "format" : "PROTOBUF_NOSR", + "properties" : { } + } + }, + "timestampColumn" : null, + "sourceSchema" : "`ID` BIGINT KEY, `VALUE` STRING", + "pseudoColumnVersion" : 1 + }, + "keyColumnNames" : [ "ID" ], + "selectedKeys" : null, + "selectExpressions" : [ "ID AS ID", "VALUE AS VALUE" ] + }, + "internalFormats" : { + "keyFormat" : { + "format" : "KAFKA", + "properties" : { } + }, + "valueFormat" : { + "format" : "PROTOBUF_NOSR", + "properties" : { } + } + } + }, + "internalFormats" : { + "keyFormat" : { + "format" : "KAFKA", + "properties" : { } + }, + "valueFormat" : { + "format" : "PROTOBUF_NOSR", + "properties" : { } + } + }, + "nonAggregateColumns" : [ "ID", "VALUE" ], + "aggregationFunctions" : [ "HISTOGRAM(VALUE)" ] + }, + "keyColumnNames" : [ "ID" ], + "selectExpressions" : [ "KSQL_AGG_VARIABLE_0 AS COUNTS" ], + "internalFormats" : { + "keyFormat" : { + "format" : "KAFKA", + "properties" : { } + }, + "valueFormat" : { + "format" : "PROTOBUF_NOSR", + "properties" : { } + } + } + }, + "formats" : { + "keyFormat" : { + "format" : "KAFKA", + "properties" : { } + }, + "valueFormat" : { + "format" : "PROTOBUF_NOSR", + "properties" : { } + } + }, + "topicName" : "S2", + "timestampColumn" : null + }, + "queryId" : "CTAS_S2_0", + "runtimeId" : null + } + } ], + "configs" : { + "ksql.extension.dir" : "ext", + "ksql.streams.cache.max.bytes.buffering" : "0", + "metric.reporters" : "", + "ksql.query.status.running.threshold.seconds" : "300", + "ksql.connect.basic.auth.credentials.reload" : "false", + "ksql.output.topic.name.prefix" : "", + "ksql.query.pull.stream.enabled" : "true", + "ksql.query.push.v2.interpreter.enabled" : "true", + "ksql.queryanonymizer.logs_enabled" : "true", + "ksql.variable.substitution.enable" : "true", + "ksql.streams.shutdown.timeout.ms" : "300000", + "ksql.query.pull.max.allowed.offset.lag" : "9223372036854775807", + "ksql.query.pull.max.qps" : "2147483647", + "ksql.access.validator.enable" : "auto", + "ksql.streams.bootstrap.servers" : "localhost:0", + "ksql.query.pull.metrics.enabled" : "true", + "ksql.metrics.extension" : null, + "ksql.query.push.v2.alos.enabled" : "true", + "ksql.query.push.v2.max.hourly.bandwidth.megabytes" : "2147483647", + "ksql.query.pull.range.scan.enabled" : "true", + "ksql.transient.query.cleanup.service.initial.delay.seconds" : "600", + "ksql.hidden.topics" : "_confluent.*,__confluent.*,_schemas,__consumer_offsets,__transaction_state,connect-configs,connect-offsets,connect-status,connect-statuses", + "ksql.lambdas.enabled" : "true", + "ksql.source.table.materialization.enabled" : "true", + "ksql.query.pull.max.hourly.bandwidth.megabytes" : "2147483647", + "ksql.sink.window.change.log.additional.retention" : "1000000", + "ksql.query.persistent.active.limit" : "2147483647", + "ksql.persistence.wrap.single.values" : null, + "ksql.query.transient.max.bytes.buffering.total" : "-1", + "ksql.connect.basic.auth.credentials.source" : "NONE", + "ksql.schema.registry.url" : "schema_registry.url:0", + "ksql.properties.overrides.denylist" : "", + "ksql.service.id" : "some.ksql.service.id", + "ksql.query.push.v2.max.catchup.consumers" : "5", + "ksql.query.push.v2.enabled" : "false", + "ksql.transient.query.cleanup.service.enable" : "true", + "ksql.query.push.v2.metrics.enabled" : "true", + "ksql.rowpartition.rowoffset.enabled" : "true", + "ksql.streams.commit.interval.ms" : "2000", + "ksql.query.pull.table.scan.enabled" : "true", + "ksql.streams.auto.commit.interval.ms" : "0", + "ksql.streams.topology.optimization" : "all", + "ksql.endpoint.migrate.query" : "true", + "ksql.query.push.v2.registry.installed" : "false", + "ksql.streams.num.stream.threads" : "4", + "ksql.metrics.tags.custom" : "", + "ksql.query.push.v2.catchup.consumer.msg.window" : "50", + "ksql.runtime.feature.shared.enabled" : "false", + "ksql.udf.collect.metrics" : "false", + "ksql.new.query.planner.enabled" : "false", + "ksql.connect.request.headers.plugin" : null, + "ksql.security.extension.class" : null, + "ksql.transient.prefix" : "transient_", + "ksql.headers.columns.enabled" : "true", + "ksql.streams.default.deserialization.exception.handler" : "io.confluent.ksql.errors.LogMetricAndContinueExceptionHandler", + "ksql.connect.request.timeout.ms" : "5000", + "ksql.query.pull.enable.standby.reads" : "false", + "ksql.persistence.default.format.key" : "KAFKA", + "ksql.query.persistent.max.bytes.buffering.total" : "-1", + "ksql.query.error.max.queue.size" : "10", + "ksql.query.cleanup.shutdown.timeout.ms" : "30000", + "ksql.internal.topic.min.insync.replicas" : "1", + "ksql.internal.topic.replicas" : "1", + "ksql.insert.into.values.enabled" : "true", + "ksql.queryanonymizer.cluster_namespace" : null, + "ksql.create.or.replace.enabled" : "true", + "ksql.shared.runtimes.count" : "2", + "ksql.cast.strings.preserve.nulls" : "true", + "ksql.authorization.cache.max.entries" : "10000", + "ksql.pull.queries.enable" : "true", + "ksql.transient.query.cleanup.service.period.seconds" : "600", + "ksql.suppress.enabled" : "false", + "ksql.readonly.topics" : "_confluent.*,__confluent.*,_schemas,__consumer_offsets,__transaction_state,connect-configs,connect-offsets,connect-status,connect-statuses", + "ksql.connect.basic.auth.credentials.file" : "", + "ksql.authorization.cache.expiry.time.secs" : "30", + "ksql.query.retry.backoff.initial.ms" : "15000", + "ksql.query.pull.max.concurrent.requests" : "2147483647", + "ksql.streams.auto.offset.reset" : "earliest", + "ksql.connect.url" : "http://localhost:8083", + "ksql.query.push.v2.new.latest.delay.ms" : "5000", + "ksql.query.push.v2.latest.reset.age.ms" : "30000", + "ksql.streams.default.production.exception.handler" : "io.confluent.ksql.errors.ProductionExceptionHandlerUtil$LogAndFailProductionExceptionHandler", + "ksql.query.pull.interpreter.enabled" : "true", + "ksql.query.pull.limit.clause.enabled" : "true", + "ksql.query.pull.router.thread.pool.size" : "50", + "ksql.query.push.v2.continuation.tokens.enabled" : "false", + "ksql.query.retry.backoff.max.ms" : "900000", + "ksql.timestamp.throw.on.invalid" : "false", + "ksql.persistence.default.format.value" : null, + "ksql.udfs.enabled" : "true", + "ksql.udf.enable.security.manager" : "true", + "ksql.connect.worker.config" : "", + "ksql.nested.error.set.null" : "true", + "ksql.query.pull.thread.pool.size" : "50", + "ksql.persistent.prefix" : "query_", + "ksql.metastore.backup.location" : "", + "ksql.error.classifier.regex" : "", + "ksql.suppress.buffer.size.bytes" : "-1" + } +} \ No newline at end of file diff --git a/ksqldb-functional-tests/src/test/resources/historical_plans/histogram_-_histogram_string_-_PROTOBUF_NOSR/7.3.0_1651069862592/spec.json b/ksqldb-functional-tests/src/test/resources/historical_plans/histogram_-_histogram_string_-_PROTOBUF_NOSR/7.3.0_1651069862592/spec.json new file mode 100644 index 000000000000..c1b8f8579fb9 --- /dev/null +++ b/ksqldb-functional-tests/src/test/resources/historical_plans/histogram_-_histogram_string_-_PROTOBUF_NOSR/7.3.0_1651069862592/spec.json @@ -0,0 +1,193 @@ +{ + "version" : "7.3.0", + "timestamp" : 1651069862592, + "path" : "query-validation-tests/histogram.json", + "schemas" : { + "CTAS_S2_0.Aggregate.Aggregate.Materialize" : { + "schema" : "`ID` BIGINT KEY, `ID` BIGINT, `VALUE` STRING, `KSQL_AGG_VARIABLE_0` MAP", + "keyFormat" : { + "format" : "KAFKA" + }, + "valueFormat" : { + "format" : "PROTOBUF_NOSR" + } + }, + "CTAS_S2_0.Aggregate.Project" : { + "schema" : "`ID` BIGINT KEY, `COUNTS` MAP", + "keyFormat" : { + "format" : "KAFKA" + }, + "valueFormat" : { + "format" : "PROTOBUF_NOSR" + } + }, + "CTAS_S2_0.KsqlTopic.Source" : { + "schema" : "`ID` BIGINT KEY, `VALUE` STRING", + "keyFormat" : { + "format" : "KAFKA" + }, + "valueFormat" : { + "format" : "PROTOBUF_NOSR" + } + }, + "CTAS_S2_0.S2" : { + "schema" : "`ID` BIGINT KEY, `COUNTS` MAP", + "keyFormat" : { + "format" : "KAFKA" + }, + "valueFormat" : { + "format" : "PROTOBUF_NOSR" + } + }, + "CTAS_S2_0.Aggregate.GroupBy" : { + "schema" : "`ID` BIGINT KEY, `ID` BIGINT, `VALUE` STRING", + "keyFormat" : { + "format" : "KAFKA" + }, + "valueFormat" : { + "format" : "PROTOBUF_NOSR" + } + } + }, + "testCase" : { + "name" : "histogram string - PROTOBUF_NOSR", + "inputs" : [ { + "topic" : "test_topic", + "key" : 0, + "value" : { + "VALUE" : "foo" + } + }, { + "topic" : "test_topic", + "key" : 100, + "value" : { + "VALUE" : "baz" + } + }, { + "topic" : "test_topic", + "key" : 0, + "value" : { + "VALUE" : "bar" + } + }, { + "topic" : "test_topic", + "key" : 100, + "value" : { + "VALUE" : "baz" + } + }, { + "topic" : "test_topic", + "key" : 100, + "value" : { + "VALUE" : "foo" + } + } ], + "outputs" : [ { + "topic" : "S2", + "key" : 0, + "value" : { + "COUNTS" : { + "foo" : 1 + } + } + }, { + "topic" : "S2", + "key" : 100, + "value" : { + "COUNTS" : { + "baz" : 1 + } + } + }, { + "topic" : "S2", + "key" : 0, + "value" : { + "COUNTS" : { + "foo" : 1, + "bar" : 1 + } + } + }, { + "topic" : "S2", + "key" : 100, + "value" : { + "COUNTS" : { + "baz" : 2 + } + } + }, { + "topic" : "S2", + "key" : 100, + "value" : { + "COUNTS" : { + "baz" : 2, + "foo" : 1 + } + } + } ], + "topics" : [ { + "name" : "test_topic", + "replicas" : 1, + "numPartitions" : 4 + }, { + "name" : "S2", + "replicas" : 1, + "numPartitions" : 4 + } ], + "statements" : [ "CREATE STREAM TEST (ID BIGINT KEY, VALUE varchar) WITH (kafka_topic='test_topic', value_format='PROTOBUF_NOSR');", "CREATE TABLE S2 as SELECT ID, histogram(value) as counts FROM test group by id;" ], + "post" : { + "sources" : [ { + "name" : "S2", + "type" : "TABLE", + "schema" : "`ID` BIGINT KEY, `COUNTS` MAP", + "keyFormat" : { + "format" : "KAFKA" + }, + "valueFormat" : "PROTOBUF_NOSR", + "keyFeatures" : [ ], + "valueFeatures" : [ ], + "isSource" : false + }, { + "name" : "TEST", + "type" : "STREAM", + "schema" : "`ID` BIGINT KEY, `VALUE` STRING", + "keyFormat" : { + "format" : "KAFKA" + }, + "valueFormat" : "PROTOBUF_NOSR", + "keyFeatures" : [ ], + "valueFeatures" : [ ], + "isSource" : false + } ], + "topics" : { + "topics" : [ { + "name" : "_confluent-ksql-some.ksql.service.idquery_CTAS_S2_0-Aggregate-Aggregate-Materialize-changelog", + "keyFormat" : { + "format" : "KAFKA" + }, + "valueFormat" : { + "format" : "PROTOBUF_NOSR" + } + }, { + "name" : "test_topic", + "keyFormat" : { + "format" : "KAFKA" + }, + "valueFormat" : { + "format" : "PROTOBUF_NOSR" + }, + "partitions" : 4 + }, { + "name" : "S2", + "keyFormat" : { + "format" : "KAFKA" + }, + "valueFormat" : { + "format" : "PROTOBUF_NOSR" + }, + "partitions" : 4 + } ] + } + } + } +} \ No newline at end of file diff --git a/ksqldb-functional-tests/src/test/resources/historical_plans/histogram_-_histogram_string_-_PROTOBUF_NOSR/7.3.0_1651069862592/topology b/ksqldb-functional-tests/src/test/resources/historical_plans/histogram_-_histogram_string_-_PROTOBUF_NOSR/7.3.0_1651069862592/topology new file mode 100644 index 000000000000..fcfc8fb28665 --- /dev/null +++ b/ksqldb-functional-tests/src/test/resources/historical_plans/histogram_-_histogram_string_-_PROTOBUF_NOSR/7.3.0_1651069862592/topology @@ -0,0 +1,25 @@ +Topologies: + Sub-topology: 0 + Source: KSTREAM-SOURCE-0000000000 (topics: [test_topic]) + --> KSTREAM-TRANSFORMVALUES-0000000001 + Processor: KSTREAM-TRANSFORMVALUES-0000000001 (stores: []) + --> Aggregate-Prepare + <-- KSTREAM-SOURCE-0000000000 + Processor: Aggregate-Prepare (stores: []) + --> KSTREAM-AGGREGATE-0000000003 + <-- KSTREAM-TRANSFORMVALUES-0000000001 + Processor: KSTREAM-AGGREGATE-0000000003 (stores: [Aggregate-Aggregate-Materialize]) + --> Aggregate-Aggregate-ToOutputSchema + <-- Aggregate-Prepare + Processor: Aggregate-Aggregate-ToOutputSchema (stores: []) + --> Aggregate-Project + <-- KSTREAM-AGGREGATE-0000000003 + Processor: Aggregate-Project (stores: []) + --> KTABLE-TOSTREAM-0000000006 + <-- Aggregate-Aggregate-ToOutputSchema + Processor: KTABLE-TOSTREAM-0000000006 (stores: []) + --> KSTREAM-SINK-0000000007 + <-- Aggregate-Project + Sink: KSTREAM-SINK-0000000007 (topic: S2) + <-- KTABLE-TOSTREAM-0000000006 + diff --git a/ksqldb-functional-tests/src/test/resources/historical_plans/initcap_-_do_initcap_-_AVRO/7.3.0_1651069881394/plan.json b/ksqldb-functional-tests/src/test/resources/historical_plans/initcap_-_do_initcap_-_AVRO/7.3.0_1651069881394/plan.json new file mode 100644 index 000000000000..38a6b8774d9c --- /dev/null +++ b/ksqldb-functional-tests/src/test/resources/historical_plans/initcap_-_do_initcap_-_AVRO/7.3.0_1651069881394/plan.json @@ -0,0 +1,203 @@ +{ + "plan" : [ { + "@type" : "ksqlPlanV1", + "statementText" : "CREATE STREAM TEST (K STRING KEY, SOURCE STRING) WITH (KAFKA_TOPIC='test_topic', KEY_FORMAT='KAFKA', VALUE_FORMAT='AVRO');", + "ddlCommand" : { + "@type" : "createStreamV1", + "sourceName" : "TEST", + "schema" : "`K` STRING KEY, `SOURCE` STRING", + "timestampColumn" : null, + "topicName" : "test_topic", + "formats" : { + "keyFormat" : { + "format" : "KAFKA", + "properties" : { } + }, + "valueFormat" : { + "format" : "AVRO", + "properties" : { } + } + }, + "windowInfo" : null, + "orReplace" : false, + "isSource" : false + }, + "queryPlan" : null + }, { + "@type" : "ksqlPlanV1", + "statementText" : "CREATE STREAM OUTPUT AS SELECT\n TEST.K K,\n INITCAP(TEST.SOURCE) INITCAP\nFROM TEST TEST\nEMIT CHANGES", + "ddlCommand" : { + "@type" : "createStreamV1", + "sourceName" : "OUTPUT", + "schema" : "`K` STRING KEY, `INITCAP` STRING", + "timestampColumn" : null, + "topicName" : "OUTPUT", + "formats" : { + "keyFormat" : { + "format" : "KAFKA", + "properties" : { } + }, + "valueFormat" : { + "format" : "AVRO", + "properties" : { } + } + }, + "windowInfo" : null, + "orReplace" : false, + "isSource" : false + }, + "queryPlan" : { + "sources" : [ "TEST" ], + "sink" : "OUTPUT", + "physicalPlan" : { + "@type" : "streamSinkV1", + "properties" : { + "queryContext" : "OUTPUT" + }, + "source" : { + "@type" : "streamSelectV1", + "properties" : { + "queryContext" : "Project" + }, + "source" : { + "@type" : "streamSourceV1", + "properties" : { + "queryContext" : "KsqlTopic/Source" + }, + "topicName" : "test_topic", + "formats" : { + "keyFormat" : { + "format" : "KAFKA", + "properties" : { } + }, + "valueFormat" : { + "format" : "AVRO", + "properties" : { } + } + }, + "timestampColumn" : null, + "sourceSchema" : "`K` STRING KEY, `SOURCE` STRING", + "pseudoColumnVersion" : 1 + }, + "keyColumnNames" : [ "K" ], + "selectedKeys" : null, + "selectExpressions" : [ "INITCAP(SOURCE) AS INITCAP" ] + }, + "formats" : { + "keyFormat" : { + "format" : "KAFKA", + "properties" : { } + }, + "valueFormat" : { + "format" : "AVRO", + "properties" : { } + } + }, + "topicName" : "OUTPUT", + "timestampColumn" : null + }, + "queryId" : "CSAS_OUTPUT_0", + "runtimeId" : null + } + } ], + "configs" : { + "ksql.extension.dir" : "ext", + "ksql.streams.cache.max.bytes.buffering" : "0", + "metric.reporters" : "", + "ksql.query.status.running.threshold.seconds" : "300", + "ksql.connect.basic.auth.credentials.reload" : "false", + "ksql.output.topic.name.prefix" : "", + "ksql.query.pull.stream.enabled" : "true", + "ksql.query.push.v2.interpreter.enabled" : "true", + "ksql.queryanonymizer.logs_enabled" : "true", + "ksql.variable.substitution.enable" : "true", + "ksql.streams.shutdown.timeout.ms" : "300000", + "ksql.query.pull.max.allowed.offset.lag" : "9223372036854775807", + "ksql.query.pull.max.qps" : "2147483647", + "ksql.access.validator.enable" : "auto", + "ksql.streams.bootstrap.servers" : "localhost:0", + "ksql.query.pull.metrics.enabled" : "true", + "ksql.metrics.extension" : null, + "ksql.query.push.v2.alos.enabled" : "true", + "ksql.query.push.v2.max.hourly.bandwidth.megabytes" : "2147483647", + "ksql.query.pull.range.scan.enabled" : "true", + "ksql.transient.query.cleanup.service.initial.delay.seconds" : "600", + "ksql.hidden.topics" : "_confluent.*,__confluent.*,_schemas,__consumer_offsets,__transaction_state,connect-configs,connect-offsets,connect-status,connect-statuses", + "ksql.lambdas.enabled" : "true", + "ksql.source.table.materialization.enabled" : "true", + "ksql.query.pull.max.hourly.bandwidth.megabytes" : "2147483647", + "ksql.sink.window.change.log.additional.retention" : "1000000", + "ksql.query.persistent.active.limit" : "2147483647", + "ksql.persistence.wrap.single.values" : null, + "ksql.query.transient.max.bytes.buffering.total" : "-1", + "ksql.connect.basic.auth.credentials.source" : "NONE", + "ksql.schema.registry.url" : "schema_registry.url:0", + "ksql.properties.overrides.denylist" : "", + "ksql.service.id" : "some.ksql.service.id", + "ksql.query.push.v2.max.catchup.consumers" : "5", + "ksql.query.push.v2.enabled" : "false", + "ksql.transient.query.cleanup.service.enable" : "true", + "ksql.query.push.v2.metrics.enabled" : "true", + "ksql.rowpartition.rowoffset.enabled" : "true", + "ksql.streams.commit.interval.ms" : "2000", + "ksql.query.pull.table.scan.enabled" : "true", + "ksql.streams.auto.commit.interval.ms" : "0", + "ksql.streams.topology.optimization" : "all", + "ksql.endpoint.migrate.query" : "true", + "ksql.query.push.v2.registry.installed" : "false", + "ksql.streams.num.stream.threads" : "4", + "ksql.metrics.tags.custom" : "", + "ksql.query.push.v2.catchup.consumer.msg.window" : "50", + "ksql.runtime.feature.shared.enabled" : "false", + "ksql.udf.collect.metrics" : "false", + "ksql.new.query.planner.enabled" : "false", + "ksql.connect.request.headers.plugin" : null, + "ksql.security.extension.class" : null, + "ksql.transient.prefix" : "transient_", + "ksql.headers.columns.enabled" : "true", + "ksql.streams.default.deserialization.exception.handler" : "io.confluent.ksql.errors.LogMetricAndContinueExceptionHandler", + "ksql.connect.request.timeout.ms" : "5000", + "ksql.query.pull.enable.standby.reads" : "false", + "ksql.persistence.default.format.key" : "KAFKA", + "ksql.query.persistent.max.bytes.buffering.total" : "-1", + "ksql.query.error.max.queue.size" : "10", + "ksql.query.cleanup.shutdown.timeout.ms" : "30000", + "ksql.internal.topic.min.insync.replicas" : "1", + "ksql.internal.topic.replicas" : "1", + "ksql.insert.into.values.enabled" : "true", + "ksql.queryanonymizer.cluster_namespace" : null, + "ksql.create.or.replace.enabled" : "true", + "ksql.shared.runtimes.count" : "2", + "ksql.cast.strings.preserve.nulls" : "true", + "ksql.authorization.cache.max.entries" : "10000", + "ksql.pull.queries.enable" : "true", + "ksql.transient.query.cleanup.service.period.seconds" : "600", + "ksql.suppress.enabled" : "false", + "ksql.readonly.topics" : "_confluent.*,__confluent.*,_schemas,__consumer_offsets,__transaction_state,connect-configs,connect-offsets,connect-status,connect-statuses", + "ksql.connect.basic.auth.credentials.file" : "", + "ksql.authorization.cache.expiry.time.secs" : "30", + "ksql.query.retry.backoff.initial.ms" : "15000", + "ksql.query.pull.max.concurrent.requests" : "2147483647", + "ksql.streams.auto.offset.reset" : "earliest", + "ksql.connect.url" : "http://localhost:8083", + "ksql.query.push.v2.new.latest.delay.ms" : "5000", + "ksql.query.push.v2.latest.reset.age.ms" : "30000", + "ksql.streams.default.production.exception.handler" : "io.confluent.ksql.errors.ProductionExceptionHandlerUtil$LogAndFailProductionExceptionHandler", + "ksql.query.pull.interpreter.enabled" : "true", + "ksql.query.pull.limit.clause.enabled" : "true", + "ksql.query.pull.router.thread.pool.size" : "50", + "ksql.query.push.v2.continuation.tokens.enabled" : "false", + "ksql.query.retry.backoff.max.ms" : "900000", + "ksql.timestamp.throw.on.invalid" : "false", + "ksql.persistence.default.format.value" : null, + "ksql.udfs.enabled" : "true", + "ksql.udf.enable.security.manager" : "true", + "ksql.connect.worker.config" : "", + "ksql.nested.error.set.null" : "true", + "ksql.query.pull.thread.pool.size" : "50", + "ksql.persistent.prefix" : "query_", + "ksql.metastore.backup.location" : "", + "ksql.error.classifier.regex" : "", + "ksql.suppress.buffer.size.bytes" : "-1" + } +} \ No newline at end of file diff --git a/ksqldb-functional-tests/src/test/resources/historical_plans/initcap_-_do_initcap_-_AVRO/7.3.0_1651069881394/spec.json b/ksqldb-functional-tests/src/test/resources/historical_plans/initcap_-_do_initcap_-_AVRO/7.3.0_1651069881394/spec.json new file mode 100644 index 000000000000..d1436c7f24af --- /dev/null +++ b/ksqldb-functional-tests/src/test/resources/historical_plans/initcap_-_do_initcap_-_AVRO/7.3.0_1651069881394/spec.json @@ -0,0 +1,158 @@ +{ + "version" : "7.3.0", + "timestamp" : 1651069881394, + "path" : "query-validation-tests/initcap.json", + "schemas" : { + "CSAS_OUTPUT_0.KsqlTopic.Source" : { + "schema" : "`K` STRING KEY, `SOURCE` STRING", + "keyFormat" : { + "format" : "KAFKA" + }, + "valueFormat" : { + "format" : "AVRO" + } + }, + "CSAS_OUTPUT_0.OUTPUT" : { + "schema" : "`K` STRING KEY, `INITCAP` STRING", + "keyFormat" : { + "format" : "KAFKA" + }, + "valueFormat" : { + "format" : "AVRO" + } + } + }, + "testCase" : { + "name" : "do initcap - AVRO", + "inputs" : [ { + "topic" : "test_topic", + "key" : null, + "value" : { + "source" : "some_string" + } + }, { + "topic" : "test_topic", + "key" : null, + "value" : { + "source" : null + } + }, { + "topic" : "test_topic", + "key" : null, + "value" : { + "source" : "the Quick br0wn fOx" + } + } ], + "outputs" : [ { + "topic" : "OUTPUT", + "key" : null, + "value" : { + "INITCAP" : "Some_string" + } + }, { + "topic" : "OUTPUT", + "key" : null, + "value" : { + "INITCAP" : null + } + }, { + "topic" : "OUTPUT", + "key" : null, + "value" : { + "INITCAP" : "The Quick Br0wn Fox" + } + } ], + "topics" : [ { + "name" : "OUTPUT", + "replicas" : 1, + "numPartitions" : 4 + }, { + "name" : "test_topic", + "valueSchema" : { + "type" : "record", + "name" : "KsqlDataSourceSchema", + "namespace" : "io.confluent.ksql.avro_schemas", + "fields" : [ { + "name" : "SOURCE", + "type" : [ "null", "string" ], + "default" : null + } ], + "connect.name" : "io.confluent.ksql.avro_schemas.KsqlDataSourceSchema" + }, + "valueFormat" : "AVRO", + "replicas" : 1, + "numPartitions" : 4 + } ], + "statements" : [ "CREATE STREAM TEST (K STRING KEY, source VARCHAR) WITH (kafka_topic='test_topic', value_format='AVRO');", "CREATE STREAM OUTPUT AS SELECT K, INITCAP(source) AS INITCAP FROM TEST;" ], + "properties" : { + "ksql.functions.substring.legacy.args" : false + }, + "post" : { + "sources" : [ { + "name" : "OUTPUT", + "type" : "STREAM", + "schema" : "`K` STRING KEY, `INITCAP` STRING", + "keyFormat" : { + "format" : "KAFKA" + }, + "valueFormat" : "AVRO", + "keyFeatures" : [ ], + "valueFeatures" : [ ], + "isSource" : false + }, { + "name" : "TEST", + "type" : "STREAM", + "schema" : "`K` STRING KEY, `SOURCE` STRING", + "keyFormat" : { + "format" : "KAFKA" + }, + "valueFormat" : "AVRO", + "keyFeatures" : [ ], + "valueFeatures" : [ ], + "isSource" : false + } ], + "topics" : { + "topics" : [ { + "name" : "test_topic", + "keyFormat" : { + "format" : "KAFKA" + }, + "valueFormat" : { + "format" : "AVRO" + }, + "partitions" : 4, + "valueSchema" : { + "type" : "record", + "name" : "KsqlDataSourceSchema", + "namespace" : "io.confluent.ksql.avro_schemas", + "fields" : [ { + "name" : "SOURCE", + "type" : [ "null", "string" ], + "default" : null + } ], + "connect.name" : "io.confluent.ksql.avro_schemas.KsqlDataSourceSchema" + } + }, { + "name" : "OUTPUT", + "keyFormat" : { + "format" : "KAFKA" + }, + "valueFormat" : { + "format" : "AVRO" + }, + "partitions" : 4, + "valueSchema" : { + "type" : "record", + "name" : "KsqlDataSourceSchema", + "namespace" : "io.confluent.ksql.avro_schemas", + "fields" : [ { + "name" : "INITCAP", + "type" : [ "null", "string" ], + "default" : null + } ] + } + } ] + } + } + } +} \ No newline at end of file diff --git a/ksqldb-functional-tests/src/test/resources/historical_plans/initcap_-_do_initcap_-_AVRO/7.3.0_1651069881394/topology b/ksqldb-functional-tests/src/test/resources/historical_plans/initcap_-_do_initcap_-_AVRO/7.3.0_1651069881394/topology new file mode 100644 index 000000000000..441a8f282644 --- /dev/null +++ b/ksqldb-functional-tests/src/test/resources/historical_plans/initcap_-_do_initcap_-_AVRO/7.3.0_1651069881394/topology @@ -0,0 +1,13 @@ +Topologies: + Sub-topology: 0 + Source: KSTREAM-SOURCE-0000000000 (topics: [test_topic]) + --> KSTREAM-TRANSFORMVALUES-0000000001 + Processor: KSTREAM-TRANSFORMVALUES-0000000001 (stores: []) + --> Project + <-- KSTREAM-SOURCE-0000000000 + Processor: Project (stores: []) + --> KSTREAM-SINK-0000000003 + <-- KSTREAM-TRANSFORMVALUES-0000000001 + Sink: KSTREAM-SINK-0000000003 (topic: OUTPUT) + <-- Project + diff --git a/ksqldb-functional-tests/src/test/resources/historical_plans/initcap_-_do_initcap_-_PROTOBUFs_-_PROTOBUF/7.3.0_1651069881416/plan.json b/ksqldb-functional-tests/src/test/resources/historical_plans/initcap_-_do_initcap_-_PROTOBUFs_-_PROTOBUF/7.3.0_1651069881416/plan.json new file mode 100644 index 000000000000..048f18962f22 --- /dev/null +++ b/ksqldb-functional-tests/src/test/resources/historical_plans/initcap_-_do_initcap_-_PROTOBUFs_-_PROTOBUF/7.3.0_1651069881416/plan.json @@ -0,0 +1,211 @@ +{ + "plan" : [ { + "@type" : "ksqlPlanV1", + "statementText" : "CREATE STREAM TEST (K STRING KEY, SOURCE STRING) WITH (KAFKA_TOPIC='test_topic', KEY_FORMAT='KAFKA', VALUE_FORMAT='PROTOBUF');", + "ddlCommand" : { + "@type" : "createStreamV1", + "sourceName" : "TEST", + "schema" : "`K` STRING KEY, `SOURCE` STRING", + "timestampColumn" : null, + "topicName" : "test_topic", + "formats" : { + "keyFormat" : { + "format" : "KAFKA", + "properties" : { } + }, + "valueFormat" : { + "format" : "PROTOBUF", + "properties" : { + "unwrapPrimitives" : "true" + } + } + }, + "windowInfo" : null, + "orReplace" : false, + "isSource" : false + }, + "queryPlan" : null + }, { + "@type" : "ksqlPlanV1", + "statementText" : "CREATE STREAM OUTPUT AS SELECT\n TEST.K K,\n INITCAP(TEST.SOURCE) INITCAP\nFROM TEST TEST\nEMIT CHANGES", + "ddlCommand" : { + "@type" : "createStreamV1", + "sourceName" : "OUTPUT", + "schema" : "`K` STRING KEY, `INITCAP` STRING", + "timestampColumn" : null, + "topicName" : "OUTPUT", + "formats" : { + "keyFormat" : { + "format" : "KAFKA", + "properties" : { } + }, + "valueFormat" : { + "format" : "PROTOBUF", + "properties" : { + "unwrapPrimitives" : "true" + } + } + }, + "windowInfo" : null, + "orReplace" : false, + "isSource" : false + }, + "queryPlan" : { + "sources" : [ "TEST" ], + "sink" : "OUTPUT", + "physicalPlan" : { + "@type" : "streamSinkV1", + "properties" : { + "queryContext" : "OUTPUT" + }, + "source" : { + "@type" : "streamSelectV1", + "properties" : { + "queryContext" : "Project" + }, + "source" : { + "@type" : "streamSourceV1", + "properties" : { + "queryContext" : "KsqlTopic/Source" + }, + "topicName" : "test_topic", + "formats" : { + "keyFormat" : { + "format" : "KAFKA", + "properties" : { } + }, + "valueFormat" : { + "format" : "PROTOBUF", + "properties" : { + "unwrapPrimitives" : "true" + } + } + }, + "timestampColumn" : null, + "sourceSchema" : "`K` STRING KEY, `SOURCE` STRING", + "pseudoColumnVersion" : 1 + }, + "keyColumnNames" : [ "K" ], + "selectedKeys" : null, + "selectExpressions" : [ "INITCAP(SOURCE) AS INITCAP" ] + }, + "formats" : { + "keyFormat" : { + "format" : "KAFKA", + "properties" : { } + }, + "valueFormat" : { + "format" : "PROTOBUF", + "properties" : { + "unwrapPrimitives" : "true" + } + } + }, + "topicName" : "OUTPUT", + "timestampColumn" : null + }, + "queryId" : "CSAS_OUTPUT_0", + "runtimeId" : null + } + } ], + "configs" : { + "ksql.extension.dir" : "ext", + "ksql.streams.cache.max.bytes.buffering" : "0", + "metric.reporters" : "", + "ksql.query.status.running.threshold.seconds" : "300", + "ksql.connect.basic.auth.credentials.reload" : "false", + "ksql.output.topic.name.prefix" : "", + "ksql.query.pull.stream.enabled" : "true", + "ksql.query.push.v2.interpreter.enabled" : "true", + "ksql.queryanonymizer.logs_enabled" : "true", + "ksql.variable.substitution.enable" : "true", + "ksql.streams.shutdown.timeout.ms" : "300000", + "ksql.query.pull.max.allowed.offset.lag" : "9223372036854775807", + "ksql.query.pull.max.qps" : "2147483647", + "ksql.access.validator.enable" : "auto", + "ksql.streams.bootstrap.servers" : "localhost:0", + "ksql.query.pull.metrics.enabled" : "true", + "ksql.metrics.extension" : null, + "ksql.query.push.v2.alos.enabled" : "true", + "ksql.query.push.v2.max.hourly.bandwidth.megabytes" : "2147483647", + "ksql.query.pull.range.scan.enabled" : "true", + "ksql.transient.query.cleanup.service.initial.delay.seconds" : "600", + "ksql.hidden.topics" : "_confluent.*,__confluent.*,_schemas,__consumer_offsets,__transaction_state,connect-configs,connect-offsets,connect-status,connect-statuses", + "ksql.lambdas.enabled" : "true", + "ksql.source.table.materialization.enabled" : "true", + "ksql.query.pull.max.hourly.bandwidth.megabytes" : "2147483647", + "ksql.sink.window.change.log.additional.retention" : "1000000", + "ksql.query.persistent.active.limit" : "2147483647", + "ksql.persistence.wrap.single.values" : null, + "ksql.query.transient.max.bytes.buffering.total" : "-1", + "ksql.connect.basic.auth.credentials.source" : "NONE", + "ksql.schema.registry.url" : "schema_registry.url:0", + "ksql.properties.overrides.denylist" : "", + "ksql.service.id" : "some.ksql.service.id", + "ksql.query.push.v2.max.catchup.consumers" : "5", + "ksql.query.push.v2.enabled" : "false", + "ksql.transient.query.cleanup.service.enable" : "true", + "ksql.query.push.v2.metrics.enabled" : "true", + "ksql.rowpartition.rowoffset.enabled" : "true", + "ksql.streams.commit.interval.ms" : "2000", + "ksql.query.pull.table.scan.enabled" : "true", + "ksql.streams.auto.commit.interval.ms" : "0", + "ksql.streams.topology.optimization" : "all", + "ksql.endpoint.migrate.query" : "true", + "ksql.query.push.v2.registry.installed" : "false", + "ksql.streams.num.stream.threads" : "4", + "ksql.metrics.tags.custom" : "", + "ksql.query.push.v2.catchup.consumer.msg.window" : "50", + "ksql.runtime.feature.shared.enabled" : "false", + "ksql.udf.collect.metrics" : "false", + "ksql.new.query.planner.enabled" : "false", + "ksql.connect.request.headers.plugin" : null, + "ksql.security.extension.class" : null, + "ksql.transient.prefix" : "transient_", + "ksql.headers.columns.enabled" : "true", + "ksql.streams.default.deserialization.exception.handler" : "io.confluent.ksql.errors.LogMetricAndContinueExceptionHandler", + "ksql.connect.request.timeout.ms" : "5000", + "ksql.query.pull.enable.standby.reads" : "false", + "ksql.persistence.default.format.key" : "KAFKA", + "ksql.query.persistent.max.bytes.buffering.total" : "-1", + "ksql.query.error.max.queue.size" : "10", + "ksql.query.cleanup.shutdown.timeout.ms" : "30000", + "ksql.internal.topic.min.insync.replicas" : "1", + "ksql.internal.topic.replicas" : "1", + "ksql.insert.into.values.enabled" : "true", + "ksql.queryanonymizer.cluster_namespace" : null, + "ksql.create.or.replace.enabled" : "true", + "ksql.shared.runtimes.count" : "2", + "ksql.cast.strings.preserve.nulls" : "true", + "ksql.authorization.cache.max.entries" : "10000", + "ksql.pull.queries.enable" : "true", + "ksql.transient.query.cleanup.service.period.seconds" : "600", + "ksql.suppress.enabled" : "false", + "ksql.readonly.topics" : "_confluent.*,__confluent.*,_schemas,__consumer_offsets,__transaction_state,connect-configs,connect-offsets,connect-status,connect-statuses", + "ksql.connect.basic.auth.credentials.file" : "", + "ksql.authorization.cache.expiry.time.secs" : "30", + "ksql.query.retry.backoff.initial.ms" : "15000", + "ksql.query.pull.max.concurrent.requests" : "2147483647", + "ksql.streams.auto.offset.reset" : "earliest", + "ksql.connect.url" : "http://localhost:8083", + "ksql.query.push.v2.new.latest.delay.ms" : "5000", + "ksql.query.push.v2.latest.reset.age.ms" : "30000", + "ksql.streams.default.production.exception.handler" : "io.confluent.ksql.errors.ProductionExceptionHandlerUtil$LogAndFailProductionExceptionHandler", + "ksql.query.pull.interpreter.enabled" : "true", + "ksql.query.pull.limit.clause.enabled" : "true", + "ksql.query.pull.router.thread.pool.size" : "50", + "ksql.query.push.v2.continuation.tokens.enabled" : "false", + "ksql.query.retry.backoff.max.ms" : "900000", + "ksql.timestamp.throw.on.invalid" : "false", + "ksql.persistence.default.format.value" : null, + "ksql.udfs.enabled" : "true", + "ksql.udf.enable.security.manager" : "true", + "ksql.connect.worker.config" : "", + "ksql.nested.error.set.null" : "true", + "ksql.query.pull.thread.pool.size" : "50", + "ksql.persistent.prefix" : "query_", + "ksql.metastore.backup.location" : "", + "ksql.error.classifier.regex" : "", + "ksql.suppress.buffer.size.bytes" : "-1" + } +} \ No newline at end of file diff --git a/ksqldb-functional-tests/src/test/resources/historical_plans/initcap_-_do_initcap_-_PROTOBUFs_-_PROTOBUF/7.3.0_1651069881416/spec.json b/ksqldb-functional-tests/src/test/resources/historical_plans/initcap_-_do_initcap_-_PROTOBUFs_-_PROTOBUF/7.3.0_1651069881416/spec.json new file mode 100644 index 000000000000..6931e03eb978 --- /dev/null +++ b/ksqldb-functional-tests/src/test/resources/historical_plans/initcap_-_do_initcap_-_PROTOBUFs_-_PROTOBUF/7.3.0_1651069881416/spec.json @@ -0,0 +1,141 @@ +{ + "version" : "7.3.0", + "timestamp" : 1651069881416, + "path" : "query-validation-tests/initcap.json", + "schemas" : { + "CSAS_OUTPUT_0.KsqlTopic.Source" : { + "schema" : "`K` STRING KEY, `SOURCE` STRING", + "keyFormat" : { + "format" : "KAFKA" + }, + "valueFormat" : { + "format" : "PROTOBUF", + "properties" : { + "unwrapPrimitives" : "true" + } + } + }, + "CSAS_OUTPUT_0.OUTPUT" : { + "schema" : "`K` STRING KEY, `INITCAP` STRING", + "keyFormat" : { + "format" : "KAFKA" + }, + "valueFormat" : { + "format" : "PROTOBUF", + "properties" : { + "unwrapPrimitives" : "true" + } + } + } + }, + "testCase" : { + "name" : "do initcap - PROTOBUFs - PROTOBUF", + "inputs" : [ { + "topic" : "test_topic", + "key" : null, + "value" : { + "source" : "some_string" + } + }, { + "topic" : "test_topic", + "key" : null, + "value" : { + "source" : null + } + }, { + "topic" : "test_topic", + "key" : null, + "value" : { + "source" : "the Quick br0wn fOx" + } + } ], + "outputs" : [ { + "topic" : "OUTPUT", + "key" : null, + "value" : { + "INITCAP" : "Some_string" + } + }, { + "topic" : "OUTPUT", + "key" : null, + "value" : { + "INITCAP" : "" + } + }, { + "topic" : "OUTPUT", + "key" : null, + "value" : { + "INITCAP" : "The Quick Br0wn Fox" + } + } ], + "topics" : [ { + "name" : "OUTPUT", + "replicas" : 1, + "numPartitions" : 4 + }, { + "name" : "test_topic", + "valueSchema" : "syntax = \"proto3\";\n\nmessage ConnectDefault1 {\n string SOURCE = 1;\n}\n", + "valueFormat" : "PROTOBUF", + "replicas" : 1, + "numPartitions" : 4 + } ], + "statements" : [ "CREATE STREAM TEST (K STRING KEY, source VARCHAR) WITH (kafka_topic='test_topic', value_format='PROTOBUF');", "CREATE STREAM OUTPUT AS SELECT K, INITCAP(source) AS INITCAP FROM TEST;" ], + "properties" : { + "ksql.functions.substring.legacy.args" : false + }, + "post" : { + "sources" : [ { + "name" : "OUTPUT", + "type" : "STREAM", + "schema" : "`K` STRING KEY, `INITCAP` STRING", + "keyFormat" : { + "format" : "KAFKA" + }, + "valueFormat" : "PROTOBUF", + "keyFeatures" : [ ], + "valueFeatures" : [ ], + "isSource" : false + }, { + "name" : "TEST", + "type" : "STREAM", + "schema" : "`K` STRING KEY, `SOURCE` STRING", + "keyFormat" : { + "format" : "KAFKA" + }, + "valueFormat" : "PROTOBUF", + "keyFeatures" : [ ], + "valueFeatures" : [ ], + "isSource" : false + } ], + "topics" : { + "topics" : [ { + "name" : "test_topic", + "keyFormat" : { + "format" : "KAFKA" + }, + "valueFormat" : { + "format" : "PROTOBUF", + "properties" : { + "unwrapPrimitives" : "true" + } + }, + "partitions" : 4, + "valueSchema" : "syntax = \"proto3\";\n\nmessage ConnectDefault1 {\n string SOURCE = 1;\n}\n" + }, { + "name" : "OUTPUT", + "keyFormat" : { + "format" : "KAFKA" + }, + "valueFormat" : { + "format" : "PROTOBUF", + "properties" : { + "unwrapPrimitives" : "true" + } + }, + "partitions" : 4, + "valueSchema" : "syntax = \"proto3\";\n\nmessage ConnectDefault1 {\n string INITCAP = 1;\n}\n" + } ] + } + } + } +} \ No newline at end of file diff --git a/ksqldb-functional-tests/src/test/resources/historical_plans/initcap_-_do_initcap_-_PROTOBUFs_-_PROTOBUF/7.3.0_1651069881416/topology b/ksqldb-functional-tests/src/test/resources/historical_plans/initcap_-_do_initcap_-_PROTOBUFs_-_PROTOBUF/7.3.0_1651069881416/topology new file mode 100644 index 000000000000..441a8f282644 --- /dev/null +++ b/ksqldb-functional-tests/src/test/resources/historical_plans/initcap_-_do_initcap_-_PROTOBUFs_-_PROTOBUF/7.3.0_1651069881416/topology @@ -0,0 +1,13 @@ +Topologies: + Sub-topology: 0 + Source: KSTREAM-SOURCE-0000000000 (topics: [test_topic]) + --> KSTREAM-TRANSFORMVALUES-0000000001 + Processor: KSTREAM-TRANSFORMVALUES-0000000001 (stores: []) + --> Project + <-- KSTREAM-SOURCE-0000000000 + Processor: Project (stores: []) + --> KSTREAM-SINK-0000000003 + <-- KSTREAM-TRANSFORMVALUES-0000000001 + Sink: KSTREAM-SINK-0000000003 (topic: OUTPUT) + <-- Project + diff --git a/ksqldb-functional-tests/src/test/resources/historical_plans/initcap_-_do_initcap_-_PROTOBUFs_-_PROTOBUF_NOSR/7.3.0_1651069881439/plan.json b/ksqldb-functional-tests/src/test/resources/historical_plans/initcap_-_do_initcap_-_PROTOBUFs_-_PROTOBUF_NOSR/7.3.0_1651069881439/plan.json new file mode 100644 index 000000000000..a77efe0599bf --- /dev/null +++ b/ksqldb-functional-tests/src/test/resources/historical_plans/initcap_-_do_initcap_-_PROTOBUFs_-_PROTOBUF_NOSR/7.3.0_1651069881439/plan.json @@ -0,0 +1,203 @@ +{ + "plan" : [ { + "@type" : "ksqlPlanV1", + "statementText" : "CREATE STREAM TEST (K STRING KEY, SOURCE STRING) WITH (KAFKA_TOPIC='test_topic', KEY_FORMAT='KAFKA', VALUE_FORMAT='PROTOBUF_NOSR');", + "ddlCommand" : { + "@type" : "createStreamV1", + "sourceName" : "TEST", + "schema" : "`K` STRING KEY, `SOURCE` STRING", + "timestampColumn" : null, + "topicName" : "test_topic", + "formats" : { + "keyFormat" : { + "format" : "KAFKA", + "properties" : { } + }, + "valueFormat" : { + "format" : "PROTOBUF_NOSR", + "properties" : { } + } + }, + "windowInfo" : null, + "orReplace" : false, + "isSource" : false + }, + "queryPlan" : null + }, { + "@type" : "ksqlPlanV1", + "statementText" : "CREATE STREAM OUTPUT AS SELECT\n TEST.K K,\n INITCAP(TEST.SOURCE) INITCAP\nFROM TEST TEST\nEMIT CHANGES", + "ddlCommand" : { + "@type" : "createStreamV1", + "sourceName" : "OUTPUT", + "schema" : "`K` STRING KEY, `INITCAP` STRING", + "timestampColumn" : null, + "topicName" : "OUTPUT", + "formats" : { + "keyFormat" : { + "format" : "KAFKA", + "properties" : { } + }, + "valueFormat" : { + "format" : "PROTOBUF_NOSR", + "properties" : { } + } + }, + "windowInfo" : null, + "orReplace" : false, + "isSource" : false + }, + "queryPlan" : { + "sources" : [ "TEST" ], + "sink" : "OUTPUT", + "physicalPlan" : { + "@type" : "streamSinkV1", + "properties" : { + "queryContext" : "OUTPUT" + }, + "source" : { + "@type" : "streamSelectV1", + "properties" : { + "queryContext" : "Project" + }, + "source" : { + "@type" : "streamSourceV1", + "properties" : { + "queryContext" : "KsqlTopic/Source" + }, + "topicName" : "test_topic", + "formats" : { + "keyFormat" : { + "format" : "KAFKA", + "properties" : { } + }, + "valueFormat" : { + "format" : "PROTOBUF_NOSR", + "properties" : { } + } + }, + "timestampColumn" : null, + "sourceSchema" : "`K` STRING KEY, `SOURCE` STRING", + "pseudoColumnVersion" : 1 + }, + "keyColumnNames" : [ "K" ], + "selectedKeys" : null, + "selectExpressions" : [ "INITCAP(SOURCE) AS INITCAP" ] + }, + "formats" : { + "keyFormat" : { + "format" : "KAFKA", + "properties" : { } + }, + "valueFormat" : { + "format" : "PROTOBUF_NOSR", + "properties" : { } + } + }, + "topicName" : "OUTPUT", + "timestampColumn" : null + }, + "queryId" : "CSAS_OUTPUT_0", + "runtimeId" : null + } + } ], + "configs" : { + "ksql.extension.dir" : "ext", + "ksql.streams.cache.max.bytes.buffering" : "0", + "metric.reporters" : "", + "ksql.query.status.running.threshold.seconds" : "300", + "ksql.connect.basic.auth.credentials.reload" : "false", + "ksql.output.topic.name.prefix" : "", + "ksql.query.pull.stream.enabled" : "true", + "ksql.query.push.v2.interpreter.enabled" : "true", + "ksql.queryanonymizer.logs_enabled" : "true", + "ksql.variable.substitution.enable" : "true", + "ksql.streams.shutdown.timeout.ms" : "300000", + "ksql.query.pull.max.allowed.offset.lag" : "9223372036854775807", + "ksql.query.pull.max.qps" : "2147483647", + "ksql.access.validator.enable" : "auto", + "ksql.streams.bootstrap.servers" : "localhost:0", + "ksql.query.pull.metrics.enabled" : "true", + "ksql.metrics.extension" : null, + "ksql.query.push.v2.alos.enabled" : "true", + "ksql.query.push.v2.max.hourly.bandwidth.megabytes" : "2147483647", + "ksql.query.pull.range.scan.enabled" : "true", + "ksql.transient.query.cleanup.service.initial.delay.seconds" : "600", + "ksql.hidden.topics" : "_confluent.*,__confluent.*,_schemas,__consumer_offsets,__transaction_state,connect-configs,connect-offsets,connect-status,connect-statuses", + "ksql.lambdas.enabled" : "true", + "ksql.source.table.materialization.enabled" : "true", + "ksql.query.pull.max.hourly.bandwidth.megabytes" : "2147483647", + "ksql.sink.window.change.log.additional.retention" : "1000000", + "ksql.query.persistent.active.limit" : "2147483647", + "ksql.persistence.wrap.single.values" : null, + "ksql.query.transient.max.bytes.buffering.total" : "-1", + "ksql.connect.basic.auth.credentials.source" : "NONE", + "ksql.schema.registry.url" : "schema_registry.url:0", + "ksql.properties.overrides.denylist" : "", + "ksql.service.id" : "some.ksql.service.id", + "ksql.query.push.v2.max.catchup.consumers" : "5", + "ksql.query.push.v2.enabled" : "false", + "ksql.transient.query.cleanup.service.enable" : "true", + "ksql.query.push.v2.metrics.enabled" : "true", + "ksql.rowpartition.rowoffset.enabled" : "true", + "ksql.streams.commit.interval.ms" : "2000", + "ksql.query.pull.table.scan.enabled" : "true", + "ksql.streams.auto.commit.interval.ms" : "0", + "ksql.streams.topology.optimization" : "all", + "ksql.endpoint.migrate.query" : "true", + "ksql.query.push.v2.registry.installed" : "false", + "ksql.streams.num.stream.threads" : "4", + "ksql.metrics.tags.custom" : "", + "ksql.query.push.v2.catchup.consumer.msg.window" : "50", + "ksql.runtime.feature.shared.enabled" : "false", + "ksql.udf.collect.metrics" : "false", + "ksql.new.query.planner.enabled" : "false", + "ksql.connect.request.headers.plugin" : null, + "ksql.security.extension.class" : null, + "ksql.transient.prefix" : "transient_", + "ksql.headers.columns.enabled" : "true", + "ksql.streams.default.deserialization.exception.handler" : "io.confluent.ksql.errors.LogMetricAndContinueExceptionHandler", + "ksql.connect.request.timeout.ms" : "5000", + "ksql.query.pull.enable.standby.reads" : "false", + "ksql.persistence.default.format.key" : "KAFKA", + "ksql.query.persistent.max.bytes.buffering.total" : "-1", + "ksql.query.error.max.queue.size" : "10", + "ksql.query.cleanup.shutdown.timeout.ms" : "30000", + "ksql.internal.topic.min.insync.replicas" : "1", + "ksql.internal.topic.replicas" : "1", + "ksql.insert.into.values.enabled" : "true", + "ksql.queryanonymizer.cluster_namespace" : null, + "ksql.create.or.replace.enabled" : "true", + "ksql.shared.runtimes.count" : "2", + "ksql.cast.strings.preserve.nulls" : "true", + "ksql.authorization.cache.max.entries" : "10000", + "ksql.pull.queries.enable" : "true", + "ksql.transient.query.cleanup.service.period.seconds" : "600", + "ksql.suppress.enabled" : "false", + "ksql.readonly.topics" : "_confluent.*,__confluent.*,_schemas,__consumer_offsets,__transaction_state,connect-configs,connect-offsets,connect-status,connect-statuses", + "ksql.connect.basic.auth.credentials.file" : "", + "ksql.authorization.cache.expiry.time.secs" : "30", + "ksql.query.retry.backoff.initial.ms" : "15000", + "ksql.query.pull.max.concurrent.requests" : "2147483647", + "ksql.streams.auto.offset.reset" : "earliest", + "ksql.connect.url" : "http://localhost:8083", + "ksql.query.push.v2.new.latest.delay.ms" : "5000", + "ksql.query.push.v2.latest.reset.age.ms" : "30000", + "ksql.streams.default.production.exception.handler" : "io.confluent.ksql.errors.ProductionExceptionHandlerUtil$LogAndFailProductionExceptionHandler", + "ksql.query.pull.interpreter.enabled" : "true", + "ksql.query.pull.limit.clause.enabled" : "true", + "ksql.query.pull.router.thread.pool.size" : "50", + "ksql.query.push.v2.continuation.tokens.enabled" : "false", + "ksql.query.retry.backoff.max.ms" : "900000", + "ksql.timestamp.throw.on.invalid" : "false", + "ksql.persistence.default.format.value" : null, + "ksql.udfs.enabled" : "true", + "ksql.udf.enable.security.manager" : "true", + "ksql.connect.worker.config" : "", + "ksql.nested.error.set.null" : "true", + "ksql.query.pull.thread.pool.size" : "50", + "ksql.persistent.prefix" : "query_", + "ksql.metastore.backup.location" : "", + "ksql.error.classifier.regex" : "", + "ksql.suppress.buffer.size.bytes" : "-1" + } +} \ No newline at end of file diff --git a/ksqldb-functional-tests/src/test/resources/historical_plans/initcap_-_do_initcap_-_PROTOBUFs_-_PROTOBUF_NOSR/7.3.0_1651069881439/spec.json b/ksqldb-functional-tests/src/test/resources/historical_plans/initcap_-_do_initcap_-_PROTOBUFs_-_PROTOBUF_NOSR/7.3.0_1651069881439/spec.json new file mode 100644 index 000000000000..847499fb4e3e --- /dev/null +++ b/ksqldb-functional-tests/src/test/resources/historical_plans/initcap_-_do_initcap_-_PROTOBUFs_-_PROTOBUF_NOSR/7.3.0_1651069881439/spec.json @@ -0,0 +1,125 @@ +{ + "version" : "7.3.0", + "timestamp" : 1651069881439, + "path" : "query-validation-tests/initcap.json", + "schemas" : { + "CSAS_OUTPUT_0.KsqlTopic.Source" : { + "schema" : "`K` STRING KEY, `SOURCE` STRING", + "keyFormat" : { + "format" : "KAFKA" + }, + "valueFormat" : { + "format" : "PROTOBUF_NOSR" + } + }, + "CSAS_OUTPUT_0.OUTPUT" : { + "schema" : "`K` STRING KEY, `INITCAP` STRING", + "keyFormat" : { + "format" : "KAFKA" + }, + "valueFormat" : { + "format" : "PROTOBUF_NOSR" + } + } + }, + "testCase" : { + "name" : "do initcap - PROTOBUFs - PROTOBUF_NOSR", + "inputs" : [ { + "topic" : "test_topic", + "key" : null, + "value" : { + "source" : "some_string" + } + }, { + "topic" : "test_topic", + "key" : null, + "value" : { + "source" : null + } + }, { + "topic" : "test_topic", + "key" : null, + "value" : { + "source" : "the Quick br0wn fOx" + } + } ], + "outputs" : [ { + "topic" : "OUTPUT", + "key" : null, + "value" : { + "INITCAP" : "Some_string" + } + }, { + "topic" : "OUTPUT", + "key" : null, + "value" : { + "INITCAP" : "" + } + }, { + "topic" : "OUTPUT", + "key" : null, + "value" : { + "INITCAP" : "The Quick Br0wn Fox" + } + } ], + "topics" : [ { + "name" : "OUTPUT", + "replicas" : 1, + "numPartitions" : 4 + }, { + "name" : "test_topic", + "replicas" : 1, + "numPartitions" : 4 + } ], + "statements" : [ "CREATE STREAM TEST (K STRING KEY, source VARCHAR) WITH (kafka_topic='test_topic', value_format='PROTOBUF_NOSR');", "CREATE STREAM OUTPUT AS SELECT K, INITCAP(source) AS INITCAP FROM TEST;" ], + "properties" : { + "ksql.functions.substring.legacy.args" : false + }, + "post" : { + "sources" : [ { + "name" : "OUTPUT", + "type" : "STREAM", + "schema" : "`K` STRING KEY, `INITCAP` STRING", + "keyFormat" : { + "format" : "KAFKA" + }, + "valueFormat" : "PROTOBUF_NOSR", + "keyFeatures" : [ ], + "valueFeatures" : [ ], + "isSource" : false + }, { + "name" : "TEST", + "type" : "STREAM", + "schema" : "`K` STRING KEY, `SOURCE` STRING", + "keyFormat" : { + "format" : "KAFKA" + }, + "valueFormat" : "PROTOBUF_NOSR", + "keyFeatures" : [ ], + "valueFeatures" : [ ], + "isSource" : false + } ], + "topics" : { + "topics" : [ { + "name" : "test_topic", + "keyFormat" : { + "format" : "KAFKA" + }, + "valueFormat" : { + "format" : "PROTOBUF_NOSR" + }, + "partitions" : 4 + }, { + "name" : "OUTPUT", + "keyFormat" : { + "format" : "KAFKA" + }, + "valueFormat" : { + "format" : "PROTOBUF_NOSR" + }, + "partitions" : 4 + } ] + } + } + } +} \ No newline at end of file diff --git a/ksqldb-functional-tests/src/test/resources/historical_plans/initcap_-_do_initcap_-_PROTOBUFs_-_PROTOBUF_NOSR/7.3.0_1651069881439/topology b/ksqldb-functional-tests/src/test/resources/historical_plans/initcap_-_do_initcap_-_PROTOBUFs_-_PROTOBUF_NOSR/7.3.0_1651069881439/topology new file mode 100644 index 000000000000..441a8f282644 --- /dev/null +++ b/ksqldb-functional-tests/src/test/resources/historical_plans/initcap_-_do_initcap_-_PROTOBUFs_-_PROTOBUF_NOSR/7.3.0_1651069881439/topology @@ -0,0 +1,13 @@ +Topologies: + Sub-topology: 0 + Source: KSTREAM-SOURCE-0000000000 (topics: [test_topic]) + --> KSTREAM-TRANSFORMVALUES-0000000001 + Processor: KSTREAM-TRANSFORMVALUES-0000000001 (stores: []) + --> Project + <-- KSTREAM-SOURCE-0000000000 + Processor: Project (stores: []) + --> KSTREAM-SINK-0000000003 + <-- KSTREAM-TRANSFORMVALUES-0000000001 + Sink: KSTREAM-SINK-0000000003 (topic: OUTPUT) + <-- Project + diff --git a/ksqldb-functional-tests/src/test/resources/historical_plans/insert-into_-_INSERT_INTO_stream_with_SCHEMA_ID_and_SCHEMA_FULL_NAME/7.2.0_1650587884607/plan.json b/ksqldb-functional-tests/src/test/resources/historical_plans/insert-into_-_INSERT_INTO_stream_with_SCHEMA_ID_and_SCHEMA_FULL_NAME/7.2.0_1650587884607/plan.json new file mode 100644 index 000000000000..5b1967aaefd4 --- /dev/null +++ b/ksqldb-functional-tests/src/test/resources/historical_plans/insert-into_-_INSERT_INTO_stream_with_SCHEMA_ID_and_SCHEMA_FULL_NAME/7.2.0_1650587884607/plan.json @@ -0,0 +1,226 @@ +{ + "plan" : [ { + "@type" : "ksqlPlanV1", + "statementText" : "CREATE STREAM SOURCE (K STRING KEY, A BIGINT, B STRING) WITH (FORMAT='PROTOBUF', KAFKA_TOPIC='source', KEY_SCHEMA_FULL_NAME='ProtobufKey2', KEY_SCHEMA_ID=1, VALUE_SCHEMA_FULL_NAME='ProtobufValue2', VALUE_SCHEMA_ID=2);", + "ddlCommand" : { + "@type" : "createStreamV1", + "sourceName" : "SOURCE", + "schema" : "`K` STRING KEY, `A` BIGINT, `B` STRING", + "topicName" : "source", + "formats" : { + "keyFormat" : { + "format" : "PROTOBUF", + "properties" : { + "fullSchemaName" : "ProtobufKey2", + "unwrapPrimitives" : "true", + "schemaId" : "1" + } + }, + "valueFormat" : { + "format" : "PROTOBUF", + "properties" : { + "fullSchemaName" : "ProtobufValue2", + "unwrapPrimitives" : "true", + "schemaId" : "2" + } + } + }, + "orReplace" : false, + "isSource" : false + } + }, { + "@type" : "ksqlPlanV1", + "statementText" : "CREATE STREAM SINK (K STRING KEY, A BIGINT, B STRING) WITH (FORMAT='PROTOBUF', KAFKA_TOPIC='sink', KEY_SCHEMA_FULL_NAME='ProtobufKey2', VALUE_SCHEMA_FULL_NAME='ProtobufValue2');", + "ddlCommand" : { + "@type" : "createStreamV1", + "sourceName" : "SINK", + "schema" : "`K` STRING KEY, `A` BIGINT, `B` STRING", + "topicName" : "sink", + "formats" : { + "keyFormat" : { + "format" : "PROTOBUF", + "properties" : { + "fullSchemaName" : "ProtobufKey2", + "unwrapPrimitives" : "true" + } + }, + "valueFormat" : { + "format" : "PROTOBUF", + "properties" : { + "fullSchemaName" : "ProtobufValue2", + "unwrapPrimitives" : "true" + } + } + }, + "orReplace" : false, + "isSource" : false + } + }, { + "@type" : "ksqlPlanV1", + "statementText" : "INSERT INTO SINK SELECT * FROM SOURCE;", + "queryPlan" : { + "sources" : [ "SOURCE" ], + "sink" : "SINK", + "physicalPlan" : { + "@type" : "streamSinkV1", + "properties" : { + "queryContext" : "SINK" + }, + "source" : { + "@type" : "streamSelectV1", + "properties" : { + "queryContext" : "Project" + }, + "source" : { + "@type" : "streamSourceV1", + "properties" : { + "queryContext" : "KsqlTopic/Source" + }, + "topicName" : "source", + "formats" : { + "keyFormat" : { + "format" : "PROTOBUF", + "properties" : { + "fullSchemaName" : "ProtobufKey2", + "unwrapPrimitives" : "true", + "schemaId" : "1" + } + }, + "valueFormat" : { + "format" : "PROTOBUF", + "properties" : { + "fullSchemaName" : "ProtobufValue2", + "unwrapPrimitives" : "true", + "schemaId" : "2" + } + } + }, + "sourceSchema" : "`K` STRING KEY, `A` BIGINT, `B` STRING", + "pseudoColumnVersion" : 1 + }, + "keyColumnNames" : [ "K" ], + "selectExpressions" : [ "A AS A", "B AS B" ] + }, + "formats" : { + "keyFormat" : { + "format" : "PROTOBUF", + "properties" : { + "fullSchemaName" : "ProtobufKey2", + "unwrapPrimitives" : "true" + } + }, + "valueFormat" : { + "format" : "PROTOBUF", + "properties" : { + "fullSchemaName" : "ProtobufValue2", + "unwrapPrimitives" : "true" + } + } + }, + "topicName" : "sink" + }, + "queryId" : "INSERTQUERY_0" + } + } ], + "configs" : { + "ksql.extension.dir" : "ext", + "ksql.streams.cache.max.bytes.buffering" : "0", + "metric.reporters" : "", + "ksql.query.status.running.threshold.seconds" : "300", + "ksql.connect.basic.auth.credentials.reload" : "false", + "ksql.output.topic.name.prefix" : "", + "ksql.query.pull.stream.enabled" : "true", + "ksql.query.push.v2.interpreter.enabled" : "true", + "ksql.queryanonymizer.logs_enabled" : "true", + "ksql.variable.substitution.enable" : "true", + "ksql.streams.shutdown.timeout.ms" : "300000", + "ksql.query.pull.max.allowed.offset.lag" : "9223372036854775807", + "ksql.query.pull.max.qps" : "2147483647", + "ksql.access.validator.enable" : "auto", + "ksql.streams.bootstrap.servers" : "localhost:0", + "ksql.query.pull.metrics.enabled" : "true", + "ksql.metrics.extension" : null, + "ksql.query.push.v2.alos.enabled" : "true", + "ksql.query.push.v2.max.hourly.bandwidth.megabytes" : "2147483647", + "ksql.query.pull.range.scan.enabled" : "true", + "ksql.transient.query.cleanup.service.initial.delay.seconds" : "600", + "ksql.hidden.topics" : "_confluent.*,__confluent.*,_schemas,__consumer_offsets,__transaction_state,connect-configs,connect-offsets,connect-status,connect-statuses", + "ksql.lambdas.enabled" : "true", + "ksql.source.table.materialization.enabled" : "true", + "ksql.query.pull.max.hourly.bandwidth.megabytes" : "2147483647", + "ksql.sink.window.change.log.additional.retention" : "1000000", + "ksql.query.persistent.active.limit" : "2147483647", + "ksql.persistence.wrap.single.values" : null, + "ksql.query.transient.max.bytes.buffering.total" : "-1", + "ksql.connect.basic.auth.credentials.source" : "NONE", + "ksql.schema.registry.url" : "schema_registry.url:0", + "ksql.properties.overrides.denylist" : "", + "ksql.service.id" : "some.ksql.service.id", + "ksql.query.push.v2.max.catchup.consumers" : "5", + "ksql.query.push.v2.enabled" : "false", + "ksql.transient.query.cleanup.service.enable" : "true", + "ksql.query.push.v2.metrics.enabled" : "true", + "ksql.rowpartition.rowoffset.enabled" : "true", + "ksql.streams.commit.interval.ms" : "2000", + "ksql.query.pull.table.scan.enabled" : "true", + "ksql.streams.auto.commit.interval.ms" : "0", + "ksql.streams.topology.optimization" : "all", + "ksql.endpoint.migrate.query" : "true", + "ksql.query.push.v2.registry.installed" : "false", + "ksql.streams.num.stream.threads" : "4", + "ksql.metrics.tags.custom" : "", + "ksql.query.push.v2.catchup.consumer.msg.window" : "50", + "ksql.runtime.feature.shared.enabled" : "false", + "ksql.udf.collect.metrics" : "false", + "ksql.new.query.planner.enabled" : "false", + "ksql.connect.request.headers.plugin" : null, + "ksql.security.extension.class" : null, + "ksql.transient.prefix" : "transient_", + "ksql.headers.columns.enabled" : "true", + "ksql.streams.default.deserialization.exception.handler" : "io.confluent.ksql.errors.LogMetricAndContinueExceptionHandler", + "ksql.connect.request.timeout.ms" : "5000", + "ksql.query.pull.enable.standby.reads" : "false", + "ksql.persistence.default.format.key" : "KAFKA", + "ksql.query.persistent.max.bytes.buffering.total" : "-1", + "ksql.query.error.max.queue.size" : "10", + "ksql.query.cleanup.shutdown.timeout.ms" : "30000", + "ksql.internal.topic.min.insync.replicas" : "1", + "ksql.internal.topic.replicas" : "1", + "ksql.insert.into.values.enabled" : "true", + "ksql.queryanonymizer.cluster_namespace" : null, + "ksql.create.or.replace.enabled" : "true", + "ksql.shared.runtimes.count" : "2", + "ksql.cast.strings.preserve.nulls" : "true", + "ksql.authorization.cache.max.entries" : "10000", + "ksql.pull.queries.enable" : "true", + "ksql.transient.query.cleanup.service.period.seconds" : "600", + "ksql.suppress.enabled" : "false", + "ksql.readonly.topics" : "_confluent.*,__confluent.*,_schemas,__consumer_offsets,__transaction_state,connect-configs,connect-offsets,connect-status,connect-statuses", + "ksql.connect.basic.auth.credentials.file" : "", + "ksql.authorization.cache.expiry.time.secs" : "30", + "ksql.query.retry.backoff.initial.ms" : "15000", + "ksql.query.pull.max.concurrent.requests" : "2147483647", + "ksql.streams.auto.offset.reset" : "earliest", + "ksql.connect.url" : "http://localhost:8083", + "ksql.query.push.v2.new.latest.delay.ms" : "5000", + "ksql.query.push.v2.latest.reset.age.ms" : "30000", + "ksql.streams.default.production.exception.handler" : "io.confluent.ksql.errors.ProductionExceptionHandlerUtil$LogAndFailProductionExceptionHandler", + "ksql.query.pull.interpreter.enabled" : "true", + "ksql.query.pull.limit.clause.enabled" : "true", + "ksql.connect.error.handler" : null, + "ksql.query.pull.router.thread.pool.size" : "50", + "ksql.query.push.v2.continuation.tokens.enabled" : "false", + "ksql.query.retry.backoff.max.ms" : "900000", + "ksql.timestamp.throw.on.invalid" : "false", + "ksql.persistence.default.format.value" : null, + "ksql.udfs.enabled" : "true", + "ksql.udf.enable.security.manager" : "true", + "ksql.connect.worker.config" : "", + "ksql.nested.error.set.null" : "true", + "ksql.query.pull.thread.pool.size" : "50", + "ksql.persistent.prefix" : "query_", + "ksql.metastore.backup.location" : "", + "ksql.error.classifier.regex" : "", + "ksql.suppress.buffer.size.bytes" : "-1" + } +} \ No newline at end of file diff --git a/ksqldb-functional-tests/src/test/resources/historical_plans/insert-into_-_INSERT_INTO_stream_with_SCHEMA_ID_and_SCHEMA_FULL_NAME/7.2.0_1650587884607/spec.json b/ksqldb-functional-tests/src/test/resources/historical_plans/insert-into_-_INSERT_INTO_stream_with_SCHEMA_ID_and_SCHEMA_FULL_NAME/7.2.0_1650587884607/spec.json new file mode 100644 index 000000000000..a0ed74181108 --- /dev/null +++ b/ksqldb-functional-tests/src/test/resources/historical_plans/insert-into_-_INSERT_INTO_stream_with_SCHEMA_ID_and_SCHEMA_FULL_NAME/7.2.0_1650587884607/spec.json @@ -0,0 +1,194 @@ +{ + "version" : "7.2.0", + "timestamp" : 1650587884607, + "path" : "query-validation-tests/insert-into.json", + "schemas" : { + "INSERTQUERY_0.KsqlTopic.Source" : { + "schema" : "`K` STRING KEY, `A` BIGINT, `B` STRING", + "keyFormat" : { + "format" : "PROTOBUF", + "properties" : { + "fullSchemaName" : "ProtobufKey2", + "unwrapPrimitives" : "true", + "schemaId" : "1" + } + }, + "valueFormat" : { + "format" : "PROTOBUF", + "properties" : { + "fullSchemaName" : "ProtobufValue2", + "unwrapPrimitives" : "true", + "schemaId" : "2" + } + } + }, + "INSERTQUERY_0.SINK" : { + "schema" : "`K` STRING KEY, `A` BIGINT, `B` STRING", + "keyFormat" : { + "format" : "PROTOBUF", + "properties" : { + "fullSchemaName" : "ProtobufKey2", + "unwrapPrimitives" : "true" + } + }, + "valueFormat" : { + "format" : "PROTOBUF", + "properties" : { + "fullSchemaName" : "ProtobufValue2", + "unwrapPrimitives" : "true" + } + } + } + }, + "testCase" : { + "name" : "INSERT INTO stream with SCHEMA_ID and SCHEMA_FULL_NAME", + "inputs" : [ { + "topic" : "source", + "key" : { + "K" : "0" + }, + "value" : { + "A" : 123, + "B" : "falcon" + }, + "timestamp" : 0 + }, { + "topic" : "source", + "key" : { + "K" : "0" + }, + "value" : { + "A" : 456, + "B" : "giraffe" + }, + "timestamp" : 0 + }, { + "topic" : "source", + "key" : { + "K" : "0" + }, + "value" : { + "A" : 789, + "B" : "turtle" + }, + "timestamp" : 0 + } ], + "outputs" : [ { + "topic" : "sink", + "key" : { + "K" : "0" + }, + "value" : { + "A" : 123, + "B" : "falcon" + }, + "timestamp" : 0 + }, { + "topic" : "sink", + "key" : { + "K" : "0" + }, + "value" : { + "A" : 456, + "B" : "giraffe" + }, + "timestamp" : 0 + }, { + "topic" : "sink", + "key" : { + "K" : "0" + }, + "value" : { + "A" : 789, + "B" : "turtle" + }, + "timestamp" : 0 + } ], + "topics" : [ { + "name" : "sink", + "keySchema" : "syntax = \"proto3\";\n\nmessage ProtobufKey1 {\n uint32 k1 = 1;\n}\nmessage ProtobufKey2 {\n string K = 1;\n}\n", + "valueSchema" : "syntax = \"proto3\";\n\nmessage ProtobufValue1 {\n float c1 = 1;\n uint32 c2 = 2;\n}\nmessage ProtobufValue2 {\n uint64 A = 1;\n string B = 2;\n}\n", + "keyFormat" : "PROTOBUF", + "valueFormat" : "PROTOBUF", + "replicas" : 1, + "numPartitions" : 1 + }, { + "name" : "source", + "keySchema" : "syntax = \"proto3\";\n\nmessage ProtobufKey1 {\n uint32 k1 = 1;\n}\nmessage ProtobufKey2 {\n string K = 1;\n}\n", + "valueSchema" : "syntax = \"proto3\";\n\nmessage ProtobufValue1 {\n float c1 = 1;\n uint32 c2 = 2;\n}\nmessage ProtobufValue2 {\n uint64 A = 1;\n string B = 2;\n}\n", + "keyFormat" : "PROTOBUF", + "valueFormat" : "PROTOBUF", + "replicas" : 1, + "numPartitions" : 1 + } ], + "statements" : [ "CREATE STREAM SOURCE WITH (kafka_topic='source', format='PROTOBUF', KEY_SCHEMA_ID=1, KEY_SCHEMA_FULL_NAME='ProtobufKey2', VALUE_SCHEMA_ID=2, VALUE_SCHEMA_FULL_NAME='ProtobufValue2');", "CREATE STREAM SINK WITH (kafka_topic='sink', format='PROTOBUF', KEY_SCHEMA_FULL_NAME='ProtobufKey2', VALUE_SCHEMA_FULL_NAME='ProtobufValue2');", "INSERT INTO SINK SELECT * FROM SOURCE;" ], + "post" : { + "sources" : [ { + "name" : "SINK", + "type" : "STREAM", + "schema" : "`K` STRING KEY, `A` BIGINT, `B` STRING", + "keyFormat" : { + "format" : "PROTOBUF" + }, + "valueFormat" : "PROTOBUF", + "keyFeatures" : [ ], + "valueFeatures" : [ ], + "isSource" : false + }, { + "name" : "SOURCE", + "type" : "STREAM", + "schema" : "`K` STRING KEY, `A` BIGINT, `B` STRING", + "keyFormat" : { + "format" : "PROTOBUF" + }, + "valueFormat" : "PROTOBUF", + "keyFeatures" : [ ], + "valueFeatures" : [ ], + "isSource" : false + } ], + "topics" : { + "topics" : [ { + "name" : "source", + "keyFormat" : { + "format" : "PROTOBUF", + "properties" : { + "fullSchemaName" : "ProtobufKey2", + "unwrapPrimitives" : "true", + "schemaId" : "1" + } + }, + "valueFormat" : { + "format" : "PROTOBUF", + "properties" : { + "fullSchemaName" : "ProtobufValue2", + "unwrapPrimitives" : "true", + "schemaId" : "2" + } + }, + "partitions" : 1, + "keySchema" : "syntax = \"proto3\";\n\nmessage ProtobufKey2 {\n string K = 1;\n}\n", + "valueSchema" : "syntax = \"proto3\";\n\nmessage ProtobufValue2 {\n uint64 A = 1;\n string B = 2;\n}\n" + }, { + "name" : "sink", + "keyFormat" : { + "format" : "PROTOBUF", + "properties" : { + "fullSchemaName" : "ProtobufKey2", + "unwrapPrimitives" : "true" + } + }, + "valueFormat" : { + "format" : "PROTOBUF", + "properties" : { + "fullSchemaName" : "ProtobufValue2", + "unwrapPrimitives" : "true" + } + }, + "partitions" : 4, + "keySchema" : "syntax = \"proto3\";\n\nmessage ProtobufKey2 {\n string K = 1;\n}\n", + "valueSchema" : "syntax = \"proto3\";\n\nmessage ProtobufValue2 {\n int64 A = 1;\n string B = 2;\n}\n" + } ] + } + } + } +} \ No newline at end of file diff --git a/ksqldb-functional-tests/src/test/resources/historical_plans/insert-into_-_INSERT_INTO_stream_with_SCHEMA_ID_and_SCHEMA_FULL_NAME/7.2.0_1650587884607/topology b/ksqldb-functional-tests/src/test/resources/historical_plans/insert-into_-_INSERT_INTO_stream_with_SCHEMA_ID_and_SCHEMA_FULL_NAME/7.2.0_1650587884607/topology new file mode 100644 index 000000000000..f30a30881525 --- /dev/null +++ b/ksqldb-functional-tests/src/test/resources/historical_plans/insert-into_-_INSERT_INTO_stream_with_SCHEMA_ID_and_SCHEMA_FULL_NAME/7.2.0_1650587884607/topology @@ -0,0 +1,13 @@ +Topologies: + Sub-topology: 0 + Source: KSTREAM-SOURCE-0000000000 (topics: [source]) + --> KSTREAM-TRANSFORMVALUES-0000000001 + Processor: KSTREAM-TRANSFORMVALUES-0000000001 (stores: []) + --> Project + <-- KSTREAM-SOURCE-0000000000 + Processor: Project (stores: []) + --> KSTREAM-SINK-0000000003 + <-- KSTREAM-TRANSFORMVALUES-0000000001 + Sink: KSTREAM-SINK-0000000003 (topic: sink) + <-- Project + diff --git a/ksqldb-functional-tests/src/test/resources/historical_plans/insert-into_-_INSERT_INTO_stream_with_SCHEMA_ID_and_SCHEMA_FULL_NAME/7.3.0_1650558675555/plan.json b/ksqldb-functional-tests/src/test/resources/historical_plans/insert-into_-_INSERT_INTO_stream_with_SCHEMA_ID_and_SCHEMA_FULL_NAME/7.3.0_1650558675555/plan.json new file mode 100644 index 000000000000..9c7a3c31776d --- /dev/null +++ b/ksqldb-functional-tests/src/test/resources/historical_plans/insert-into_-_INSERT_INTO_stream_with_SCHEMA_ID_and_SCHEMA_FULL_NAME/7.3.0_1650558675555/plan.json @@ -0,0 +1,236 @@ +{ + "plan" : [ { + "@type" : "ksqlPlanV1", + "statementText" : "CREATE STREAM SOURCE (K STRING KEY, A BIGINT, B STRING) WITH (FORMAT='PROTOBUF', KAFKA_TOPIC='source', KEY_SCHEMA_FULL_NAME='ProtobufKey2', KEY_SCHEMA_ID=1, VALUE_SCHEMA_FULL_NAME='ProtobufValue2', VALUE_SCHEMA_ID=2);", + "ddlCommand" : { + "@type" : "createStreamV1", + "sourceName" : "SOURCE", + "schema" : "`K` STRING KEY, `A` BIGINT, `B` STRING", + "timestampColumn" : null, + "topicName" : "source", + "formats" : { + "keyFormat" : { + "format" : "PROTOBUF", + "properties" : { + "fullSchemaName" : "ProtobufKey2", + "unwrapPrimitives" : "true", + "schemaId" : "1" + } + }, + "valueFormat" : { + "format" : "PROTOBUF", + "properties" : { + "fullSchemaName" : "ProtobufValue2", + "unwrapPrimitives" : "true", + "schemaId" : "2" + } + } + }, + "windowInfo" : null, + "orReplace" : false, + "isSource" : false + }, + "queryPlan" : null + }, { + "@type" : "ksqlPlanV1", + "statementText" : "CREATE STREAM SINK (K STRING KEY, A BIGINT, B STRING) WITH (FORMAT='PROTOBUF', KAFKA_TOPIC='sink', KEY_SCHEMA_FULL_NAME='ProtobufKey2', VALUE_SCHEMA_FULL_NAME='ProtobufValue2');", + "ddlCommand" : { + "@type" : "createStreamV1", + "sourceName" : "SINK", + "schema" : "`K` STRING KEY, `A` BIGINT, `B` STRING", + "timestampColumn" : null, + "topicName" : "sink", + "formats" : { + "keyFormat" : { + "format" : "PROTOBUF", + "properties" : { + "fullSchemaName" : "ProtobufKey2", + "unwrapPrimitives" : "true" + } + }, + "valueFormat" : { + "format" : "PROTOBUF", + "properties" : { + "fullSchemaName" : "ProtobufValue2", + "unwrapPrimitives" : "true" + } + } + }, + "windowInfo" : null, + "orReplace" : false, + "isSource" : false + }, + "queryPlan" : null + }, { + "@type" : "ksqlPlanV1", + "statementText" : "INSERT INTO SINK SELECT * FROM SOURCE;", + "ddlCommand" : null, + "queryPlan" : { + "sources" : [ "SOURCE" ], + "sink" : "SINK", + "physicalPlan" : { + "@type" : "streamSinkV1", + "properties" : { + "queryContext" : "SINK" + }, + "source" : { + "@type" : "streamSelectV1", + "properties" : { + "queryContext" : "Project" + }, + "source" : { + "@type" : "streamSourceV1", + "properties" : { + "queryContext" : "KsqlTopic/Source" + }, + "topicName" : "source", + "formats" : { + "keyFormat" : { + "format" : "PROTOBUF", + "properties" : { + "fullSchemaName" : "ProtobufKey2", + "unwrapPrimitives" : "true", + "schemaId" : "1" + } + }, + "valueFormat" : { + "format" : "PROTOBUF", + "properties" : { + "fullSchemaName" : "ProtobufValue2", + "unwrapPrimitives" : "true", + "schemaId" : "2" + } + } + }, + "timestampColumn" : null, + "sourceSchema" : "`K` STRING KEY, `A` BIGINT, `B` STRING", + "pseudoColumnVersion" : 1 + }, + "keyColumnNames" : [ "K" ], + "selectedKeys" : null, + "selectExpressions" : [ "A AS A", "B AS B" ] + }, + "formats" : { + "keyFormat" : { + "format" : "PROTOBUF", + "properties" : { + "fullSchemaName" : "ProtobufKey2", + "unwrapPrimitives" : "true" + } + }, + "valueFormat" : { + "format" : "PROTOBUF", + "properties" : { + "fullSchemaName" : "ProtobufValue2", + "unwrapPrimitives" : "true" + } + } + }, + "topicName" : "sink", + "timestampColumn" : null + }, + "queryId" : "INSERTQUERY_0", + "runtimeId" : null + } + } ], + "configs" : { + "ksql.extension.dir" : "ext", + "ksql.streams.cache.max.bytes.buffering" : "0", + "metric.reporters" : "", + "ksql.query.status.running.threshold.seconds" : "300", + "ksql.connect.basic.auth.credentials.reload" : "false", + "ksql.output.topic.name.prefix" : "", + "ksql.query.pull.stream.enabled" : "true", + "ksql.query.push.v2.interpreter.enabled" : "true", + "ksql.queryanonymizer.logs_enabled" : "true", + "ksql.variable.substitution.enable" : "true", + "ksql.streams.shutdown.timeout.ms" : "300000", + "ksql.query.pull.max.allowed.offset.lag" : "9223372036854775807", + "ksql.query.pull.max.qps" : "2147483647", + "ksql.access.validator.enable" : "auto", + "ksql.streams.bootstrap.servers" : "localhost:0", + "ksql.query.pull.metrics.enabled" : "true", + "ksql.metrics.extension" : null, + "ksql.query.push.v2.alos.enabled" : "true", + "ksql.query.push.v2.max.hourly.bandwidth.megabytes" : "2147483647", + "ksql.query.pull.range.scan.enabled" : "true", + "ksql.transient.query.cleanup.service.initial.delay.seconds" : "600", + "ksql.hidden.topics" : "_confluent.*,__confluent.*,_schemas,__consumer_offsets,__transaction_state,connect-configs,connect-offsets,connect-status,connect-statuses", + "ksql.lambdas.enabled" : "true", + "ksql.source.table.materialization.enabled" : "true", + "ksql.query.pull.max.hourly.bandwidth.megabytes" : "2147483647", + "ksql.sink.window.change.log.additional.retention" : "1000000", + "ksql.query.persistent.active.limit" : "2147483647", + "ksql.persistence.wrap.single.values" : null, + "ksql.query.transient.max.bytes.buffering.total" : "-1", + "ksql.connect.basic.auth.credentials.source" : "NONE", + "ksql.schema.registry.url" : "schema_registry.url:0", + "ksql.properties.overrides.denylist" : "", + "ksql.service.id" : "some.ksql.service.id", + "ksql.query.push.v2.max.catchup.consumers" : "5", + "ksql.query.push.v2.enabled" : "false", + "ksql.transient.query.cleanup.service.enable" : "true", + "ksql.query.push.v2.metrics.enabled" : "true", + "ksql.rowpartition.rowoffset.enabled" : "true", + "ksql.streams.commit.interval.ms" : "2000", + "ksql.query.pull.table.scan.enabled" : "true", + "ksql.streams.auto.commit.interval.ms" : "0", + "ksql.streams.topology.optimization" : "all", + "ksql.endpoint.migrate.query" : "true", + "ksql.query.push.v2.registry.installed" : "false", + "ksql.streams.num.stream.threads" : "4", + "ksql.metrics.tags.custom" : "", + "ksql.query.push.v2.catchup.consumer.msg.window" : "50", + "ksql.runtime.feature.shared.enabled" : "false", + "ksql.udf.collect.metrics" : "false", + "ksql.new.query.planner.enabled" : "false", + "ksql.connect.request.headers.plugin" : null, + "ksql.security.extension.class" : null, + "ksql.transient.prefix" : "transient_", + "ksql.headers.columns.enabled" : "true", + "ksql.streams.default.deserialization.exception.handler" : "io.confluent.ksql.errors.LogMetricAndContinueExceptionHandler", + "ksql.connect.request.timeout.ms" : "5000", + "ksql.query.pull.enable.standby.reads" : "false", + "ksql.persistence.default.format.key" : "KAFKA", + "ksql.query.persistent.max.bytes.buffering.total" : "-1", + "ksql.query.error.max.queue.size" : "10", + "ksql.query.cleanup.shutdown.timeout.ms" : "30000", + "ksql.internal.topic.min.insync.replicas" : "1", + "ksql.internal.topic.replicas" : "1", + "ksql.insert.into.values.enabled" : "true", + "ksql.queryanonymizer.cluster_namespace" : null, + "ksql.create.or.replace.enabled" : "true", + "ksql.shared.runtimes.count" : "2", + "ksql.cast.strings.preserve.nulls" : "true", + "ksql.authorization.cache.max.entries" : "10000", + "ksql.pull.queries.enable" : "true", + "ksql.transient.query.cleanup.service.period.seconds" : "600", + "ksql.suppress.enabled" : "false", + "ksql.readonly.topics" : "_confluent.*,__confluent.*,_schemas,__consumer_offsets,__transaction_state,connect-configs,connect-offsets,connect-status,connect-statuses", + "ksql.connect.basic.auth.credentials.file" : "", + "ksql.authorization.cache.expiry.time.secs" : "30", + "ksql.query.retry.backoff.initial.ms" : "15000", + "ksql.query.pull.max.concurrent.requests" : "2147483647", + "ksql.streams.auto.offset.reset" : "earliest", + "ksql.connect.url" : "http://localhost:8083", + "ksql.query.push.v2.new.latest.delay.ms" : "5000", + "ksql.query.push.v2.latest.reset.age.ms" : "30000", + "ksql.streams.default.production.exception.handler" : "io.confluent.ksql.errors.ProductionExceptionHandlerUtil$LogAndFailProductionExceptionHandler", + "ksql.query.pull.interpreter.enabled" : "true", + "ksql.query.pull.limit.clause.enabled" : "true", + "ksql.query.pull.router.thread.pool.size" : "50", + "ksql.query.push.v2.continuation.tokens.enabled" : "false", + "ksql.query.retry.backoff.max.ms" : "900000", + "ksql.timestamp.throw.on.invalid" : "false", + "ksql.persistence.default.format.value" : null, + "ksql.udfs.enabled" : "true", + "ksql.udf.enable.security.manager" : "true", + "ksql.connect.worker.config" : "", + "ksql.nested.error.set.null" : "true", + "ksql.query.pull.thread.pool.size" : "50", + "ksql.persistent.prefix" : "query_", + "ksql.metastore.backup.location" : "", + "ksql.error.classifier.regex" : "", + "ksql.suppress.buffer.size.bytes" : "-1" + } +} \ No newline at end of file diff --git a/ksqldb-functional-tests/src/test/resources/historical_plans/insert-into_-_INSERT_INTO_stream_with_SCHEMA_ID_and_SCHEMA_FULL_NAME/7.3.0_1650558675555/spec.json b/ksqldb-functional-tests/src/test/resources/historical_plans/insert-into_-_INSERT_INTO_stream_with_SCHEMA_ID_and_SCHEMA_FULL_NAME/7.3.0_1650558675555/spec.json new file mode 100644 index 000000000000..2d9b18bd1791 --- /dev/null +++ b/ksqldb-functional-tests/src/test/resources/historical_plans/insert-into_-_INSERT_INTO_stream_with_SCHEMA_ID_and_SCHEMA_FULL_NAME/7.3.0_1650558675555/spec.json @@ -0,0 +1,194 @@ +{ + "version" : "7.3.0", + "timestamp" : 1650558675555, + "path" : "query-validation-tests/insert-into.json", + "schemas" : { + "INSERTQUERY_0.KsqlTopic.Source" : { + "schema" : "`K` STRING KEY, `A` BIGINT, `B` STRING", + "keyFormat" : { + "format" : "PROTOBUF", + "properties" : { + "fullSchemaName" : "ProtobufKey2", + "unwrapPrimitives" : "true", + "schemaId" : "1" + } + }, + "valueFormat" : { + "format" : "PROTOBUF", + "properties" : { + "fullSchemaName" : "ProtobufValue2", + "unwrapPrimitives" : "true", + "schemaId" : "2" + } + } + }, + "INSERTQUERY_0.SINK" : { + "schema" : "`K` STRING KEY, `A` BIGINT, `B` STRING", + "keyFormat" : { + "format" : "PROTOBUF", + "properties" : { + "fullSchemaName" : "ProtobufKey2", + "unwrapPrimitives" : "true" + } + }, + "valueFormat" : { + "format" : "PROTOBUF", + "properties" : { + "fullSchemaName" : "ProtobufValue2", + "unwrapPrimitives" : "true" + } + } + } + }, + "testCase" : { + "name" : "INSERT INTO stream with SCHEMA_ID and SCHEMA_FULL_NAME", + "inputs" : [ { + "topic" : "source", + "key" : { + "K" : "0" + }, + "value" : { + "A" : 123, + "B" : "falcon" + }, + "timestamp" : 0 + }, { + "topic" : "source", + "key" : { + "K" : "0" + }, + "value" : { + "A" : 456, + "B" : "giraffe" + }, + "timestamp" : 0 + }, { + "topic" : "source", + "key" : { + "K" : "0" + }, + "value" : { + "A" : 789, + "B" : "turtle" + }, + "timestamp" : 0 + } ], + "outputs" : [ { + "topic" : "sink", + "key" : { + "K" : "0" + }, + "value" : { + "A" : 123, + "B" : "falcon" + }, + "timestamp" : 0 + }, { + "topic" : "sink", + "key" : { + "K" : "0" + }, + "value" : { + "A" : 456, + "B" : "giraffe" + }, + "timestamp" : 0 + }, { + "topic" : "sink", + "key" : { + "K" : "0" + }, + "value" : { + "A" : 789, + "B" : "turtle" + }, + "timestamp" : 0 + } ], + "topics" : [ { + "name" : "sink", + "keySchema" : "syntax = \"proto3\";\n\nmessage ProtobufKey1 {\n uint32 k1 = 1;\n}\nmessage ProtobufKey2 {\n string K = 1;\n}\n", + "valueSchema" : "syntax = \"proto3\";\n\nmessage ProtobufValue1 {\n float c1 = 1;\n uint32 c2 = 2;\n}\nmessage ProtobufValue2 {\n uint64 A = 1;\n string B = 2;\n}\n", + "keyFormat" : "PROTOBUF", + "valueFormat" : "PROTOBUF", + "replicas" : 1, + "numPartitions" : 1 + }, { + "name" : "source", + "keySchema" : "syntax = \"proto3\";\n\nmessage ProtobufKey1 {\n uint32 k1 = 1;\n}\nmessage ProtobufKey2 {\n string K = 1;\n}\n", + "valueSchema" : "syntax = \"proto3\";\n\nmessage ProtobufValue1 {\n float c1 = 1;\n uint32 c2 = 2;\n}\nmessage ProtobufValue2 {\n uint64 A = 1;\n string B = 2;\n}\n", + "keyFormat" : "PROTOBUF", + "valueFormat" : "PROTOBUF", + "replicas" : 1, + "numPartitions" : 1 + } ], + "statements" : [ "CREATE STREAM SOURCE WITH (kafka_topic='source', format='PROTOBUF', KEY_SCHEMA_ID=1, KEY_SCHEMA_FULL_NAME='ProtobufKey2', VALUE_SCHEMA_ID=2, VALUE_SCHEMA_FULL_NAME='ProtobufValue2');", "CREATE STREAM SINK WITH (kafka_topic='sink', format='PROTOBUF', KEY_SCHEMA_FULL_NAME='ProtobufKey2', VALUE_SCHEMA_FULL_NAME='ProtobufValue2');", "INSERT INTO SINK SELECT * FROM SOURCE;" ], + "post" : { + "sources" : [ { + "name" : "SINK", + "type" : "STREAM", + "schema" : "`K` STRING KEY, `A` BIGINT, `B` STRING", + "keyFormat" : { + "format" : "PROTOBUF" + }, + "valueFormat" : "PROTOBUF", + "keyFeatures" : [ ], + "valueFeatures" : [ ], + "isSource" : false + }, { + "name" : "SOURCE", + "type" : "STREAM", + "schema" : "`K` STRING KEY, `A` BIGINT, `B` STRING", + "keyFormat" : { + "format" : "PROTOBUF" + }, + "valueFormat" : "PROTOBUF", + "keyFeatures" : [ ], + "valueFeatures" : [ ], + "isSource" : false + } ], + "topics" : { + "topics" : [ { + "name" : "source", + "keyFormat" : { + "format" : "PROTOBUF", + "properties" : { + "fullSchemaName" : "ProtobufKey2", + "unwrapPrimitives" : "true", + "schemaId" : "1" + } + }, + "valueFormat" : { + "format" : "PROTOBUF", + "properties" : { + "fullSchemaName" : "ProtobufValue2", + "unwrapPrimitives" : "true", + "schemaId" : "2" + } + }, + "partitions" : 1, + "keySchema" : "syntax = \"proto3\";\n\nmessage ProtobufKey2 {\n string K = 1;\n}\n", + "valueSchema" : "syntax = \"proto3\";\n\nmessage ProtobufValue2 {\n uint64 A = 1;\n string B = 2;\n}\n" + }, { + "name" : "sink", + "keyFormat" : { + "format" : "PROTOBUF", + "properties" : { + "fullSchemaName" : "ProtobufKey2", + "unwrapPrimitives" : "true" + } + }, + "valueFormat" : { + "format" : "PROTOBUF", + "properties" : { + "fullSchemaName" : "ProtobufValue2", + "unwrapPrimitives" : "true" + } + }, + "partitions" : 4, + "keySchema" : "syntax = \"proto3\";\n\nmessage ProtobufKey2 {\n string K = 1;\n}\n", + "valueSchema" : "syntax = \"proto3\";\n\nmessage ProtobufValue2 {\n int64 A = 1;\n string B = 2;\n}\n" + } ] + } + } + } +} \ No newline at end of file diff --git a/ksqldb-functional-tests/src/test/resources/historical_plans/insert-into_-_INSERT_INTO_stream_with_SCHEMA_ID_and_SCHEMA_FULL_NAME/7.3.0_1650558675555/topology b/ksqldb-functional-tests/src/test/resources/historical_plans/insert-into_-_INSERT_INTO_stream_with_SCHEMA_ID_and_SCHEMA_FULL_NAME/7.3.0_1650558675555/topology new file mode 100644 index 000000000000..f30a30881525 --- /dev/null +++ b/ksqldb-functional-tests/src/test/resources/historical_plans/insert-into_-_INSERT_INTO_stream_with_SCHEMA_ID_and_SCHEMA_FULL_NAME/7.3.0_1650558675555/topology @@ -0,0 +1,13 @@ +Topologies: + Sub-topology: 0 + Source: KSTREAM-SOURCE-0000000000 (topics: [source]) + --> KSTREAM-TRANSFORMVALUES-0000000001 + Processor: KSTREAM-TRANSFORMVALUES-0000000001 (stores: []) + --> Project + <-- KSTREAM-SOURCE-0000000000 + Processor: Project (stores: []) + --> KSTREAM-SINK-0000000003 + <-- KSTREAM-TRANSFORMVALUES-0000000001 + Sink: KSTREAM-SINK-0000000003 (topic: sink) + <-- Project + diff --git a/ksqldb-functional-tests/src/test/resources/historical_plans/insert-into_-_convert_formats__JSON_to_PROTOBUF_NOSR/7.3.0_1651069881670/plan.json b/ksqldb-functional-tests/src/test/resources/historical_plans/insert-into_-_convert_formats__JSON_to_PROTOBUF_NOSR/7.3.0_1651069881670/plan.json new file mode 100644 index 000000000000..d6fa95f87a88 --- /dev/null +++ b/ksqldb-functional-tests/src/test/resources/historical_plans/insert-into_-_convert_formats__JSON_to_PROTOBUF_NOSR/7.3.0_1651069881670/plan.json @@ -0,0 +1,208 @@ +{ + "plan" : [ { + "@type" : "ksqlPlanV1", + "statementText" : "CREATE STREAM SOURCE (K STRING KEY, A BIGINT, B STRING) WITH (KAFKA_TOPIC='source', KEY_FORMAT='KAFKA', VALUE_FORMAT='JSON');", + "ddlCommand" : { + "@type" : "createStreamV1", + "sourceName" : "SOURCE", + "schema" : "`K` STRING KEY, `A` BIGINT, `B` STRING", + "timestampColumn" : null, + "topicName" : "source", + "formats" : { + "keyFormat" : { + "format" : "KAFKA", + "properties" : { } + }, + "valueFormat" : { + "format" : "JSON", + "properties" : { } + } + }, + "windowInfo" : null, + "orReplace" : false, + "isSource" : false + }, + "queryPlan" : null + }, { + "@type" : "ksqlPlanV1", + "statementText" : "CREATE STREAM SINK (K STRING KEY, A BIGINT, B STRING) WITH (KAFKA_TOPIC='sink', KEY_FORMAT='KAFKA', VALUE_FORMAT='PROTOBUF_NOSR');", + "ddlCommand" : { + "@type" : "createStreamV1", + "sourceName" : "SINK", + "schema" : "`K` STRING KEY, `A` BIGINT, `B` STRING", + "timestampColumn" : null, + "topicName" : "sink", + "formats" : { + "keyFormat" : { + "format" : "KAFKA", + "properties" : { } + }, + "valueFormat" : { + "format" : "PROTOBUF_NOSR", + "properties" : { } + } + }, + "windowInfo" : null, + "orReplace" : false, + "isSource" : false + }, + "queryPlan" : null + }, { + "@type" : "ksqlPlanV1", + "statementText" : "INSERT INTO SINK SELECT * FROM SOURCE;", + "ddlCommand" : null, + "queryPlan" : { + "sources" : [ "SOURCE" ], + "sink" : "SINK", + "physicalPlan" : { + "@type" : "streamSinkV1", + "properties" : { + "queryContext" : "SINK" + }, + "source" : { + "@type" : "streamSelectV1", + "properties" : { + "queryContext" : "Project" + }, + "source" : { + "@type" : "streamSourceV1", + "properties" : { + "queryContext" : "KsqlTopic/Source" + }, + "topicName" : "source", + "formats" : { + "keyFormat" : { + "format" : "KAFKA", + "properties" : { } + }, + "valueFormat" : { + "format" : "JSON", + "properties" : { } + } + }, + "timestampColumn" : null, + "sourceSchema" : "`K` STRING KEY, `A` BIGINT, `B` STRING", + "pseudoColumnVersion" : 1 + }, + "keyColumnNames" : [ "K" ], + "selectedKeys" : null, + "selectExpressions" : [ "A AS A", "B AS B" ] + }, + "formats" : { + "keyFormat" : { + "format" : "KAFKA", + "properties" : { } + }, + "valueFormat" : { + "format" : "PROTOBUF_NOSR", + "properties" : { } + } + }, + "topicName" : "sink", + "timestampColumn" : null + }, + "queryId" : "INSERTQUERY_0", + "runtimeId" : null + } + } ], + "configs" : { + "ksql.extension.dir" : "ext", + "ksql.streams.cache.max.bytes.buffering" : "0", + "metric.reporters" : "", + "ksql.query.status.running.threshold.seconds" : "300", + "ksql.connect.basic.auth.credentials.reload" : "false", + "ksql.output.topic.name.prefix" : "", + "ksql.query.pull.stream.enabled" : "true", + "ksql.query.push.v2.interpreter.enabled" : "true", + "ksql.queryanonymizer.logs_enabled" : "true", + "ksql.variable.substitution.enable" : "true", + "ksql.streams.shutdown.timeout.ms" : "300000", + "ksql.query.pull.max.allowed.offset.lag" : "9223372036854775807", + "ksql.query.pull.max.qps" : "2147483647", + "ksql.access.validator.enable" : "auto", + "ksql.streams.bootstrap.servers" : "localhost:0", + "ksql.query.pull.metrics.enabled" : "true", + "ksql.metrics.extension" : null, + "ksql.query.push.v2.alos.enabled" : "true", + "ksql.query.push.v2.max.hourly.bandwidth.megabytes" : "2147483647", + "ksql.query.pull.range.scan.enabled" : "true", + "ksql.transient.query.cleanup.service.initial.delay.seconds" : "600", + "ksql.hidden.topics" : "_confluent.*,__confluent.*,_schemas,__consumer_offsets,__transaction_state,connect-configs,connect-offsets,connect-status,connect-statuses", + "ksql.lambdas.enabled" : "true", + "ksql.source.table.materialization.enabled" : "true", + "ksql.query.pull.max.hourly.bandwidth.megabytes" : "2147483647", + "ksql.sink.window.change.log.additional.retention" : "1000000", + "ksql.query.persistent.active.limit" : "2147483647", + "ksql.persistence.wrap.single.values" : null, + "ksql.query.transient.max.bytes.buffering.total" : "-1", + "ksql.connect.basic.auth.credentials.source" : "NONE", + "ksql.schema.registry.url" : "schema_registry.url:0", + "ksql.properties.overrides.denylist" : "", + "ksql.service.id" : "some.ksql.service.id", + "ksql.query.push.v2.max.catchup.consumers" : "5", + "ksql.query.push.v2.enabled" : "false", + "ksql.transient.query.cleanup.service.enable" : "true", + "ksql.query.push.v2.metrics.enabled" : "true", + "ksql.rowpartition.rowoffset.enabled" : "true", + "ksql.streams.commit.interval.ms" : "2000", + "ksql.query.pull.table.scan.enabled" : "true", + "ksql.streams.auto.commit.interval.ms" : "0", + "ksql.streams.topology.optimization" : "all", + "ksql.endpoint.migrate.query" : "true", + "ksql.query.push.v2.registry.installed" : "false", + "ksql.streams.num.stream.threads" : "4", + "ksql.metrics.tags.custom" : "", + "ksql.query.push.v2.catchup.consumer.msg.window" : "50", + "ksql.runtime.feature.shared.enabled" : "false", + "ksql.udf.collect.metrics" : "false", + "ksql.new.query.planner.enabled" : "false", + "ksql.connect.request.headers.plugin" : null, + "ksql.security.extension.class" : null, + "ksql.transient.prefix" : "transient_", + "ksql.headers.columns.enabled" : "true", + "ksql.streams.default.deserialization.exception.handler" : "io.confluent.ksql.errors.LogMetricAndContinueExceptionHandler", + "ksql.connect.request.timeout.ms" : "5000", + "ksql.query.pull.enable.standby.reads" : "false", + "ksql.persistence.default.format.key" : "KAFKA", + "ksql.query.persistent.max.bytes.buffering.total" : "-1", + "ksql.query.error.max.queue.size" : "10", + "ksql.query.cleanup.shutdown.timeout.ms" : "30000", + "ksql.internal.topic.min.insync.replicas" : "1", + "ksql.internal.topic.replicas" : "1", + "ksql.insert.into.values.enabled" : "true", + "ksql.queryanonymizer.cluster_namespace" : null, + "ksql.create.or.replace.enabled" : "true", + "ksql.shared.runtimes.count" : "2", + "ksql.cast.strings.preserve.nulls" : "true", + "ksql.authorization.cache.max.entries" : "10000", + "ksql.pull.queries.enable" : "true", + "ksql.transient.query.cleanup.service.period.seconds" : "600", + "ksql.suppress.enabled" : "false", + "ksql.readonly.topics" : "_confluent.*,__confluent.*,_schemas,__consumer_offsets,__transaction_state,connect-configs,connect-offsets,connect-status,connect-statuses", + "ksql.connect.basic.auth.credentials.file" : "", + "ksql.authorization.cache.expiry.time.secs" : "30", + "ksql.query.retry.backoff.initial.ms" : "15000", + "ksql.query.pull.max.concurrent.requests" : "2147483647", + "ksql.streams.auto.offset.reset" : "earliest", + "ksql.connect.url" : "http://localhost:8083", + "ksql.query.push.v2.new.latest.delay.ms" : "5000", + "ksql.query.push.v2.latest.reset.age.ms" : "30000", + "ksql.streams.default.production.exception.handler" : "io.confluent.ksql.errors.ProductionExceptionHandlerUtil$LogAndFailProductionExceptionHandler", + "ksql.query.pull.interpreter.enabled" : "true", + "ksql.query.pull.limit.clause.enabled" : "true", + "ksql.query.pull.router.thread.pool.size" : "50", + "ksql.query.push.v2.continuation.tokens.enabled" : "false", + "ksql.query.retry.backoff.max.ms" : "900000", + "ksql.timestamp.throw.on.invalid" : "false", + "ksql.persistence.default.format.value" : null, + "ksql.udfs.enabled" : "true", + "ksql.udf.enable.security.manager" : "true", + "ksql.connect.worker.config" : "", + "ksql.nested.error.set.null" : "true", + "ksql.query.pull.thread.pool.size" : "50", + "ksql.persistent.prefix" : "query_", + "ksql.metastore.backup.location" : "", + "ksql.error.classifier.regex" : "", + "ksql.suppress.buffer.size.bytes" : "-1" + } +} \ No newline at end of file diff --git a/ksqldb-functional-tests/src/test/resources/historical_plans/insert-into_-_convert_formats__JSON_to_PROTOBUF_NOSR/7.3.0_1651069881670/spec.json b/ksqldb-functional-tests/src/test/resources/historical_plans/insert-into_-_convert_formats__JSON_to_PROTOBUF_NOSR/7.3.0_1651069881670/spec.json new file mode 100644 index 000000000000..4fc88e12eaf8 --- /dev/null +++ b/ksqldb-functional-tests/src/test/resources/historical_plans/insert-into_-_convert_formats__JSON_to_PROTOBUF_NOSR/7.3.0_1651069881670/spec.json @@ -0,0 +1,134 @@ +{ + "version" : "7.3.0", + "timestamp" : 1651069881670, + "path" : "query-validation-tests/insert-into.json", + "schemas" : { + "INSERTQUERY_0.KsqlTopic.Source" : { + "schema" : "`K` STRING KEY, `A` BIGINT, `B` STRING", + "keyFormat" : { + "format" : "KAFKA" + }, + "valueFormat" : { + "format" : "JSON" + } + }, + "INSERTQUERY_0.SINK" : { + "schema" : "`K` STRING KEY, `A` BIGINT, `B` STRING", + "keyFormat" : { + "format" : "KAFKA" + }, + "valueFormat" : { + "format" : "PROTOBUF_NOSR" + } + } + }, + "testCase" : { + "name" : "convert formats: JSON to PROTOBUF_NOSR", + "inputs" : [ { + "topic" : "source", + "key" : "0", + "value" : { + "A" : 123, + "B" : "falcon" + }, + "timestamp" : 0 + }, { + "topic" : "source", + "key" : "0", + "value" : { + "A" : 456, + "B" : "giraffe" + }, + "timestamp" : 0 + }, { + "topic" : "source", + "key" : "0", + "value" : { + "A" : 789, + "B" : "turtle" + }, + "timestamp" : 0 + } ], + "outputs" : [ { + "topic" : "sink", + "key" : "0", + "value" : { + "A" : 123, + "B" : "falcon" + }, + "timestamp" : 0 + }, { + "topic" : "sink", + "key" : "0", + "value" : { + "A" : 456, + "B" : "giraffe" + }, + "timestamp" : 0 + }, { + "topic" : "sink", + "key" : "0", + "value" : { + "A" : 789, + "B" : "turtle" + }, + "timestamp" : 0 + } ], + "topics" : [ { + "name" : "sink", + "replicas" : 1, + "numPartitions" : 4 + }, { + "name" : "source", + "replicas" : 1, + "numPartitions" : 4 + } ], + "statements" : [ "CREATE STREAM SOURCE (K STRING KEY, A bigint, B varchar) WITH (kafka_topic='source', value_format='JSON');", "CREATE STREAM SINK (K STRING KEY, A bigint, B varchar) WITH (kafka_topic='sink', value_format='PROTOBUF_NOSR');", "INSERT INTO SINK SELECT * FROM SOURCE;" ], + "post" : { + "sources" : [ { + "name" : "SINK", + "type" : "STREAM", + "schema" : "`K` STRING KEY, `A` BIGINT, `B` STRING", + "keyFormat" : { + "format" : "KAFKA" + }, + "valueFormat" : "PROTOBUF_NOSR", + "keyFeatures" : [ ], + "valueFeatures" : [ ], + "isSource" : false + }, { + "name" : "SOURCE", + "type" : "STREAM", + "schema" : "`K` STRING KEY, `A` BIGINT, `B` STRING", + "keyFormat" : { + "format" : "KAFKA" + }, + "valueFormat" : "JSON", + "keyFeatures" : [ ], + "valueFeatures" : [ ], + "isSource" : false + } ], + "topics" : { + "topics" : [ { + "name" : "source", + "keyFormat" : { + "format" : "KAFKA" + }, + "valueFormat" : { + "format" : "JSON" + }, + "partitions" : 4 + }, { + "name" : "sink", + "keyFormat" : { + "format" : "KAFKA" + }, + "valueFormat" : { + "format" : "PROTOBUF_NOSR" + }, + "partitions" : 4 + } ] + } + } + } +} \ No newline at end of file diff --git a/ksqldb-functional-tests/src/test/resources/historical_plans/insert-into_-_convert_formats__JSON_to_PROTOBUF_NOSR/7.3.0_1651069881670/topology b/ksqldb-functional-tests/src/test/resources/historical_plans/insert-into_-_convert_formats__JSON_to_PROTOBUF_NOSR/7.3.0_1651069881670/topology new file mode 100644 index 000000000000..f30a30881525 --- /dev/null +++ b/ksqldb-functional-tests/src/test/resources/historical_plans/insert-into_-_convert_formats__JSON_to_PROTOBUF_NOSR/7.3.0_1651069881670/topology @@ -0,0 +1,13 @@ +Topologies: + Sub-topology: 0 + Source: KSTREAM-SOURCE-0000000000 (topics: [source]) + --> KSTREAM-TRANSFORMVALUES-0000000001 + Processor: KSTREAM-TRANSFORMVALUES-0000000001 (stores: []) + --> Project + <-- KSTREAM-SOURCE-0000000000 + Processor: Project (stores: []) + --> KSTREAM-SINK-0000000003 + <-- KSTREAM-TRANSFORMVALUES-0000000001 + Sink: KSTREAM-SINK-0000000003 (topic: sink) + <-- Project + diff --git a/ksqldb-functional-tests/src/test/resources/historical_plans/insert-into_-_convert_formats__PROTOBUF_NOSR_to_JSON/7.3.0_1651069881691/plan.json b/ksqldb-functional-tests/src/test/resources/historical_plans/insert-into_-_convert_formats__PROTOBUF_NOSR_to_JSON/7.3.0_1651069881691/plan.json new file mode 100644 index 000000000000..7075ab9222c7 --- /dev/null +++ b/ksqldb-functional-tests/src/test/resources/historical_plans/insert-into_-_convert_formats__PROTOBUF_NOSR_to_JSON/7.3.0_1651069881691/plan.json @@ -0,0 +1,208 @@ +{ + "plan" : [ { + "@type" : "ksqlPlanV1", + "statementText" : "CREATE STREAM SOURCE (K STRING KEY, A BIGINT, B STRING) WITH (KAFKA_TOPIC='source', KEY_FORMAT='KAFKA', VALUE_FORMAT='PROTOBUF_NOSR');", + "ddlCommand" : { + "@type" : "createStreamV1", + "sourceName" : "SOURCE", + "schema" : "`K` STRING KEY, `A` BIGINT, `B` STRING", + "timestampColumn" : null, + "topicName" : "source", + "formats" : { + "keyFormat" : { + "format" : "KAFKA", + "properties" : { } + }, + "valueFormat" : { + "format" : "PROTOBUF_NOSR", + "properties" : { } + } + }, + "windowInfo" : null, + "orReplace" : false, + "isSource" : false + }, + "queryPlan" : null + }, { + "@type" : "ksqlPlanV1", + "statementText" : "CREATE STREAM SINK (K STRING KEY, A BIGINT, B STRING) WITH (KAFKA_TOPIC='sink', KEY_FORMAT='KAFKA', VALUE_FORMAT='JSON');", + "ddlCommand" : { + "@type" : "createStreamV1", + "sourceName" : "SINK", + "schema" : "`K` STRING KEY, `A` BIGINT, `B` STRING", + "timestampColumn" : null, + "topicName" : "sink", + "formats" : { + "keyFormat" : { + "format" : "KAFKA", + "properties" : { } + }, + "valueFormat" : { + "format" : "JSON", + "properties" : { } + } + }, + "windowInfo" : null, + "orReplace" : false, + "isSource" : false + }, + "queryPlan" : null + }, { + "@type" : "ksqlPlanV1", + "statementText" : "INSERT INTO SINK SELECT * FROM SOURCE;", + "ddlCommand" : null, + "queryPlan" : { + "sources" : [ "SOURCE" ], + "sink" : "SINK", + "physicalPlan" : { + "@type" : "streamSinkV1", + "properties" : { + "queryContext" : "SINK" + }, + "source" : { + "@type" : "streamSelectV1", + "properties" : { + "queryContext" : "Project" + }, + "source" : { + "@type" : "streamSourceV1", + "properties" : { + "queryContext" : "KsqlTopic/Source" + }, + "topicName" : "source", + "formats" : { + "keyFormat" : { + "format" : "KAFKA", + "properties" : { } + }, + "valueFormat" : { + "format" : "PROTOBUF_NOSR", + "properties" : { } + } + }, + "timestampColumn" : null, + "sourceSchema" : "`K` STRING KEY, `A` BIGINT, `B` STRING", + "pseudoColumnVersion" : 1 + }, + "keyColumnNames" : [ "K" ], + "selectedKeys" : null, + "selectExpressions" : [ "A AS A", "B AS B" ] + }, + "formats" : { + "keyFormat" : { + "format" : "KAFKA", + "properties" : { } + }, + "valueFormat" : { + "format" : "JSON", + "properties" : { } + } + }, + "topicName" : "sink", + "timestampColumn" : null + }, + "queryId" : "INSERTQUERY_0", + "runtimeId" : null + } + } ], + "configs" : { + "ksql.extension.dir" : "ext", + "ksql.streams.cache.max.bytes.buffering" : "0", + "metric.reporters" : "", + "ksql.query.status.running.threshold.seconds" : "300", + "ksql.connect.basic.auth.credentials.reload" : "false", + "ksql.output.topic.name.prefix" : "", + "ksql.query.pull.stream.enabled" : "true", + "ksql.query.push.v2.interpreter.enabled" : "true", + "ksql.queryanonymizer.logs_enabled" : "true", + "ksql.variable.substitution.enable" : "true", + "ksql.streams.shutdown.timeout.ms" : "300000", + "ksql.query.pull.max.allowed.offset.lag" : "9223372036854775807", + "ksql.query.pull.max.qps" : "2147483647", + "ksql.access.validator.enable" : "auto", + "ksql.streams.bootstrap.servers" : "localhost:0", + "ksql.query.pull.metrics.enabled" : "true", + "ksql.metrics.extension" : null, + "ksql.query.push.v2.alos.enabled" : "true", + "ksql.query.push.v2.max.hourly.bandwidth.megabytes" : "2147483647", + "ksql.query.pull.range.scan.enabled" : "true", + "ksql.transient.query.cleanup.service.initial.delay.seconds" : "600", + "ksql.hidden.topics" : "_confluent.*,__confluent.*,_schemas,__consumer_offsets,__transaction_state,connect-configs,connect-offsets,connect-status,connect-statuses", + "ksql.lambdas.enabled" : "true", + "ksql.source.table.materialization.enabled" : "true", + "ksql.query.pull.max.hourly.bandwidth.megabytes" : "2147483647", + "ksql.sink.window.change.log.additional.retention" : "1000000", + "ksql.query.persistent.active.limit" : "2147483647", + "ksql.persistence.wrap.single.values" : null, + "ksql.query.transient.max.bytes.buffering.total" : "-1", + "ksql.connect.basic.auth.credentials.source" : "NONE", + "ksql.schema.registry.url" : "schema_registry.url:0", + "ksql.properties.overrides.denylist" : "", + "ksql.service.id" : "some.ksql.service.id", + "ksql.query.push.v2.max.catchup.consumers" : "5", + "ksql.query.push.v2.enabled" : "false", + "ksql.transient.query.cleanup.service.enable" : "true", + "ksql.query.push.v2.metrics.enabled" : "true", + "ksql.rowpartition.rowoffset.enabled" : "true", + "ksql.streams.commit.interval.ms" : "2000", + "ksql.query.pull.table.scan.enabled" : "true", + "ksql.streams.auto.commit.interval.ms" : "0", + "ksql.streams.topology.optimization" : "all", + "ksql.endpoint.migrate.query" : "true", + "ksql.query.push.v2.registry.installed" : "false", + "ksql.streams.num.stream.threads" : "4", + "ksql.metrics.tags.custom" : "", + "ksql.query.push.v2.catchup.consumer.msg.window" : "50", + "ksql.runtime.feature.shared.enabled" : "false", + "ksql.udf.collect.metrics" : "false", + "ksql.new.query.planner.enabled" : "false", + "ksql.connect.request.headers.plugin" : null, + "ksql.security.extension.class" : null, + "ksql.transient.prefix" : "transient_", + "ksql.headers.columns.enabled" : "true", + "ksql.streams.default.deserialization.exception.handler" : "io.confluent.ksql.errors.LogMetricAndContinueExceptionHandler", + "ksql.connect.request.timeout.ms" : "5000", + "ksql.query.pull.enable.standby.reads" : "false", + "ksql.persistence.default.format.key" : "KAFKA", + "ksql.query.persistent.max.bytes.buffering.total" : "-1", + "ksql.query.error.max.queue.size" : "10", + "ksql.query.cleanup.shutdown.timeout.ms" : "30000", + "ksql.internal.topic.min.insync.replicas" : "1", + "ksql.internal.topic.replicas" : "1", + "ksql.insert.into.values.enabled" : "true", + "ksql.queryanonymizer.cluster_namespace" : null, + "ksql.create.or.replace.enabled" : "true", + "ksql.shared.runtimes.count" : "2", + "ksql.cast.strings.preserve.nulls" : "true", + "ksql.authorization.cache.max.entries" : "10000", + "ksql.pull.queries.enable" : "true", + "ksql.transient.query.cleanup.service.period.seconds" : "600", + "ksql.suppress.enabled" : "false", + "ksql.readonly.topics" : "_confluent.*,__confluent.*,_schemas,__consumer_offsets,__transaction_state,connect-configs,connect-offsets,connect-status,connect-statuses", + "ksql.connect.basic.auth.credentials.file" : "", + "ksql.authorization.cache.expiry.time.secs" : "30", + "ksql.query.retry.backoff.initial.ms" : "15000", + "ksql.query.pull.max.concurrent.requests" : "2147483647", + "ksql.streams.auto.offset.reset" : "earliest", + "ksql.connect.url" : "http://localhost:8083", + "ksql.query.push.v2.new.latest.delay.ms" : "5000", + "ksql.query.push.v2.latest.reset.age.ms" : "30000", + "ksql.streams.default.production.exception.handler" : "io.confluent.ksql.errors.ProductionExceptionHandlerUtil$LogAndFailProductionExceptionHandler", + "ksql.query.pull.interpreter.enabled" : "true", + "ksql.query.pull.limit.clause.enabled" : "true", + "ksql.query.pull.router.thread.pool.size" : "50", + "ksql.query.push.v2.continuation.tokens.enabled" : "false", + "ksql.query.retry.backoff.max.ms" : "900000", + "ksql.timestamp.throw.on.invalid" : "false", + "ksql.persistence.default.format.value" : null, + "ksql.udfs.enabled" : "true", + "ksql.udf.enable.security.manager" : "true", + "ksql.connect.worker.config" : "", + "ksql.nested.error.set.null" : "true", + "ksql.query.pull.thread.pool.size" : "50", + "ksql.persistent.prefix" : "query_", + "ksql.metastore.backup.location" : "", + "ksql.error.classifier.regex" : "", + "ksql.suppress.buffer.size.bytes" : "-1" + } +} \ No newline at end of file diff --git a/ksqldb-functional-tests/src/test/resources/historical_plans/insert-into_-_convert_formats__PROTOBUF_NOSR_to_JSON/7.3.0_1651069881691/spec.json b/ksqldb-functional-tests/src/test/resources/historical_plans/insert-into_-_convert_formats__PROTOBUF_NOSR_to_JSON/7.3.0_1651069881691/spec.json new file mode 100644 index 000000000000..e84ccb6b5c78 --- /dev/null +++ b/ksqldb-functional-tests/src/test/resources/historical_plans/insert-into_-_convert_formats__PROTOBUF_NOSR_to_JSON/7.3.0_1651069881691/spec.json @@ -0,0 +1,134 @@ +{ + "version" : "7.3.0", + "timestamp" : 1651069881691, + "path" : "query-validation-tests/insert-into.json", + "schemas" : { + "INSERTQUERY_0.KsqlTopic.Source" : { + "schema" : "`K` STRING KEY, `A` BIGINT, `B` STRING", + "keyFormat" : { + "format" : "KAFKA" + }, + "valueFormat" : { + "format" : "PROTOBUF_NOSR" + } + }, + "INSERTQUERY_0.SINK" : { + "schema" : "`K` STRING KEY, `A` BIGINT, `B` STRING", + "keyFormat" : { + "format" : "KAFKA" + }, + "valueFormat" : { + "format" : "JSON" + } + } + }, + "testCase" : { + "name" : "convert formats: PROTOBUF_NOSR to JSON", + "inputs" : [ { + "topic" : "source", + "key" : "0", + "value" : { + "A" : 123, + "B" : "falcon" + }, + "timestamp" : 0 + }, { + "topic" : "source", + "key" : "0", + "value" : { + "A" : 456, + "B" : "giraffe" + }, + "timestamp" : 0 + }, { + "topic" : "source", + "key" : "0", + "value" : { + "A" : 789, + "B" : "turtle" + }, + "timestamp" : 0 + } ], + "outputs" : [ { + "topic" : "sink", + "key" : "0", + "value" : { + "A" : 123, + "B" : "falcon" + }, + "timestamp" : 0 + }, { + "topic" : "sink", + "key" : "0", + "value" : { + "A" : 456, + "B" : "giraffe" + }, + "timestamp" : 0 + }, { + "topic" : "sink", + "key" : "0", + "value" : { + "A" : 789, + "B" : "turtle" + }, + "timestamp" : 0 + } ], + "topics" : [ { + "name" : "sink", + "replicas" : 1, + "numPartitions" : 4 + }, { + "name" : "source", + "replicas" : 1, + "numPartitions" : 4 + } ], + "statements" : [ "CREATE STREAM SOURCE (K STRING KEY, A bigint, B varchar) WITH (kafka_topic='source', value_format='PROTOBUF_NOSR');", "CREATE STREAM SINK (K STRING KEY, A bigint, B varchar) WITH (kafka_topic='sink', value_format='JSON');", "INSERT INTO SINK SELECT * FROM SOURCE;" ], + "post" : { + "sources" : [ { + "name" : "SINK", + "type" : "STREAM", + "schema" : "`K` STRING KEY, `A` BIGINT, `B` STRING", + "keyFormat" : { + "format" : "KAFKA" + }, + "valueFormat" : "JSON", + "keyFeatures" : [ ], + "valueFeatures" : [ ], + "isSource" : false + }, { + "name" : "SOURCE", + "type" : "STREAM", + "schema" : "`K` STRING KEY, `A` BIGINT, `B` STRING", + "keyFormat" : { + "format" : "KAFKA" + }, + "valueFormat" : "PROTOBUF_NOSR", + "keyFeatures" : [ ], + "valueFeatures" : [ ], + "isSource" : false + } ], + "topics" : { + "topics" : [ { + "name" : "source", + "keyFormat" : { + "format" : "KAFKA" + }, + "valueFormat" : { + "format" : "PROTOBUF_NOSR" + }, + "partitions" : 4 + }, { + "name" : "sink", + "keyFormat" : { + "format" : "KAFKA" + }, + "valueFormat" : { + "format" : "JSON" + }, + "partitions" : 4 + } ] + } + } + } +} \ No newline at end of file diff --git a/ksqldb-functional-tests/src/test/resources/historical_plans/insert-into_-_convert_formats__PROTOBUF_NOSR_to_JSON/7.3.0_1651069881691/topology b/ksqldb-functional-tests/src/test/resources/historical_plans/insert-into_-_convert_formats__PROTOBUF_NOSR_to_JSON/7.3.0_1651069881691/topology new file mode 100644 index 000000000000..f30a30881525 --- /dev/null +++ b/ksqldb-functional-tests/src/test/resources/historical_plans/insert-into_-_convert_formats__PROTOBUF_NOSR_to_JSON/7.3.0_1651069881691/topology @@ -0,0 +1,13 @@ +Topologies: + Sub-topology: 0 + Source: KSTREAM-SOURCE-0000000000 (topics: [source]) + --> KSTREAM-TRANSFORMVALUES-0000000001 + Processor: KSTREAM-TRANSFORMVALUES-0000000001 (stores: []) + --> Project + <-- KSTREAM-SOURCE-0000000000 + Processor: Project (stores: []) + --> KSTREAM-SINK-0000000003 + <-- KSTREAM-TRANSFORMVALUES-0000000001 + Sink: KSTREAM-SINK-0000000003 (topic: sink) + <-- Project + diff --git a/ksqldb-functional-tests/src/test/resources/historical_plans/join-with-custom-timestamp_-_stream_stream_inner_join_with_ts_-_PROTOBUF_NOSR/7.3.0_1651070051134/plan.json b/ksqldb-functional-tests/src/test/resources/historical_plans/join-with-custom-timestamp_-_stream_stream_inner_join_with_ts_-_PROTOBUF_NOSR/7.3.0_1651070051134/plan.json new file mode 100644 index 000000000000..90669e03b87a --- /dev/null +++ b/ksqldb-functional-tests/src/test/resources/historical_plans/join-with-custom-timestamp_-_stream_stream_inner_join_with_ts_-_PROTOBUF_NOSR/7.3.0_1651070051134/plan.json @@ -0,0 +1,308 @@ +{ + "plan" : [ { + "@type" : "ksqlPlanV1", + "statementText" : "CREATE STREAM S1 (ID BIGINT KEY, NAME STRING, TS BIGINT) WITH (KAFKA_TOPIC='s1', KEY_FORMAT='KAFKA', TIMESTAMP='TS', VALUE_FORMAT='PROTOBUF_NOSR');", + "ddlCommand" : { + "@type" : "createStreamV1", + "sourceName" : "S1", + "schema" : "`ID` BIGINT KEY, `NAME` STRING, `TS` BIGINT", + "timestampColumn" : { + "column" : "TS", + "format" : null + }, + "topicName" : "s1", + "formats" : { + "keyFormat" : { + "format" : "KAFKA", + "properties" : { } + }, + "valueFormat" : { + "format" : "PROTOBUF_NOSR", + "properties" : { } + } + }, + "windowInfo" : null, + "orReplace" : false, + "isSource" : false + }, + "queryPlan" : null + }, { + "@type" : "ksqlPlanV1", + "statementText" : "CREATE STREAM S2 (ID BIGINT KEY, F1 STRING, F2 STRING) WITH (KAFKA_TOPIC='s2', KEY_FORMAT='KAFKA', VALUE_FORMAT='PROTOBUF_NOSR');", + "ddlCommand" : { + "@type" : "createStreamV1", + "sourceName" : "S2", + "schema" : "`ID` BIGINT KEY, `F1` STRING, `F2` STRING", + "timestampColumn" : null, + "topicName" : "s2", + "formats" : { + "keyFormat" : { + "format" : "KAFKA", + "properties" : { } + }, + "valueFormat" : { + "format" : "PROTOBUF_NOSR", + "properties" : { } + } + }, + "windowInfo" : null, + "orReplace" : false, + "isSource" : false + }, + "queryPlan" : null + }, { + "@type" : "ksqlPlanV1", + "statementText" : "CREATE STREAM S1_JOIN_S2 WITH (TIMESTAMP='TS') AS SELECT\n S1.ID S1_ID,\n S1.NAME NAME,\n S1.TS TS,\n S2.F1 F1,\n S2.F2 F2\nFROM S1 S1\nINNER JOIN S2 S2 WITHIN 11 SECONDS ON ((S1.ID = S2.ID))\nEMIT CHANGES", + "ddlCommand" : { + "@type" : "createStreamV1", + "sourceName" : "S1_JOIN_S2", + "schema" : "`S1_ID` BIGINT KEY, `NAME` STRING, `TS` BIGINT, `F1` STRING, `F2` STRING", + "timestampColumn" : { + "column" : "TS", + "format" : null + }, + "topicName" : "S1_JOIN_S2", + "formats" : { + "keyFormat" : { + "format" : "KAFKA", + "properties" : { } + }, + "valueFormat" : { + "format" : "PROTOBUF_NOSR", + "properties" : { } + } + }, + "windowInfo" : null, + "orReplace" : false, + "isSource" : false + }, + "queryPlan" : { + "sources" : [ "S1", "S2" ], + "sink" : "S1_JOIN_S2", + "physicalPlan" : { + "@type" : "streamSinkV1", + "properties" : { + "queryContext" : "S1_JOIN_S2" + }, + "source" : { + "@type" : "streamSelectV1", + "properties" : { + "queryContext" : "Project" + }, + "source" : { + "@type" : "streamStreamJoinV1", + "properties" : { + "queryContext" : "Join" + }, + "joinType" : "INNER", + "leftInternalFormats" : { + "keyFormat" : { + "format" : "KAFKA", + "properties" : { } + }, + "valueFormat" : { + "format" : "PROTOBUF_NOSR", + "properties" : { } + } + }, + "rightInternalFormats" : { + "keyFormat" : { + "format" : "KAFKA", + "properties" : { } + }, + "valueFormat" : { + "format" : "PROTOBUF_NOSR", + "properties" : { } + } + }, + "leftSource" : { + "@type" : "streamSelectV1", + "properties" : { + "queryContext" : "PrependAliasLeft" + }, + "source" : { + "@type" : "streamSourceV1", + "properties" : { + "queryContext" : "KafkaTopic_Left/Source" + }, + "topicName" : "s1", + "formats" : { + "keyFormat" : { + "format" : "KAFKA", + "properties" : { } + }, + "valueFormat" : { + "format" : "PROTOBUF_NOSR", + "properties" : { } + } + }, + "timestampColumn" : { + "column" : "TS", + "format" : null + }, + "sourceSchema" : "`ID` BIGINT KEY, `NAME` STRING, `TS` BIGINT", + "pseudoColumnVersion" : 1 + }, + "keyColumnNames" : [ "S1_ID" ], + "selectedKeys" : null, + "selectExpressions" : [ "NAME AS S1_NAME", "TS AS S1_TS", "ROWTIME AS S1_ROWTIME", "ROWPARTITION AS S1_ROWPARTITION", "ROWOFFSET AS S1_ROWOFFSET", "ID AS S1_ID" ] + }, + "rightSource" : { + "@type" : "streamSelectV1", + "properties" : { + "queryContext" : "PrependAliasRight" + }, + "source" : { + "@type" : "streamSourceV1", + "properties" : { + "queryContext" : "KafkaTopic_Right/Source" + }, + "topicName" : "s2", + "formats" : { + "keyFormat" : { + "format" : "KAFKA", + "properties" : { } + }, + "valueFormat" : { + "format" : "PROTOBUF_NOSR", + "properties" : { } + } + }, + "timestampColumn" : null, + "sourceSchema" : "`ID` BIGINT KEY, `F1` STRING, `F2` STRING", + "pseudoColumnVersion" : 1 + }, + "keyColumnNames" : [ "S2_ID" ], + "selectedKeys" : null, + "selectExpressions" : [ "F1 AS S2_F1", "F2 AS S2_F2", "ROWTIME AS S2_ROWTIME", "ROWPARTITION AS S2_ROWPARTITION", "ROWOFFSET AS S2_ROWOFFSET", "ID AS S2_ID" ] + }, + "beforeMillis" : 11.000000000, + "afterMillis" : 11.000000000, + "graceMillis" : null, + "keyColName" : "S1_ID" + }, + "keyColumnNames" : [ "S1_ID" ], + "selectedKeys" : null, + "selectExpressions" : [ "S1_NAME AS NAME", "S1_TS AS TS", "S2_F1 AS F1", "S2_F2 AS F2" ] + }, + "formats" : { + "keyFormat" : { + "format" : "KAFKA", + "properties" : { } + }, + "valueFormat" : { + "format" : "PROTOBUF_NOSR", + "properties" : { } + } + }, + "topicName" : "S1_JOIN_S2", + "timestampColumn" : { + "column" : "TS", + "format" : null + } + }, + "queryId" : "CSAS_S1_JOIN_S2_0", + "runtimeId" : null + } + } ], + "configs" : { + "ksql.extension.dir" : "ext", + "ksql.streams.cache.max.bytes.buffering" : "0", + "metric.reporters" : "", + "ksql.query.status.running.threshold.seconds" : "300", + "ksql.connect.basic.auth.credentials.reload" : "false", + "ksql.output.topic.name.prefix" : "", + "ksql.query.pull.stream.enabled" : "true", + "ksql.query.push.v2.interpreter.enabled" : "true", + "ksql.queryanonymizer.logs_enabled" : "true", + "ksql.variable.substitution.enable" : "true", + "ksql.streams.shutdown.timeout.ms" : "300000", + "ksql.query.pull.max.allowed.offset.lag" : "9223372036854775807", + "ksql.query.pull.max.qps" : "2147483647", + "ksql.access.validator.enable" : "auto", + "ksql.streams.bootstrap.servers" : "localhost:0", + "ksql.query.pull.metrics.enabled" : "true", + "ksql.metrics.extension" : null, + "ksql.query.push.v2.alos.enabled" : "true", + "ksql.query.push.v2.max.hourly.bandwidth.megabytes" : "2147483647", + "ksql.query.pull.range.scan.enabled" : "true", + "ksql.transient.query.cleanup.service.initial.delay.seconds" : "600", + "ksql.hidden.topics" : "_confluent.*,__confluent.*,_schemas,__consumer_offsets,__transaction_state,connect-configs,connect-offsets,connect-status,connect-statuses", + "ksql.lambdas.enabled" : "true", + "ksql.source.table.materialization.enabled" : "true", + "ksql.query.pull.max.hourly.bandwidth.megabytes" : "2147483647", + "ksql.sink.window.change.log.additional.retention" : "1000000", + "ksql.query.persistent.active.limit" : "2147483647", + "ksql.persistence.wrap.single.values" : null, + "ksql.query.transient.max.bytes.buffering.total" : "-1", + "ksql.connect.basic.auth.credentials.source" : "NONE", + "ksql.schema.registry.url" : "schema_registry.url:0", + "ksql.properties.overrides.denylist" : "", + "ksql.service.id" : "some.ksql.service.id", + "ksql.query.push.v2.max.catchup.consumers" : "5", + "ksql.query.push.v2.enabled" : "false", + "ksql.transient.query.cleanup.service.enable" : "true", + "ksql.query.push.v2.metrics.enabled" : "true", + "ksql.rowpartition.rowoffset.enabled" : "true", + "ksql.streams.commit.interval.ms" : "2000", + "ksql.query.pull.table.scan.enabled" : "true", + "ksql.streams.auto.commit.interval.ms" : "0", + "ksql.streams.topology.optimization" : "all", + "ksql.endpoint.migrate.query" : "true", + "ksql.query.push.v2.registry.installed" : "false", + "ksql.streams.num.stream.threads" : "4", + "ksql.metrics.tags.custom" : "", + "ksql.query.push.v2.catchup.consumer.msg.window" : "50", + "ksql.runtime.feature.shared.enabled" : "false", + "ksql.udf.collect.metrics" : "false", + "ksql.new.query.planner.enabled" : "false", + "ksql.connect.request.headers.plugin" : null, + "ksql.security.extension.class" : null, + "ksql.transient.prefix" : "transient_", + "ksql.headers.columns.enabled" : "true", + "ksql.streams.default.deserialization.exception.handler" : "io.confluent.ksql.errors.LogMetricAndContinueExceptionHandler", + "ksql.connect.request.timeout.ms" : "5000", + "ksql.query.pull.enable.standby.reads" : "false", + "ksql.persistence.default.format.key" : "KAFKA", + "ksql.query.persistent.max.bytes.buffering.total" : "-1", + "ksql.query.error.max.queue.size" : "10", + "ksql.query.cleanup.shutdown.timeout.ms" : "30000", + "ksql.internal.topic.min.insync.replicas" : "1", + "ksql.internal.topic.replicas" : "1", + "ksql.insert.into.values.enabled" : "true", + "ksql.queryanonymizer.cluster_namespace" : null, + "ksql.create.or.replace.enabled" : "true", + "ksql.shared.runtimes.count" : "2", + "ksql.cast.strings.preserve.nulls" : "true", + "ksql.authorization.cache.max.entries" : "10000", + "ksql.pull.queries.enable" : "true", + "ksql.transient.query.cleanup.service.period.seconds" : "600", + "ksql.suppress.enabled" : "false", + "ksql.readonly.topics" : "_confluent.*,__confluent.*,_schemas,__consumer_offsets,__transaction_state,connect-configs,connect-offsets,connect-status,connect-statuses", + "ksql.connect.basic.auth.credentials.file" : "", + "ksql.authorization.cache.expiry.time.secs" : "30", + "ksql.query.retry.backoff.initial.ms" : "15000", + "ksql.query.pull.max.concurrent.requests" : "2147483647", + "ksql.streams.auto.offset.reset" : "earliest", + "ksql.connect.url" : "http://localhost:8083", + "ksql.query.push.v2.new.latest.delay.ms" : "5000", + "ksql.query.push.v2.latest.reset.age.ms" : "30000", + "ksql.streams.default.production.exception.handler" : "io.confluent.ksql.errors.ProductionExceptionHandlerUtil$LogAndFailProductionExceptionHandler", + "ksql.query.pull.interpreter.enabled" : "true", + "ksql.query.pull.limit.clause.enabled" : "true", + "ksql.query.pull.router.thread.pool.size" : "50", + "ksql.query.push.v2.continuation.tokens.enabled" : "false", + "ksql.query.retry.backoff.max.ms" : "900000", + "ksql.timestamp.throw.on.invalid" : "false", + "ksql.persistence.default.format.value" : null, + "ksql.udfs.enabled" : "true", + "ksql.udf.enable.security.manager" : "true", + "ksql.connect.worker.config" : "", + "ksql.nested.error.set.null" : "true", + "ksql.query.pull.thread.pool.size" : "50", + "ksql.persistent.prefix" : "query_", + "ksql.metastore.backup.location" : "", + "ksql.error.classifier.regex" : "", + "ksql.suppress.buffer.size.bytes" : "-1" + } +} \ No newline at end of file diff --git a/ksqldb-functional-tests/src/test/resources/historical_plans/join-with-custom-timestamp_-_stream_stream_inner_join_with_ts_-_PROTOBUF_NOSR/7.3.0_1651070051134/spec.json b/ksqldb-functional-tests/src/test/resources/historical_plans/join-with-custom-timestamp_-_stream_stream_inner_join_with_ts_-_PROTOBUF_NOSR/7.3.0_1651070051134/spec.json new file mode 100644 index 000000000000..28ea0d4698f9 --- /dev/null +++ b/ksqldb-functional-tests/src/test/resources/historical_plans/join-with-custom-timestamp_-_stream_stream_inner_join_with_ts_-_PROTOBUF_NOSR/7.3.0_1651070051134/spec.json @@ -0,0 +1,223 @@ +{ + "version" : "7.3.0", + "timestamp" : 1651070051134, + "path" : "query-validation-tests/join-with-custom-timestamp.json", + "schemas" : { + "CSAS_S1_JOIN_S2_0.KafkaTopic_Left.Source" : { + "schema" : "`ID` BIGINT KEY, `NAME` STRING, `TS` BIGINT", + "keyFormat" : { + "format" : "KAFKA" + }, + "valueFormat" : { + "format" : "PROTOBUF_NOSR" + } + }, + "CSAS_S1_JOIN_S2_0.Join.Right" : { + "schema" : "`S2_ID` BIGINT KEY, `S2_F1` STRING, `S2_F2` STRING, `S2_ROWTIME` BIGINT, `S2_ROWPARTITION` INTEGER, `S2_ROWOFFSET` BIGINT, `S2_ID` BIGINT", + "valueFormat" : { + "format" : "PROTOBUF_NOSR" + } + }, + "CSAS_S1_JOIN_S2_0.Join.Left" : { + "schema" : "`S1_ID` BIGINT KEY, `S1_NAME` STRING, `S1_TS` BIGINT, `S1_ROWTIME` BIGINT, `S1_ROWPARTITION` INTEGER, `S1_ROWOFFSET` BIGINT, `S1_ID` BIGINT", + "valueFormat" : { + "format" : "PROTOBUF_NOSR" + } + }, + "CSAS_S1_JOIN_S2_0.S1_JOIN_S2" : { + "schema" : "`S1_ID` BIGINT KEY, `NAME` STRING, `TS` BIGINT, `F1` STRING, `F2` STRING", + "keyFormat" : { + "format" : "KAFKA" + }, + "valueFormat" : { + "format" : "PROTOBUF_NOSR" + } + }, + "CSAS_S1_JOIN_S2_0.Join" : { + "schema" : "`S1_ID` BIGINT KEY, `S1_NAME` STRING, `S1_TS` BIGINT, `S1_ROWTIME` BIGINT, `S1_ROWPARTITION` INTEGER, `S1_ROWOFFSET` BIGINT, `S1_ID` BIGINT", + "keyFormat" : { + "format" : "KAFKA" + } + }, + "CSAS_S1_JOIN_S2_0.KafkaTopic_Right.Source" : { + "schema" : "`ID` BIGINT KEY, `F1` STRING, `F2` STRING", + "keyFormat" : { + "format" : "KAFKA" + }, + "valueFormat" : { + "format" : "PROTOBUF_NOSR" + } + } + }, + "testCase" : { + "name" : "stream stream inner join with ts - PROTOBUF_NOSR", + "inputs" : [ { + "topic" : "s1", + "key" : 0, + "value" : { + "NAME" : "zero", + "TS" : 0 + }, + "timestamp" : 0 + }, { + "topic" : "s2", + "key" : 0, + "value" : { + "F1" : "blah", + "F2" : "foo" + }, + "timestamp" : 10000 + }, { + "topic" : "s2", + "key" : 10, + "value" : { + "F1" : "foo", + "F2" : "bar" + }, + "timestamp" : 13000 + }, { + "topic" : "s1", + "key" : 10, + "value" : { + "NAME" : "100", + "TS" : 11000 + }, + "timestamp" : 22000 + }, { + "topic" : "s1", + "key" : 0, + "value" : { + "NAME" : "jan", + "TS" : 8000 + }, + "timestamp" : 33000 + } ], + "outputs" : [ { + "topic" : "S1_JOIN_S2", + "key" : 0, + "value" : { + "NAME" : "zero", + "TS" : 0, + "F1" : "blah", + "F2" : "foo" + }, + "timestamp" : 0 + }, { + "topic" : "S1_JOIN_S2", + "key" : 10, + "value" : { + "NAME" : "100", + "TS" : 11000, + "F1" : "foo", + "F2" : "bar" + }, + "timestamp" : 11000 + }, { + "topic" : "S1_JOIN_S2", + "key" : 0, + "value" : { + "NAME" : "jan", + "TS" : 8000, + "F1" : "blah", + "F2" : "foo" + }, + "timestamp" : 8000 + } ], + "topics" : [ { + "name" : "S1_JOIN_S2", + "replicas" : 1, + "numPartitions" : 4 + }, { + "name" : "s1", + "replicas" : 1, + "numPartitions" : 4 + }, { + "name" : "s2", + "replicas" : 1, + "numPartitions" : 4 + } ], + "statements" : [ "CREATE STREAM S1 (ID BIGINT KEY, NAME varchar, TS bigint) WITH (timestamp='TS', kafka_topic='s1', value_format='PROTOBUF_NOSR');", "CREATE STREAM S2 (ID BIGINT KEY, F1 varchar, F2 varchar) WITH (kafka_topic='s2', value_format='PROTOBUF_NOSR');", "CREATE STREAM S1_JOIN_S2 WITH(timestamp='TS') as SELECT S1.ID, S1.name as name, S1.ts as ts, s2.f1, s2.f2 from S1 join S2 WITHIN 11 SECONDS ON s1.id = s2.id;" ], + "post" : { + "sources" : [ { + "name" : "S1", + "type" : "STREAM", + "schema" : "`ID` BIGINT KEY, `NAME` STRING, `TS` BIGINT", + "keyFormat" : { + "format" : "KAFKA" + }, + "valueFormat" : "PROTOBUF_NOSR", + "keyFeatures" : [ ], + "valueFeatures" : [ ], + "isSource" : false + }, { + "name" : "S1_JOIN_S2", + "type" : "STREAM", + "schema" : "`S1_ID` BIGINT KEY, `NAME` STRING, `TS` BIGINT, `F1` STRING, `F2` STRING", + "keyFormat" : { + "format" : "KAFKA" + }, + "valueFormat" : "PROTOBUF_NOSR", + "keyFeatures" : [ ], + "valueFeatures" : [ ], + "isSource" : false + }, { + "name" : "S2", + "type" : "STREAM", + "schema" : "`ID` BIGINT KEY, `F1` STRING, `F2` STRING", + "keyFormat" : { + "format" : "KAFKA" + }, + "valueFormat" : "PROTOBUF_NOSR", + "keyFeatures" : [ ], + "valueFeatures" : [ ], + "isSource" : false + } ], + "topics" : { + "topics" : [ { + "name" : "_confluent-ksql-some.ksql.service.idquery_CSAS_S1_JOIN_S2_0-KSTREAM-JOINOTHER-0000000009-store-changelog", + "keyFormat" : { + "format" : "KAFKA" + }, + "valueFormat" : { + "format" : "PROTOBUF_NOSR" + } + }, { + "name" : "s2", + "keyFormat" : { + "format" : "KAFKA" + }, + "valueFormat" : { + "format" : "PROTOBUF_NOSR" + }, + "partitions" : 4 + }, { + "name" : "s1", + "keyFormat" : { + "format" : "KAFKA" + }, + "valueFormat" : { + "format" : "PROTOBUF_NOSR" + }, + "partitions" : 4 + }, { + "name" : "S1_JOIN_S2", + "keyFormat" : { + "format" : "KAFKA" + }, + "valueFormat" : { + "format" : "PROTOBUF_NOSR" + }, + "partitions" : 4 + }, { + "name" : "_confluent-ksql-some.ksql.service.idquery_CSAS_S1_JOIN_S2_0-KSTREAM-JOINTHIS-0000000008-store-changelog", + "keyFormat" : { + "format" : "KAFKA" + }, + "valueFormat" : { + "format" : "PROTOBUF_NOSR" + } + } ] + } + } + } +} \ No newline at end of file diff --git a/ksqldb-functional-tests/src/test/resources/historical_plans/join-with-custom-timestamp_-_stream_stream_inner_join_with_ts_-_PROTOBUF_NOSR/7.3.0_1651070051134/topology b/ksqldb-functional-tests/src/test/resources/historical_plans/join-with-custom-timestamp_-_stream_stream_inner_join_with_ts_-_PROTOBUF_NOSR/7.3.0_1651070051134/topology new file mode 100644 index 000000000000..f74bc03af76a --- /dev/null +++ b/ksqldb-functional-tests/src/test/resources/historical_plans/join-with-custom-timestamp_-_stream_stream_inner_join_with_ts_-_PROTOBUF_NOSR/7.3.0_1651070051134/topology @@ -0,0 +1,42 @@ +Topologies: + Sub-topology: 0 + Source: KSTREAM-SOURCE-0000000000 (topics: [s1]) + --> KSTREAM-TRANSFORMVALUES-0000000001 + Source: KSTREAM-SOURCE-0000000003 (topics: [s2]) + --> KSTREAM-TRANSFORMVALUES-0000000004 + Processor: KSTREAM-TRANSFORMVALUES-0000000001 (stores: []) + --> PrependAliasLeft + <-- KSTREAM-SOURCE-0000000000 + Processor: KSTREAM-TRANSFORMVALUES-0000000004 (stores: []) + --> PrependAliasRight + <-- KSTREAM-SOURCE-0000000003 + Processor: PrependAliasLeft (stores: []) + --> Join-this-windowed + <-- KSTREAM-TRANSFORMVALUES-0000000001 + Processor: PrependAliasRight (stores: []) + --> Join-other-windowed + <-- KSTREAM-TRANSFORMVALUES-0000000004 + Processor: Join-other-windowed (stores: [KSTREAM-JOINOTHER-0000000009-store]) + --> Join-other-join + <-- PrependAliasRight + Processor: Join-this-windowed (stores: [KSTREAM-JOINTHIS-0000000008-store]) + --> Join-this-join + <-- PrependAliasLeft + Processor: Join-other-join (stores: [KSTREAM-JOINTHIS-0000000008-store]) + --> Join-merge + <-- Join-other-windowed + Processor: Join-this-join (stores: [KSTREAM-JOINOTHER-0000000009-store]) + --> Join-merge + <-- Join-this-windowed + Processor: Join-merge (stores: []) + --> Project + <-- Join-this-join, Join-other-join + Processor: Project (stores: []) + --> ApplyTimestampTransform-S1_JOIN_S2 + <-- Join-merge + Processor: ApplyTimestampTransform-S1_JOIN_S2 (stores: []) + --> KSTREAM-SINK-0000000012 + <-- Project + Sink: KSTREAM-SINK-0000000012 (topic: S1_JOIN_S2) + <-- ApplyTimestampTransform-S1_JOIN_S2 + diff --git a/ksqldb-functional-tests/src/test/resources/historical_plans/join-with-custom-timestamp_-_stream_stream_inner_join_with_ts_extractor_both_sides_-_PROTOBUF_NOSR/7.3.0_1651070054569/plan.json b/ksqldb-functional-tests/src/test/resources/historical_plans/join-with-custom-timestamp_-_stream_stream_inner_join_with_ts_extractor_both_sides_-_PROTOBUF_NOSR/7.3.0_1651070054569/plan.json new file mode 100644 index 000000000000..84f08b90e912 --- /dev/null +++ b/ksqldb-functional-tests/src/test/resources/historical_plans/join-with-custom-timestamp_-_stream_stream_inner_join_with_ts_extractor_both_sides_-_PROTOBUF_NOSR/7.3.0_1651070054569/plan.json @@ -0,0 +1,314 @@ +{ + "plan" : [ { + "@type" : "ksqlPlanV1", + "statementText" : "CREATE STREAM S1 (ID BIGINT KEY, NAME STRING, TS BIGINT) WITH (KAFKA_TOPIC='s1', KEY_FORMAT='KAFKA', TIMESTAMP='TS', VALUE_FORMAT='PROTOBUF_NOSR');", + "ddlCommand" : { + "@type" : "createStreamV1", + "sourceName" : "S1", + "schema" : "`ID` BIGINT KEY, `NAME` STRING, `TS` BIGINT", + "timestampColumn" : { + "column" : "TS", + "format" : null + }, + "topicName" : "s1", + "formats" : { + "keyFormat" : { + "format" : "KAFKA", + "properties" : { } + }, + "valueFormat" : { + "format" : "PROTOBUF_NOSR", + "properties" : { } + } + }, + "windowInfo" : null, + "orReplace" : false, + "isSource" : false + }, + "queryPlan" : null + }, { + "@type" : "ksqlPlanV1", + "statementText" : "CREATE STREAM S2 (ID BIGINT KEY, F1 STRING, F2 STRING, RTS BIGINT) WITH (KAFKA_TOPIC='s2', KEY_FORMAT='KAFKA', TIMESTAMP='RTS', VALUE_FORMAT='PROTOBUF_NOSR');", + "ddlCommand" : { + "@type" : "createStreamV1", + "sourceName" : "S2", + "schema" : "`ID` BIGINT KEY, `F1` STRING, `F2` STRING, `RTS` BIGINT", + "timestampColumn" : { + "column" : "RTS", + "format" : null + }, + "topicName" : "s2", + "formats" : { + "keyFormat" : { + "format" : "KAFKA", + "properties" : { } + }, + "valueFormat" : { + "format" : "PROTOBUF_NOSR", + "properties" : { } + } + }, + "windowInfo" : null, + "orReplace" : false, + "isSource" : false + }, + "queryPlan" : null + }, { + "@type" : "ksqlPlanV1", + "statementText" : "CREATE STREAM S1_JOIN_S2 WITH (TIMESTAMP='TS') AS SELECT\n S1.ID S1_ID,\n S1.NAME NAME,\n S1.TS TS,\n S2.F1 F1,\n S2.F2 F2\nFROM S1 S1\nINNER JOIN S2 S2 WITHIN 11 SECONDS ON ((S1.ID = S2.ID))\nEMIT CHANGES", + "ddlCommand" : { + "@type" : "createStreamV1", + "sourceName" : "S1_JOIN_S2", + "schema" : "`S1_ID` BIGINT KEY, `NAME` STRING, `TS` BIGINT, `F1` STRING, `F2` STRING", + "timestampColumn" : { + "column" : "TS", + "format" : null + }, + "topicName" : "S1_JOIN_S2", + "formats" : { + "keyFormat" : { + "format" : "KAFKA", + "properties" : { } + }, + "valueFormat" : { + "format" : "PROTOBUF_NOSR", + "properties" : { } + } + }, + "windowInfo" : null, + "orReplace" : false, + "isSource" : false + }, + "queryPlan" : { + "sources" : [ "S1", "S2" ], + "sink" : "S1_JOIN_S2", + "physicalPlan" : { + "@type" : "streamSinkV1", + "properties" : { + "queryContext" : "S1_JOIN_S2" + }, + "source" : { + "@type" : "streamSelectV1", + "properties" : { + "queryContext" : "Project" + }, + "source" : { + "@type" : "streamStreamJoinV1", + "properties" : { + "queryContext" : "Join" + }, + "joinType" : "INNER", + "leftInternalFormats" : { + "keyFormat" : { + "format" : "KAFKA", + "properties" : { } + }, + "valueFormat" : { + "format" : "PROTOBUF_NOSR", + "properties" : { } + } + }, + "rightInternalFormats" : { + "keyFormat" : { + "format" : "KAFKA", + "properties" : { } + }, + "valueFormat" : { + "format" : "PROTOBUF_NOSR", + "properties" : { } + } + }, + "leftSource" : { + "@type" : "streamSelectV1", + "properties" : { + "queryContext" : "PrependAliasLeft" + }, + "source" : { + "@type" : "streamSourceV1", + "properties" : { + "queryContext" : "KafkaTopic_Left/Source" + }, + "topicName" : "s1", + "formats" : { + "keyFormat" : { + "format" : "KAFKA", + "properties" : { } + }, + "valueFormat" : { + "format" : "PROTOBUF_NOSR", + "properties" : { } + } + }, + "timestampColumn" : { + "column" : "TS", + "format" : null + }, + "sourceSchema" : "`ID` BIGINT KEY, `NAME` STRING, `TS` BIGINT", + "pseudoColumnVersion" : 1 + }, + "keyColumnNames" : [ "S1_ID" ], + "selectedKeys" : null, + "selectExpressions" : [ "NAME AS S1_NAME", "TS AS S1_TS", "ROWTIME AS S1_ROWTIME", "ROWPARTITION AS S1_ROWPARTITION", "ROWOFFSET AS S1_ROWOFFSET", "ID AS S1_ID" ] + }, + "rightSource" : { + "@type" : "streamSelectV1", + "properties" : { + "queryContext" : "PrependAliasRight" + }, + "source" : { + "@type" : "streamSourceV1", + "properties" : { + "queryContext" : "KafkaTopic_Right/Source" + }, + "topicName" : "s2", + "formats" : { + "keyFormat" : { + "format" : "KAFKA", + "properties" : { } + }, + "valueFormat" : { + "format" : "PROTOBUF_NOSR", + "properties" : { } + } + }, + "timestampColumn" : { + "column" : "RTS", + "format" : null + }, + "sourceSchema" : "`ID` BIGINT KEY, `F1` STRING, `F2` STRING, `RTS` BIGINT", + "pseudoColumnVersion" : 1 + }, + "keyColumnNames" : [ "S2_ID" ], + "selectedKeys" : null, + "selectExpressions" : [ "F1 AS S2_F1", "F2 AS S2_F2", "RTS AS S2_RTS", "ROWTIME AS S2_ROWTIME", "ROWPARTITION AS S2_ROWPARTITION", "ROWOFFSET AS S2_ROWOFFSET", "ID AS S2_ID" ] + }, + "beforeMillis" : 11.000000000, + "afterMillis" : 11.000000000, + "graceMillis" : null, + "keyColName" : "S1_ID" + }, + "keyColumnNames" : [ "S1_ID" ], + "selectedKeys" : null, + "selectExpressions" : [ "S1_NAME AS NAME", "S1_TS AS TS", "S2_F1 AS F1", "S2_F2 AS F2" ] + }, + "formats" : { + "keyFormat" : { + "format" : "KAFKA", + "properties" : { } + }, + "valueFormat" : { + "format" : "PROTOBUF_NOSR", + "properties" : { } + } + }, + "topicName" : "S1_JOIN_S2", + "timestampColumn" : { + "column" : "TS", + "format" : null + } + }, + "queryId" : "CSAS_S1_JOIN_S2_0", + "runtimeId" : null + } + } ], + "configs" : { + "ksql.extension.dir" : "ext", + "ksql.streams.cache.max.bytes.buffering" : "0", + "metric.reporters" : "", + "ksql.query.status.running.threshold.seconds" : "300", + "ksql.connect.basic.auth.credentials.reload" : "false", + "ksql.output.topic.name.prefix" : "", + "ksql.query.pull.stream.enabled" : "true", + "ksql.query.push.v2.interpreter.enabled" : "true", + "ksql.queryanonymizer.logs_enabled" : "true", + "ksql.variable.substitution.enable" : "true", + "ksql.streams.shutdown.timeout.ms" : "300000", + "ksql.query.pull.max.allowed.offset.lag" : "9223372036854775807", + "ksql.query.pull.max.qps" : "2147483647", + "ksql.access.validator.enable" : "auto", + "ksql.streams.bootstrap.servers" : "localhost:0", + "ksql.query.pull.metrics.enabled" : "true", + "ksql.metrics.extension" : null, + "ksql.query.push.v2.alos.enabled" : "true", + "ksql.query.push.v2.max.hourly.bandwidth.megabytes" : "2147483647", + "ksql.query.pull.range.scan.enabled" : "true", + "ksql.transient.query.cleanup.service.initial.delay.seconds" : "600", + "ksql.hidden.topics" : "_confluent.*,__confluent.*,_schemas,__consumer_offsets,__transaction_state,connect-configs,connect-offsets,connect-status,connect-statuses", + "ksql.lambdas.enabled" : "true", + "ksql.source.table.materialization.enabled" : "true", + "ksql.query.pull.max.hourly.bandwidth.megabytes" : "2147483647", + "ksql.sink.window.change.log.additional.retention" : "1000000", + "ksql.query.persistent.active.limit" : "2147483647", + "ksql.persistence.wrap.single.values" : null, + "ksql.query.transient.max.bytes.buffering.total" : "-1", + "ksql.connect.basic.auth.credentials.source" : "NONE", + "ksql.schema.registry.url" : "schema_registry.url:0", + "ksql.properties.overrides.denylist" : "", + "ksql.service.id" : "some.ksql.service.id", + "ksql.query.push.v2.max.catchup.consumers" : "5", + "ksql.query.push.v2.enabled" : "false", + "ksql.transient.query.cleanup.service.enable" : "true", + "ksql.query.push.v2.metrics.enabled" : "true", + "ksql.rowpartition.rowoffset.enabled" : "true", + "ksql.streams.commit.interval.ms" : "2000", + "ksql.query.pull.table.scan.enabled" : "true", + "ksql.streams.auto.commit.interval.ms" : "0", + "ksql.streams.topology.optimization" : "all", + "ksql.endpoint.migrate.query" : "true", + "ksql.query.push.v2.registry.installed" : "false", + "ksql.streams.num.stream.threads" : "4", + "ksql.metrics.tags.custom" : "", + "ksql.query.push.v2.catchup.consumer.msg.window" : "50", + "ksql.runtime.feature.shared.enabled" : "false", + "ksql.udf.collect.metrics" : "false", + "ksql.new.query.planner.enabled" : "false", + "ksql.connect.request.headers.plugin" : null, + "ksql.security.extension.class" : null, + "ksql.transient.prefix" : "transient_", + "ksql.headers.columns.enabled" : "true", + "ksql.streams.default.deserialization.exception.handler" : "io.confluent.ksql.errors.LogMetricAndContinueExceptionHandler", + "ksql.connect.request.timeout.ms" : "5000", + "ksql.query.pull.enable.standby.reads" : "false", + "ksql.persistence.default.format.key" : "KAFKA", + "ksql.query.persistent.max.bytes.buffering.total" : "-1", + "ksql.query.error.max.queue.size" : "10", + "ksql.query.cleanup.shutdown.timeout.ms" : "30000", + "ksql.internal.topic.min.insync.replicas" : "1", + "ksql.internal.topic.replicas" : "1", + "ksql.insert.into.values.enabled" : "true", + "ksql.queryanonymizer.cluster_namespace" : null, + "ksql.create.or.replace.enabled" : "true", + "ksql.shared.runtimes.count" : "2", + "ksql.cast.strings.preserve.nulls" : "true", + "ksql.authorization.cache.max.entries" : "10000", + "ksql.pull.queries.enable" : "true", + "ksql.transient.query.cleanup.service.period.seconds" : "600", + "ksql.suppress.enabled" : "false", + "ksql.readonly.topics" : "_confluent.*,__confluent.*,_schemas,__consumer_offsets,__transaction_state,connect-configs,connect-offsets,connect-status,connect-statuses", + "ksql.connect.basic.auth.credentials.file" : "", + "ksql.authorization.cache.expiry.time.secs" : "30", + "ksql.query.retry.backoff.initial.ms" : "15000", + "ksql.query.pull.max.concurrent.requests" : "2147483647", + "ksql.streams.auto.offset.reset" : "earliest", + "ksql.connect.url" : "http://localhost:8083", + "ksql.query.push.v2.new.latest.delay.ms" : "5000", + "ksql.query.push.v2.latest.reset.age.ms" : "30000", + "ksql.streams.default.production.exception.handler" : "io.confluent.ksql.errors.ProductionExceptionHandlerUtil$LogAndFailProductionExceptionHandler", + "ksql.query.pull.interpreter.enabled" : "true", + "ksql.query.pull.limit.clause.enabled" : "true", + "ksql.query.pull.router.thread.pool.size" : "50", + "ksql.query.push.v2.continuation.tokens.enabled" : "false", + "ksql.query.retry.backoff.max.ms" : "900000", + "ksql.timestamp.throw.on.invalid" : "false", + "ksql.persistence.default.format.value" : null, + "ksql.udfs.enabled" : "true", + "ksql.udf.enable.security.manager" : "true", + "ksql.connect.worker.config" : "", + "ksql.nested.error.set.null" : "true", + "ksql.query.pull.thread.pool.size" : "50", + "ksql.persistent.prefix" : "query_", + "ksql.metastore.backup.location" : "", + "ksql.error.classifier.regex" : "", + "ksql.suppress.buffer.size.bytes" : "-1" + } +} \ No newline at end of file diff --git a/ksqldb-functional-tests/src/test/resources/historical_plans/join-with-custom-timestamp_-_stream_stream_inner_join_with_ts_extractor_both_sides_-_PROTOBUF_NOSR/7.3.0_1651070054569/spec.json b/ksqldb-functional-tests/src/test/resources/historical_plans/join-with-custom-timestamp_-_stream_stream_inner_join_with_ts_extractor_both_sides_-_PROTOBUF_NOSR/7.3.0_1651070054569/spec.json new file mode 100644 index 000000000000..459664b8cd52 --- /dev/null +++ b/ksqldb-functional-tests/src/test/resources/historical_plans/join-with-custom-timestamp_-_stream_stream_inner_join_with_ts_extractor_both_sides_-_PROTOBUF_NOSR/7.3.0_1651070054569/spec.json @@ -0,0 +1,225 @@ +{ + "version" : "7.3.0", + "timestamp" : 1651070054569, + "path" : "query-validation-tests/join-with-custom-timestamp.json", + "schemas" : { + "CSAS_S1_JOIN_S2_0.KafkaTopic_Left.Source" : { + "schema" : "`ID` BIGINT KEY, `NAME` STRING, `TS` BIGINT", + "keyFormat" : { + "format" : "KAFKA" + }, + "valueFormat" : { + "format" : "PROTOBUF_NOSR" + } + }, + "CSAS_S1_JOIN_S2_0.Join.Right" : { + "schema" : "`S2_ID` BIGINT KEY, `S2_F1` STRING, `S2_F2` STRING, `S2_RTS` BIGINT, `S2_ROWTIME` BIGINT, `S2_ROWPARTITION` INTEGER, `S2_ROWOFFSET` BIGINT, `S2_ID` BIGINT", + "valueFormat" : { + "format" : "PROTOBUF_NOSR" + } + }, + "CSAS_S1_JOIN_S2_0.Join.Left" : { + "schema" : "`S1_ID` BIGINT KEY, `S1_NAME` STRING, `S1_TS` BIGINT, `S1_ROWTIME` BIGINT, `S1_ROWPARTITION` INTEGER, `S1_ROWOFFSET` BIGINT, `S1_ID` BIGINT", + "valueFormat" : { + "format" : "PROTOBUF_NOSR" + } + }, + "CSAS_S1_JOIN_S2_0.S1_JOIN_S2" : { + "schema" : "`S1_ID` BIGINT KEY, `NAME` STRING, `TS` BIGINT, `F1` STRING, `F2` STRING", + "keyFormat" : { + "format" : "KAFKA" + }, + "valueFormat" : { + "format" : "PROTOBUF_NOSR" + } + }, + "CSAS_S1_JOIN_S2_0.Join" : { + "schema" : "`S1_ID` BIGINT KEY, `S1_NAME` STRING, `S1_TS` BIGINT, `S1_ROWTIME` BIGINT, `S1_ROWPARTITION` INTEGER, `S1_ROWOFFSET` BIGINT, `S1_ID` BIGINT", + "keyFormat" : { + "format" : "KAFKA" + } + }, + "CSAS_S1_JOIN_S2_0.KafkaTopic_Right.Source" : { + "schema" : "`ID` BIGINT KEY, `F1` STRING, `F2` STRING, `RTS` BIGINT", + "keyFormat" : { + "format" : "KAFKA" + }, + "valueFormat" : { + "format" : "PROTOBUF_NOSR" + } + } + }, + "testCase" : { + "name" : "stream stream inner join with ts extractor both sides - PROTOBUF_NOSR", + "inputs" : [ { + "topic" : "s1", + "key" : 0, + "value" : { + "NAME" : "zero", + "TS" : 0 + }, + "timestamp" : 0 + }, { + "topic" : "s2", + "key" : 0, + "value" : { + "F1" : "blah", + "F2" : "foo", + "RTS" : 10000 + }, + "timestamp" : 0 + }, { + "topic" : "s2", + "key" : 10, + "value" : { + "F1" : "foo", + "F2" : "bar", + "RTS" : 13000 + }, + "timestamp" : 0 + }, { + "topic" : "s1", + "key" : 10, + "value" : { + "NAME" : "100", + "TS" : 11000 + }, + "timestamp" : 0 + }, { + "topic" : "s1", + "key" : 0, + "value" : { + "NAME" : "jan", + "TS" : 8000 + }, + "timestamp" : 0 + } ], + "outputs" : [ { + "topic" : "S1_JOIN_S2", + "key" : 0, + "value" : { + "NAME" : "zero", + "TS" : 0, + "F1" : "blah", + "F2" : "foo" + }, + "timestamp" : 0 + }, { + "topic" : "S1_JOIN_S2", + "key" : 10, + "value" : { + "NAME" : "100", + "TS" : 11000, + "F1" : "foo", + "F2" : "bar" + }, + "timestamp" : 11000 + }, { + "topic" : "S1_JOIN_S2", + "key" : 0, + "value" : { + "NAME" : "jan", + "TS" : 8000, + "F1" : "blah", + "F2" : "foo" + }, + "timestamp" : 8000 + } ], + "topics" : [ { + "name" : "S1_JOIN_S2", + "replicas" : 1, + "numPartitions" : 4 + }, { + "name" : "s1", + "replicas" : 1, + "numPartitions" : 4 + }, { + "name" : "s2", + "replicas" : 1, + "numPartitions" : 4 + } ], + "statements" : [ "CREATE STREAM S1 (ID BIGINT KEY, NAME varchar, TS bigint) WITH (timestamp='TS', kafka_topic='s1', value_format='PROTOBUF_NOSR');", "CREATE STREAM S2 (ID BIGINT KEY, F1 varchar, F2 varchar, RTS bigint) WITH (timestamp='RTS', kafka_topic='s2', value_format='PROTOBUF_NOSR');", "CREATE STREAM S1_JOIN_S2 WITH(timestamp='TS') as SELECT S1.ID, S1.name as name, S1.ts as ts, s2.f1, s2.f2 from S1 join S2 WITHIN 11 SECONDS ON s1.id = s2.id;" ], + "post" : { + "sources" : [ { + "name" : "S1", + "type" : "STREAM", + "schema" : "`ID` BIGINT KEY, `NAME` STRING, `TS` BIGINT", + "keyFormat" : { + "format" : "KAFKA" + }, + "valueFormat" : "PROTOBUF_NOSR", + "keyFeatures" : [ ], + "valueFeatures" : [ ], + "isSource" : false + }, { + "name" : "S1_JOIN_S2", + "type" : "STREAM", + "schema" : "`S1_ID` BIGINT KEY, `NAME` STRING, `TS` BIGINT, `F1` STRING, `F2` STRING", + "keyFormat" : { + "format" : "KAFKA" + }, + "valueFormat" : "PROTOBUF_NOSR", + "keyFeatures" : [ ], + "valueFeatures" : [ ], + "isSource" : false + }, { + "name" : "S2", + "type" : "STREAM", + "schema" : "`ID` BIGINT KEY, `F1` STRING, `F2` STRING, `RTS` BIGINT", + "keyFormat" : { + "format" : "KAFKA" + }, + "valueFormat" : "PROTOBUF_NOSR", + "keyFeatures" : [ ], + "valueFeatures" : [ ], + "isSource" : false + } ], + "topics" : { + "topics" : [ { + "name" : "_confluent-ksql-some.ksql.service.idquery_CSAS_S1_JOIN_S2_0-KSTREAM-JOINOTHER-0000000009-store-changelog", + "keyFormat" : { + "format" : "KAFKA" + }, + "valueFormat" : { + "format" : "PROTOBUF_NOSR" + } + }, { + "name" : "s2", + "keyFormat" : { + "format" : "KAFKA" + }, + "valueFormat" : { + "format" : "PROTOBUF_NOSR" + }, + "partitions" : 4 + }, { + "name" : "s1", + "keyFormat" : { + "format" : "KAFKA" + }, + "valueFormat" : { + "format" : "PROTOBUF_NOSR" + }, + "partitions" : 4 + }, { + "name" : "S1_JOIN_S2", + "keyFormat" : { + "format" : "KAFKA" + }, + "valueFormat" : { + "format" : "PROTOBUF_NOSR" + }, + "partitions" : 4 + }, { + "name" : "_confluent-ksql-some.ksql.service.idquery_CSAS_S1_JOIN_S2_0-KSTREAM-JOINTHIS-0000000008-store-changelog", + "keyFormat" : { + "format" : "KAFKA" + }, + "valueFormat" : { + "format" : "PROTOBUF_NOSR" + } + } ] + } + } + } +} \ No newline at end of file diff --git a/ksqldb-functional-tests/src/test/resources/historical_plans/join-with-custom-timestamp_-_stream_stream_inner_join_with_ts_extractor_both_sides_-_PROTOBUF_NOSR/7.3.0_1651070054569/topology b/ksqldb-functional-tests/src/test/resources/historical_plans/join-with-custom-timestamp_-_stream_stream_inner_join_with_ts_extractor_both_sides_-_PROTOBUF_NOSR/7.3.0_1651070054569/topology new file mode 100644 index 000000000000..f74bc03af76a --- /dev/null +++ b/ksqldb-functional-tests/src/test/resources/historical_plans/join-with-custom-timestamp_-_stream_stream_inner_join_with_ts_extractor_both_sides_-_PROTOBUF_NOSR/7.3.0_1651070054569/topology @@ -0,0 +1,42 @@ +Topologies: + Sub-topology: 0 + Source: KSTREAM-SOURCE-0000000000 (topics: [s1]) + --> KSTREAM-TRANSFORMVALUES-0000000001 + Source: KSTREAM-SOURCE-0000000003 (topics: [s2]) + --> KSTREAM-TRANSFORMVALUES-0000000004 + Processor: KSTREAM-TRANSFORMVALUES-0000000001 (stores: []) + --> PrependAliasLeft + <-- KSTREAM-SOURCE-0000000000 + Processor: KSTREAM-TRANSFORMVALUES-0000000004 (stores: []) + --> PrependAliasRight + <-- KSTREAM-SOURCE-0000000003 + Processor: PrependAliasLeft (stores: []) + --> Join-this-windowed + <-- KSTREAM-TRANSFORMVALUES-0000000001 + Processor: PrependAliasRight (stores: []) + --> Join-other-windowed + <-- KSTREAM-TRANSFORMVALUES-0000000004 + Processor: Join-other-windowed (stores: [KSTREAM-JOINOTHER-0000000009-store]) + --> Join-other-join + <-- PrependAliasRight + Processor: Join-this-windowed (stores: [KSTREAM-JOINTHIS-0000000008-store]) + --> Join-this-join + <-- PrependAliasLeft + Processor: Join-other-join (stores: [KSTREAM-JOINTHIS-0000000008-store]) + --> Join-merge + <-- Join-other-windowed + Processor: Join-this-join (stores: [KSTREAM-JOINOTHER-0000000009-store]) + --> Join-merge + <-- Join-this-windowed + Processor: Join-merge (stores: []) + --> Project + <-- Join-this-join, Join-other-join + Processor: Project (stores: []) + --> ApplyTimestampTransform-S1_JOIN_S2 + <-- Join-merge + Processor: ApplyTimestampTransform-S1_JOIN_S2 (stores: []) + --> KSTREAM-SINK-0000000012 + <-- Project + Sink: KSTREAM-SINK-0000000012 (topic: S1_JOIN_S2) + <-- ApplyTimestampTransform-S1_JOIN_S2 + diff --git a/ksqldb-functional-tests/src/test/resources/historical_plans/join-with-custom-timestamp_-_stream_table_join_with_ts_extractor_both_sides_-_PROTOBUF_NOSR/7.3.0_1651070057213/plan.json b/ksqldb-functional-tests/src/test/resources/historical_plans/join-with-custom-timestamp_-_stream_table_join_with_ts_extractor_both_sides_-_PROTOBUF_NOSR/7.3.0_1651070057213/plan.json new file mode 100644 index 000000000000..189720301335 --- /dev/null +++ b/ksqldb-functional-tests/src/test/resources/historical_plans/join-with-custom-timestamp_-_stream_table_join_with_ts_extractor_both_sides_-_PROTOBUF_NOSR/7.3.0_1651070057213/plan.json @@ -0,0 +1,320 @@ +{ + "plan" : [ { + "@type" : "ksqlPlanV1", + "statementText" : "CREATE STREAM S1 (ID BIGINT KEY, NAME STRING, TS BIGINT) WITH (KAFKA_TOPIC='s1', KEY_FORMAT='KAFKA', TIMESTAMP='TS', VALUE_FORMAT='PROTOBUF_NOSR');", + "ddlCommand" : { + "@type" : "createStreamV1", + "sourceName" : "S1", + "schema" : "`ID` BIGINT KEY, `NAME` STRING, `TS` BIGINT", + "timestampColumn" : { + "column" : "TS", + "format" : null + }, + "topicName" : "s1", + "formats" : { + "keyFormat" : { + "format" : "KAFKA", + "properties" : { } + }, + "valueFormat" : { + "format" : "PROTOBUF_NOSR", + "properties" : { } + } + }, + "windowInfo" : null, + "orReplace" : false, + "isSource" : false + }, + "queryPlan" : null + }, { + "@type" : "ksqlPlanV1", + "statementText" : "CREATE TABLE T1 (ID BIGINT PRIMARY KEY, F1 STRING, F2 STRING, RTS BIGINT) WITH (KAFKA_TOPIC='t1', KEY_FORMAT='KAFKA', TIMESTAMP='RTS', VALUE_FORMAT='PROTOBUF_NOSR');", + "ddlCommand" : { + "@type" : "createTableV1", + "sourceName" : "T1", + "schema" : "`ID` BIGINT KEY, `F1` STRING, `F2` STRING, `RTS` BIGINT", + "timestampColumn" : { + "column" : "RTS", + "format" : null + }, + "topicName" : "t1", + "formats" : { + "keyFormat" : { + "format" : "KAFKA", + "properties" : { } + }, + "valueFormat" : { + "format" : "PROTOBUF_NOSR", + "properties" : { } + } + }, + "windowInfo" : null, + "orReplace" : false, + "isSource" : false + }, + "queryPlan" : null + }, { + "@type" : "ksqlPlanV1", + "statementText" : "CREATE STREAM S1_JOIN_T1 WITH (TIMESTAMP='TS') AS SELECT\n S1.ID S1_ID,\n S1.NAME NAME,\n S1.TS TS,\n T1.F1 F1,\n T1.F2 F2\nFROM S1 S1\nINNER JOIN T1 T1 ON ((S1.ID = T1.ID))\nEMIT CHANGES", + "ddlCommand" : { + "@type" : "createStreamV1", + "sourceName" : "S1_JOIN_T1", + "schema" : "`S1_ID` BIGINT KEY, `NAME` STRING, `TS` BIGINT, `F1` STRING, `F2` STRING", + "timestampColumn" : { + "column" : "TS", + "format" : null + }, + "topicName" : "S1_JOIN_T1", + "formats" : { + "keyFormat" : { + "format" : "KAFKA", + "properties" : { } + }, + "valueFormat" : { + "format" : "PROTOBUF_NOSR", + "properties" : { } + } + }, + "windowInfo" : null, + "orReplace" : false, + "isSource" : false + }, + "queryPlan" : { + "sources" : [ "S1", "T1" ], + "sink" : "S1_JOIN_T1", + "physicalPlan" : { + "@type" : "streamSinkV1", + "properties" : { + "queryContext" : "S1_JOIN_T1" + }, + "source" : { + "@type" : "streamSelectV1", + "properties" : { + "queryContext" : "Project" + }, + "source" : { + "@type" : "streamTableJoinV1", + "properties" : { + "queryContext" : "Join" + }, + "joinType" : "INNER", + "internalFormats" : { + "keyFormat" : { + "format" : "KAFKA", + "properties" : { } + }, + "valueFormat" : { + "format" : "PROTOBUF_NOSR", + "properties" : { } + } + }, + "leftSource" : { + "@type" : "streamSelectV1", + "properties" : { + "queryContext" : "PrependAliasLeft" + }, + "source" : { + "@type" : "streamSourceV1", + "properties" : { + "queryContext" : "KafkaTopic_Left/Source" + }, + "topicName" : "s1", + "formats" : { + "keyFormat" : { + "format" : "KAFKA", + "properties" : { } + }, + "valueFormat" : { + "format" : "PROTOBUF_NOSR", + "properties" : { } + } + }, + "timestampColumn" : { + "column" : "TS", + "format" : null + }, + "sourceSchema" : "`ID` BIGINT KEY, `NAME` STRING, `TS` BIGINT", + "pseudoColumnVersion" : 1 + }, + "keyColumnNames" : [ "S1_ID" ], + "selectedKeys" : null, + "selectExpressions" : [ "NAME AS S1_NAME", "TS AS S1_TS", "ROWTIME AS S1_ROWTIME", "ROWPARTITION AS S1_ROWPARTITION", "ROWOFFSET AS S1_ROWOFFSET", "ID AS S1_ID" ] + }, + "rightSource" : { + "@type" : "tableSelectV1", + "properties" : { + "queryContext" : "PrependAliasRight" + }, + "source" : { + "@type" : "tableSourceV2", + "properties" : { + "queryContext" : "KafkaTopic_Right/Source" + }, + "topicName" : "t1", + "formats" : { + "keyFormat" : { + "format" : "KAFKA", + "properties" : { } + }, + "valueFormat" : { + "format" : "PROTOBUF_NOSR", + "properties" : { } + } + }, + "timestampColumn" : { + "column" : "RTS", + "format" : null + }, + "sourceSchema" : "`ID` BIGINT KEY, `F1` STRING, `F2` STRING, `RTS` BIGINT", + "pseudoColumnVersion" : 1, + "stateStoreFormats" : { + "keyFormat" : { + "format" : "KAFKA", + "properties" : { } + }, + "valueFormat" : { + "format" : "PROTOBUF_NOSR", + "properties" : { } + } + } + }, + "keyColumnNames" : [ "T1_ID" ], + "selectExpressions" : [ "F1 AS T1_F1", "F2 AS T1_F2", "RTS AS T1_RTS", "ROWTIME AS T1_ROWTIME", "ROWPARTITION AS T1_ROWPARTITION", "ROWOFFSET AS T1_ROWOFFSET", "ID AS T1_ID" ], + "internalFormats" : { + "keyFormat" : { + "format" : "KAFKA", + "properties" : { } + }, + "valueFormat" : { + "format" : "PROTOBUF_NOSR", + "properties" : { } + } + } + }, + "keyColName" : "S1_ID" + }, + "keyColumnNames" : [ "S1_ID" ], + "selectedKeys" : null, + "selectExpressions" : [ "S1_NAME AS NAME", "S1_TS AS TS", "T1_F1 AS F1", "T1_F2 AS F2" ] + }, + "formats" : { + "keyFormat" : { + "format" : "KAFKA", + "properties" : { } + }, + "valueFormat" : { + "format" : "PROTOBUF_NOSR", + "properties" : { } + } + }, + "topicName" : "S1_JOIN_T1", + "timestampColumn" : { + "column" : "TS", + "format" : null + } + }, + "queryId" : "CSAS_S1_JOIN_T1_0", + "runtimeId" : null + } + } ], + "configs" : { + "ksql.extension.dir" : "ext", + "ksql.streams.cache.max.bytes.buffering" : "0", + "metric.reporters" : "", + "ksql.query.status.running.threshold.seconds" : "300", + "ksql.connect.basic.auth.credentials.reload" : "false", + "ksql.output.topic.name.prefix" : "", + "ksql.query.pull.stream.enabled" : "true", + "ksql.query.push.v2.interpreter.enabled" : "true", + "ksql.queryanonymizer.logs_enabled" : "true", + "ksql.variable.substitution.enable" : "true", + "ksql.streams.shutdown.timeout.ms" : "300000", + "ksql.query.pull.max.allowed.offset.lag" : "9223372036854775807", + "ksql.query.pull.max.qps" : "2147483647", + "ksql.access.validator.enable" : "auto", + "ksql.streams.bootstrap.servers" : "localhost:0", + "ksql.query.pull.metrics.enabled" : "true", + "ksql.metrics.extension" : null, + "ksql.query.push.v2.alos.enabled" : "true", + "ksql.query.push.v2.max.hourly.bandwidth.megabytes" : "2147483647", + "ksql.query.pull.range.scan.enabled" : "true", + "ksql.transient.query.cleanup.service.initial.delay.seconds" : "600", + "ksql.hidden.topics" : "_confluent.*,__confluent.*,_schemas,__consumer_offsets,__transaction_state,connect-configs,connect-offsets,connect-status,connect-statuses", + "ksql.lambdas.enabled" : "true", + "ksql.source.table.materialization.enabled" : "true", + "ksql.query.pull.max.hourly.bandwidth.megabytes" : "2147483647", + "ksql.sink.window.change.log.additional.retention" : "1000000", + "ksql.query.persistent.active.limit" : "2147483647", + "ksql.persistence.wrap.single.values" : null, + "ksql.query.transient.max.bytes.buffering.total" : "-1", + "ksql.connect.basic.auth.credentials.source" : "NONE", + "ksql.schema.registry.url" : "schema_registry.url:0", + "ksql.properties.overrides.denylist" : "", + "ksql.service.id" : "some.ksql.service.id", + "ksql.query.push.v2.max.catchup.consumers" : "5", + "ksql.query.push.v2.enabled" : "false", + "ksql.transient.query.cleanup.service.enable" : "true", + "ksql.query.push.v2.metrics.enabled" : "true", + "ksql.rowpartition.rowoffset.enabled" : "true", + "ksql.streams.commit.interval.ms" : "2000", + "ksql.query.pull.table.scan.enabled" : "true", + "ksql.streams.auto.commit.interval.ms" : "0", + "ksql.streams.topology.optimization" : "all", + "ksql.endpoint.migrate.query" : "true", + "ksql.query.push.v2.registry.installed" : "false", + "ksql.streams.num.stream.threads" : "4", + "ksql.metrics.tags.custom" : "", + "ksql.query.push.v2.catchup.consumer.msg.window" : "50", + "ksql.runtime.feature.shared.enabled" : "false", + "ksql.udf.collect.metrics" : "false", + "ksql.new.query.planner.enabled" : "false", + "ksql.connect.request.headers.plugin" : null, + "ksql.security.extension.class" : null, + "ksql.transient.prefix" : "transient_", + "ksql.headers.columns.enabled" : "true", + "ksql.streams.default.deserialization.exception.handler" : "io.confluent.ksql.errors.LogMetricAndContinueExceptionHandler", + "ksql.connect.request.timeout.ms" : "5000", + "ksql.query.pull.enable.standby.reads" : "false", + "ksql.persistence.default.format.key" : "KAFKA", + "ksql.query.persistent.max.bytes.buffering.total" : "-1", + "ksql.query.error.max.queue.size" : "10", + "ksql.query.cleanup.shutdown.timeout.ms" : "30000", + "ksql.internal.topic.min.insync.replicas" : "1", + "ksql.internal.topic.replicas" : "1", + "ksql.insert.into.values.enabled" : "true", + "ksql.queryanonymizer.cluster_namespace" : null, + "ksql.create.or.replace.enabled" : "true", + "ksql.shared.runtimes.count" : "2", + "ksql.cast.strings.preserve.nulls" : "true", + "ksql.authorization.cache.max.entries" : "10000", + "ksql.pull.queries.enable" : "true", + "ksql.transient.query.cleanup.service.period.seconds" : "600", + "ksql.suppress.enabled" : "false", + "ksql.readonly.topics" : "_confluent.*,__confluent.*,_schemas,__consumer_offsets,__transaction_state,connect-configs,connect-offsets,connect-status,connect-statuses", + "ksql.connect.basic.auth.credentials.file" : "", + "ksql.authorization.cache.expiry.time.secs" : "30", + "ksql.query.retry.backoff.initial.ms" : "15000", + "ksql.query.pull.max.concurrent.requests" : "2147483647", + "ksql.streams.auto.offset.reset" : "earliest", + "ksql.connect.url" : "http://localhost:8083", + "ksql.query.push.v2.new.latest.delay.ms" : "5000", + "ksql.query.push.v2.latest.reset.age.ms" : "30000", + "ksql.streams.default.production.exception.handler" : "io.confluent.ksql.errors.ProductionExceptionHandlerUtil$LogAndFailProductionExceptionHandler", + "ksql.query.pull.interpreter.enabled" : "true", + "ksql.query.pull.limit.clause.enabled" : "true", + "ksql.query.pull.router.thread.pool.size" : "50", + "ksql.query.push.v2.continuation.tokens.enabled" : "false", + "ksql.query.retry.backoff.max.ms" : "900000", + "ksql.timestamp.throw.on.invalid" : "false", + "ksql.persistence.default.format.value" : null, + "ksql.udfs.enabled" : "true", + "ksql.udf.enable.security.manager" : "true", + "ksql.connect.worker.config" : "", + "ksql.nested.error.set.null" : "true", + "ksql.query.pull.thread.pool.size" : "50", + "ksql.persistent.prefix" : "query_", + "ksql.metastore.backup.location" : "", + "ksql.error.classifier.regex" : "", + "ksql.suppress.buffer.size.bytes" : "-1" + } +} \ No newline at end of file diff --git a/ksqldb-functional-tests/src/test/resources/historical_plans/join-with-custom-timestamp_-_stream_table_join_with_ts_extractor_both_sides_-_PROTOBUF_NOSR/7.3.0_1651070057213/spec.json b/ksqldb-functional-tests/src/test/resources/historical_plans/join-with-custom-timestamp_-_stream_table_join_with_ts_extractor_both_sides_-_PROTOBUF_NOSR/7.3.0_1651070057213/spec.json new file mode 100644 index 000000000000..87e9ffc830d3 --- /dev/null +++ b/ksqldb-functional-tests/src/test/resources/historical_plans/join-with-custom-timestamp_-_stream_table_join_with_ts_extractor_both_sides_-_PROTOBUF_NOSR/7.3.0_1651070057213/spec.json @@ -0,0 +1,229 @@ +{ + "version" : "7.3.0", + "timestamp" : 1651070057213, + "path" : "query-validation-tests/join-with-custom-timestamp.json", + "schemas" : { + "CSAS_S1_JOIN_T1_0.KafkaTopic_Right.Source.Materialized" : { + "schema" : "`ID` BIGINT KEY, `F1` STRING, `F2` STRING, `RTS` BIGINT, `ROWPARTITION` INTEGER, `ROWOFFSET` BIGINT", + "keyFormat" : { + "format" : "KAFKA" + }, + "valueFormat" : { + "format" : "PROTOBUF_NOSR" + } + }, + "CSAS_S1_JOIN_T1_0.KafkaTopic_Left.Source" : { + "schema" : "`ID` BIGINT KEY, `NAME` STRING, `TS` BIGINT", + "keyFormat" : { + "format" : "KAFKA" + }, + "valueFormat" : { + "format" : "PROTOBUF_NOSR" + } + }, + "CSAS_S1_JOIN_T1_0.Join" : { + "schema" : "`S1_ID` BIGINT KEY, `S1_NAME` STRING, `S1_TS` BIGINT, `S1_ROWTIME` BIGINT, `S1_ROWPARTITION` INTEGER, `S1_ROWOFFSET` BIGINT, `S1_ID` BIGINT", + "keyFormat" : { + "format" : "KAFKA" + } + }, + "CSAS_S1_JOIN_T1_0.S1_JOIN_T1" : { + "schema" : "`S1_ID` BIGINT KEY, `NAME` STRING, `TS` BIGINT, `F1` STRING, `F2` STRING", + "keyFormat" : { + "format" : "KAFKA" + }, + "valueFormat" : { + "format" : "PROTOBUF_NOSR" + } + }, + "CSAS_S1_JOIN_T1_0.PrependAliasRight" : { + "schema" : "`T1_ID` BIGINT KEY, `T1_F1` STRING, `T1_F2` STRING, `T1_RTS` BIGINT, `T1_ROWTIME` BIGINT, `T1_ROWPARTITION` INTEGER, `T1_ROWOFFSET` BIGINT, `T1_ID` BIGINT", + "keyFormat" : { + "format" : "KAFKA" + }, + "valueFormat" : { + "format" : "PROTOBUF_NOSR" + } + }, + "CSAS_S1_JOIN_T1_0.Join.Left" : { + "schema" : "`S1_ID` BIGINT KEY, `S1_NAME` STRING, `S1_TS` BIGINT, `S1_ROWTIME` BIGINT, `S1_ROWPARTITION` INTEGER, `S1_ROWOFFSET` BIGINT, `S1_ID` BIGINT", + "valueFormat" : { + "format" : "PROTOBUF_NOSR" + } + }, + "CSAS_S1_JOIN_T1_0.KafkaTopic_Right.Source" : { + "schema" : "`ID` BIGINT KEY, `F1` STRING, `F2` STRING, `RTS` BIGINT", + "keyFormat" : { + "format" : "KAFKA" + }, + "valueFormat" : { + "format" : "PROTOBUF_NOSR" + } + } + }, + "testCase" : { + "name" : "stream table join with ts extractor both sides - PROTOBUF_NOSR", + "inputs" : [ { + "topic" : "t1", + "key" : 0, + "value" : { + "F1" : "blah", + "F2" : "foo", + "RTS" : 10000 + }, + "timestamp" : 10000 + }, { + "topic" : "s1", + "key" : 0, + "value" : { + "NAME" : "zero", + "TS" : 0 + }, + "timestamp" : 10000 + }, { + "topic" : "t1", + "key" : 10, + "value" : { + "F1" : "foo", + "F2" : "bar", + "RTS" : 13000 + }, + "timestamp" : 90000 + }, { + "topic" : "s1", + "key" : 10, + "value" : { + "NAME" : "100", + "TS" : 11000 + }, + "timestamp" : 800000 + }, { + "topic" : "s1", + "key" : 0, + "value" : { + "NAME" : "jan", + "TS" : 8000 + }, + "timestamp" : 0 + } ], + "outputs" : [ { + "topic" : "S1_JOIN_T1", + "key" : 0, + "value" : { + "NAME" : "zero", + "TS" : 0, + "F1" : "blah", + "F2" : "foo" + }, + "timestamp" : 0 + }, { + "topic" : "S1_JOIN_T1", + "key" : 10, + "value" : { + "NAME" : "100", + "TS" : 11000, + "F1" : "foo", + "F2" : "bar" + }, + "timestamp" : 11000 + }, { + "topic" : "S1_JOIN_T1", + "key" : 0, + "value" : { + "NAME" : "jan", + "TS" : 8000, + "F1" : "blah", + "F2" : "foo" + }, + "timestamp" : 8000 + } ], + "topics" : [ { + "name" : "t1", + "replicas" : 1, + "numPartitions" : 4 + }, { + "name" : "s1", + "replicas" : 1, + "numPartitions" : 4 + }, { + "name" : "S1_JOIN_T1", + "replicas" : 1, + "numPartitions" : 4 + } ], + "statements" : [ "CREATE STREAM S1 (ID BIGINT KEY, NAME varchar, TS bigint) WITH (timestamp='TS', kafka_topic='s1', value_format='PROTOBUF_NOSR');", "CREATE TABLE T1 (ID BIGINT PRIMARY KEY, F1 varchar, F2 varchar, RTS bigint) WITH (timestamp='RTS', kafka_topic='t1', value_format='PROTOBUF_NOSR');", "CREATE STREAM S1_JOIN_T1 WITH(timestamp='TS') as SELECT S1.ID, S1.name as name, S1.ts as ts, T1.f1, T1.f2 from S1 inner join T1 ON s1.id = t1.id;" ], + "post" : { + "sources" : [ { + "name" : "S1", + "type" : "STREAM", + "schema" : "`ID` BIGINT KEY, `NAME` STRING, `TS` BIGINT", + "keyFormat" : { + "format" : "KAFKA" + }, + "valueFormat" : "PROTOBUF_NOSR", + "keyFeatures" : [ ], + "valueFeatures" : [ ], + "isSource" : false + }, { + "name" : "S1_JOIN_T1", + "type" : "STREAM", + "schema" : "`S1_ID` BIGINT KEY, `NAME` STRING, `TS` BIGINT, `F1` STRING, `F2` STRING", + "keyFormat" : { + "format" : "KAFKA" + }, + "valueFormat" : "PROTOBUF_NOSR", + "keyFeatures" : [ ], + "valueFeatures" : [ ], + "isSource" : false + }, { + "name" : "T1", + "type" : "TABLE", + "schema" : "`ID` BIGINT KEY, `F1` STRING, `F2` STRING, `RTS` BIGINT", + "keyFormat" : { + "format" : "KAFKA" + }, + "valueFormat" : "PROTOBUF_NOSR", + "keyFeatures" : [ ], + "valueFeatures" : [ ], + "isSource" : false + } ], + "topics" : { + "topics" : [ { + "name" : "s1", + "keyFormat" : { + "format" : "KAFKA" + }, + "valueFormat" : { + "format" : "PROTOBUF_NOSR" + }, + "partitions" : 4 + }, { + "name" : "S1_JOIN_T1", + "keyFormat" : { + "format" : "KAFKA" + }, + "valueFormat" : { + "format" : "PROTOBUF_NOSR" + }, + "partitions" : 4 + }, { + "name" : "_confluent-ksql-some.ksql.service.idquery_CSAS_S1_JOIN_T1_0-KafkaTopic_Right-Reduce-changelog", + "keyFormat" : { + "format" : "KAFKA" + }, + "valueFormat" : { + "format" : "PROTOBUF_NOSR" + } + }, { + "name" : "t1", + "keyFormat" : { + "format" : "KAFKA" + }, + "valueFormat" : { + "format" : "PROTOBUF_NOSR" + }, + "partitions" : 4 + } ] + } + } + } +} \ No newline at end of file diff --git a/ksqldb-functional-tests/src/test/resources/historical_plans/join-with-custom-timestamp_-_stream_table_join_with_ts_extractor_both_sides_-_PROTOBUF_NOSR/7.3.0_1651070057213/topology b/ksqldb-functional-tests/src/test/resources/historical_plans/join-with-custom-timestamp_-_stream_table_join_with_ts_extractor_both_sides_-_PROTOBUF_NOSR/7.3.0_1651070057213/topology new file mode 100644 index 000000000000..42320e146e8c --- /dev/null +++ b/ksqldb-functional-tests/src/test/resources/historical_plans/join-with-custom-timestamp_-_stream_table_join_with_ts_extractor_both_sides_-_PROTOBUF_NOSR/7.3.0_1651070057213/topology @@ -0,0 +1,36 @@ +Topologies: + Sub-topology: 0 + Source: KSTREAM-SOURCE-0000000006 (topics: [s1]) + --> KSTREAM-TRANSFORMVALUES-0000000007 + Processor: KSTREAM-TRANSFORMVALUES-0000000007 (stores: []) + --> PrependAliasLeft + <-- KSTREAM-SOURCE-0000000006 + Source: KSTREAM-SOURCE-0000000001 (topics: [t1]) + --> KTABLE-SOURCE-0000000002 + Processor: PrependAliasLeft (stores: []) + --> Join + <-- KSTREAM-TRANSFORMVALUES-0000000007 + Processor: Join (stores: [KafkaTopic_Right-Reduce]) + --> Project + <-- PrependAliasLeft + Processor: KTABLE-SOURCE-0000000002 (stores: []) + --> KTABLE-TRANSFORMVALUES-0000000003 + <-- KSTREAM-SOURCE-0000000001 + Processor: KTABLE-TRANSFORMVALUES-0000000003 (stores: [KafkaTopic_Right-Reduce]) + --> KTABLE-TRANSFORMVALUES-0000000004 + <-- KTABLE-SOURCE-0000000002 + Processor: Project (stores: []) + --> ApplyTimestampTransform-S1_JOIN_T1 + <-- Join + Processor: ApplyTimestampTransform-S1_JOIN_T1 (stores: []) + --> KSTREAM-SINK-0000000011 + <-- Project + Processor: KTABLE-TRANSFORMVALUES-0000000004 (stores: []) + --> PrependAliasRight + <-- KTABLE-TRANSFORMVALUES-0000000003 + Sink: KSTREAM-SINK-0000000011 (topic: S1_JOIN_T1) + <-- ApplyTimestampTransform-S1_JOIN_T1 + Processor: PrependAliasRight (stores: []) + --> none + <-- KTABLE-TRANSFORMVALUES-0000000004 + diff --git a/ksqldb-functional-tests/src/test/resources/historical_plans/join-with-custom-timestamp_-_table_table_inner_join_with_ts_extractor_both_sides_-_PROTOBUF_NOSR/7.3.0_1651070068563/plan.json b/ksqldb-functional-tests/src/test/resources/historical_plans/join-with-custom-timestamp_-_table_table_inner_join_with_ts_extractor_both_sides_-_PROTOBUF_NOSR/7.3.0_1651070068563/plan.json new file mode 100644 index 000000000000..26db675970c4 --- /dev/null +++ b/ksqldb-functional-tests/src/test/resources/historical_plans/join-with-custom-timestamp_-_table_table_inner_join_with_ts_extractor_both_sides_-_PROTOBUF_NOSR/7.3.0_1651070068563/plan.json @@ -0,0 +1,338 @@ +{ + "plan" : [ { + "@type" : "ksqlPlanV1", + "statementText" : "CREATE TABLE S1 (ID BIGINT PRIMARY KEY, NAME STRING, TS BIGINT) WITH (KAFKA_TOPIC='s1', KEY_FORMAT='KAFKA', TIMESTAMP='TS', VALUE_FORMAT='PROTOBUF_NOSR');", + "ddlCommand" : { + "@type" : "createTableV1", + "sourceName" : "S1", + "schema" : "`ID` BIGINT KEY, `NAME` STRING, `TS` BIGINT", + "timestampColumn" : { + "column" : "TS", + "format" : null + }, + "topicName" : "s1", + "formats" : { + "keyFormat" : { + "format" : "KAFKA", + "properties" : { } + }, + "valueFormat" : { + "format" : "PROTOBUF_NOSR", + "properties" : { } + } + }, + "windowInfo" : null, + "orReplace" : false, + "isSource" : false + }, + "queryPlan" : null + }, { + "@type" : "ksqlPlanV1", + "statementText" : "CREATE TABLE S2 (ID BIGINT PRIMARY KEY, F1 STRING, F2 STRING, RTS BIGINT) WITH (KAFKA_TOPIC='s2', KEY_FORMAT='KAFKA', TIMESTAMP='RTS', VALUE_FORMAT='PROTOBUF_NOSR');", + "ddlCommand" : { + "@type" : "createTableV1", + "sourceName" : "S2", + "schema" : "`ID` BIGINT KEY, `F1` STRING, `F2` STRING, `RTS` BIGINT", + "timestampColumn" : { + "column" : "RTS", + "format" : null + }, + "topicName" : "s2", + "formats" : { + "keyFormat" : { + "format" : "KAFKA", + "properties" : { } + }, + "valueFormat" : { + "format" : "PROTOBUF_NOSR", + "properties" : { } + } + }, + "windowInfo" : null, + "orReplace" : false, + "isSource" : false + }, + "queryPlan" : null + }, { + "@type" : "ksqlPlanV1", + "statementText" : "CREATE TABLE S1_JOIN_S2 WITH (TIMESTAMP='TS') AS SELECT\n S1.ID S1_ID,\n S1.NAME NAME,\n S1.TS TS,\n S2.F1 F1,\n S2.F2 F2\nFROM S1 S1\nINNER JOIN S2 S2 ON ((S1.ID = S2.ID))\nEMIT CHANGES", + "ddlCommand" : { + "@type" : "createTableV1", + "sourceName" : "S1_JOIN_S2", + "schema" : "`S1_ID` BIGINT KEY, `NAME` STRING, `TS` BIGINT, `F1` STRING, `F2` STRING", + "timestampColumn" : { + "column" : "TS", + "format" : null + }, + "topicName" : "S1_JOIN_S2", + "formats" : { + "keyFormat" : { + "format" : "KAFKA", + "properties" : { } + }, + "valueFormat" : { + "format" : "PROTOBUF_NOSR", + "properties" : { } + } + }, + "windowInfo" : null, + "orReplace" : false, + "isSource" : false + }, + "queryPlan" : { + "sources" : [ "S1", "S2" ], + "sink" : "S1_JOIN_S2", + "physicalPlan" : { + "@type" : "tableSinkV1", + "properties" : { + "queryContext" : "S1_JOIN_S2" + }, + "source" : { + "@type" : "tableSelectV1", + "properties" : { + "queryContext" : "Project" + }, + "source" : { + "@type" : "tableTableJoinV1", + "properties" : { + "queryContext" : "Join" + }, + "joinType" : "INNER", + "leftSource" : { + "@type" : "tableSelectV1", + "properties" : { + "queryContext" : "PrependAliasLeft" + }, + "source" : { + "@type" : "tableSourceV2", + "properties" : { + "queryContext" : "KafkaTopic_Left/Source" + }, + "topicName" : "s1", + "formats" : { + "keyFormat" : { + "format" : "KAFKA", + "properties" : { } + }, + "valueFormat" : { + "format" : "PROTOBUF_NOSR", + "properties" : { } + } + }, + "timestampColumn" : { + "column" : "TS", + "format" : null + }, + "sourceSchema" : "`ID` BIGINT KEY, `NAME` STRING, `TS` BIGINT", + "pseudoColumnVersion" : 1, + "stateStoreFormats" : { + "keyFormat" : { + "format" : "KAFKA", + "properties" : { } + }, + "valueFormat" : { + "format" : "PROTOBUF_NOSR", + "properties" : { } + } + } + }, + "keyColumnNames" : [ "S1_ID" ], + "selectExpressions" : [ "NAME AS S1_NAME", "TS AS S1_TS", "ROWTIME AS S1_ROWTIME", "ROWPARTITION AS S1_ROWPARTITION", "ROWOFFSET AS S1_ROWOFFSET", "ID AS S1_ID" ], + "internalFormats" : { + "keyFormat" : { + "format" : "KAFKA", + "properties" : { } + }, + "valueFormat" : { + "format" : "PROTOBUF_NOSR", + "properties" : { } + } + } + }, + "rightSource" : { + "@type" : "tableSelectV1", + "properties" : { + "queryContext" : "PrependAliasRight" + }, + "source" : { + "@type" : "tableSourceV2", + "properties" : { + "queryContext" : "KafkaTopic_Right/Source" + }, + "topicName" : "s2", + "formats" : { + "keyFormat" : { + "format" : "KAFKA", + "properties" : { } + }, + "valueFormat" : { + "format" : "PROTOBUF_NOSR", + "properties" : { } + } + }, + "timestampColumn" : { + "column" : "RTS", + "format" : null + }, + "sourceSchema" : "`ID` BIGINT KEY, `F1` STRING, `F2` STRING, `RTS` BIGINT", + "pseudoColumnVersion" : 1, + "stateStoreFormats" : { + "keyFormat" : { + "format" : "KAFKA", + "properties" : { } + }, + "valueFormat" : { + "format" : "PROTOBUF_NOSR", + "properties" : { } + } + } + }, + "keyColumnNames" : [ "S2_ID" ], + "selectExpressions" : [ "F1 AS S2_F1", "F2 AS S2_F2", "RTS AS S2_RTS", "ROWTIME AS S2_ROWTIME", "ROWPARTITION AS S2_ROWPARTITION", "ROWOFFSET AS S2_ROWOFFSET", "ID AS S2_ID" ], + "internalFormats" : { + "keyFormat" : { + "format" : "KAFKA", + "properties" : { } + }, + "valueFormat" : { + "format" : "PROTOBUF_NOSR", + "properties" : { } + } + } + }, + "keyColName" : "S1_ID" + }, + "keyColumnNames" : [ "S1_ID" ], + "selectExpressions" : [ "S1_NAME AS NAME", "S1_TS AS TS", "S2_F1 AS F1", "S2_F2 AS F2" ], + "internalFormats" : { + "keyFormat" : { + "format" : "KAFKA", + "properties" : { } + }, + "valueFormat" : { + "format" : "PROTOBUF_NOSR", + "properties" : { } + } + } + }, + "formats" : { + "keyFormat" : { + "format" : "KAFKA", + "properties" : { } + }, + "valueFormat" : { + "format" : "PROTOBUF_NOSR", + "properties" : { } + } + }, + "topicName" : "S1_JOIN_S2", + "timestampColumn" : { + "column" : "TS", + "format" : null + } + }, + "queryId" : "CTAS_S1_JOIN_S2_0", + "runtimeId" : null + } + } ], + "configs" : { + "ksql.extension.dir" : "ext", + "ksql.streams.cache.max.bytes.buffering" : "0", + "metric.reporters" : "", + "ksql.query.status.running.threshold.seconds" : "300", + "ksql.connect.basic.auth.credentials.reload" : "false", + "ksql.output.topic.name.prefix" : "", + "ksql.query.pull.stream.enabled" : "true", + "ksql.query.push.v2.interpreter.enabled" : "true", + "ksql.queryanonymizer.logs_enabled" : "true", + "ksql.variable.substitution.enable" : "true", + "ksql.streams.shutdown.timeout.ms" : "300000", + "ksql.query.pull.max.allowed.offset.lag" : "9223372036854775807", + "ksql.query.pull.max.qps" : "2147483647", + "ksql.access.validator.enable" : "auto", + "ksql.streams.bootstrap.servers" : "localhost:0", + "ksql.query.pull.metrics.enabled" : "true", + "ksql.metrics.extension" : null, + "ksql.query.push.v2.alos.enabled" : "true", + "ksql.query.push.v2.max.hourly.bandwidth.megabytes" : "2147483647", + "ksql.query.pull.range.scan.enabled" : "true", + "ksql.transient.query.cleanup.service.initial.delay.seconds" : "600", + "ksql.hidden.topics" : "_confluent.*,__confluent.*,_schemas,__consumer_offsets,__transaction_state,connect-configs,connect-offsets,connect-status,connect-statuses", + "ksql.lambdas.enabled" : "true", + "ksql.source.table.materialization.enabled" : "true", + "ksql.query.pull.max.hourly.bandwidth.megabytes" : "2147483647", + "ksql.sink.window.change.log.additional.retention" : "1000000", + "ksql.query.persistent.active.limit" : "2147483647", + "ksql.persistence.wrap.single.values" : null, + "ksql.query.transient.max.bytes.buffering.total" : "-1", + "ksql.connect.basic.auth.credentials.source" : "NONE", + "ksql.schema.registry.url" : "schema_registry.url:0", + "ksql.properties.overrides.denylist" : "", + "ksql.service.id" : "some.ksql.service.id", + "ksql.query.push.v2.max.catchup.consumers" : "5", + "ksql.query.push.v2.enabled" : "false", + "ksql.transient.query.cleanup.service.enable" : "true", + "ksql.query.push.v2.metrics.enabled" : "true", + "ksql.rowpartition.rowoffset.enabled" : "true", + "ksql.streams.commit.interval.ms" : "2000", + "ksql.query.pull.table.scan.enabled" : "true", + "ksql.streams.auto.commit.interval.ms" : "0", + "ksql.streams.topology.optimization" : "all", + "ksql.endpoint.migrate.query" : "true", + "ksql.query.push.v2.registry.installed" : "false", + "ksql.streams.num.stream.threads" : "4", + "ksql.metrics.tags.custom" : "", + "ksql.query.push.v2.catchup.consumer.msg.window" : "50", + "ksql.runtime.feature.shared.enabled" : "false", + "ksql.udf.collect.metrics" : "false", + "ksql.new.query.planner.enabled" : "false", + "ksql.connect.request.headers.plugin" : null, + "ksql.security.extension.class" : null, + "ksql.transient.prefix" : "transient_", + "ksql.headers.columns.enabled" : "true", + "ksql.streams.default.deserialization.exception.handler" : "io.confluent.ksql.errors.LogMetricAndContinueExceptionHandler", + "ksql.connect.request.timeout.ms" : "5000", + "ksql.query.pull.enable.standby.reads" : "false", + "ksql.persistence.default.format.key" : "KAFKA", + "ksql.query.persistent.max.bytes.buffering.total" : "-1", + "ksql.query.error.max.queue.size" : "10", + "ksql.query.cleanup.shutdown.timeout.ms" : "30000", + "ksql.internal.topic.min.insync.replicas" : "1", + "ksql.internal.topic.replicas" : "1", + "ksql.insert.into.values.enabled" : "true", + "ksql.queryanonymizer.cluster_namespace" : null, + "ksql.create.or.replace.enabled" : "true", + "ksql.shared.runtimes.count" : "2", + "ksql.cast.strings.preserve.nulls" : "true", + "ksql.authorization.cache.max.entries" : "10000", + "ksql.pull.queries.enable" : "true", + "ksql.transient.query.cleanup.service.period.seconds" : "600", + "ksql.suppress.enabled" : "false", + "ksql.readonly.topics" : "_confluent.*,__confluent.*,_schemas,__consumer_offsets,__transaction_state,connect-configs,connect-offsets,connect-status,connect-statuses", + "ksql.connect.basic.auth.credentials.file" : "", + "ksql.authorization.cache.expiry.time.secs" : "30", + "ksql.query.retry.backoff.initial.ms" : "15000", + "ksql.query.pull.max.concurrent.requests" : "2147483647", + "ksql.streams.auto.offset.reset" : "earliest", + "ksql.connect.url" : "http://localhost:8083", + "ksql.query.push.v2.new.latest.delay.ms" : "5000", + "ksql.query.push.v2.latest.reset.age.ms" : "30000", + "ksql.streams.default.production.exception.handler" : "io.confluent.ksql.errors.ProductionExceptionHandlerUtil$LogAndFailProductionExceptionHandler", + "ksql.query.pull.interpreter.enabled" : "true", + "ksql.query.pull.limit.clause.enabled" : "true", + "ksql.query.pull.router.thread.pool.size" : "50", + "ksql.query.push.v2.continuation.tokens.enabled" : "false", + "ksql.query.retry.backoff.max.ms" : "900000", + "ksql.timestamp.throw.on.invalid" : "false", + "ksql.persistence.default.format.value" : null, + "ksql.udfs.enabled" : "true", + "ksql.udf.enable.security.manager" : "true", + "ksql.connect.worker.config" : "", + "ksql.nested.error.set.null" : "true", + "ksql.query.pull.thread.pool.size" : "50", + "ksql.persistent.prefix" : "query_", + "ksql.metastore.backup.location" : "", + "ksql.error.classifier.regex" : "", + "ksql.suppress.buffer.size.bytes" : "-1" + } +} \ No newline at end of file diff --git a/ksqldb-functional-tests/src/test/resources/historical_plans/join-with-custom-timestamp_-_table_table_inner_join_with_ts_extractor_both_sides_-_PROTOBUF_NOSR/7.3.0_1651070068563/spec.json b/ksqldb-functional-tests/src/test/resources/historical_plans/join-with-custom-timestamp_-_table_table_inner_join_with_ts_extractor_both_sides_-_PROTOBUF_NOSR/7.3.0_1651070068563/spec.json new file mode 100644 index 000000000000..00e41eff9cf6 --- /dev/null +++ b/ksqldb-functional-tests/src/test/resources/historical_plans/join-with-custom-timestamp_-_table_table_inner_join_with_ts_extractor_both_sides_-_PROTOBUF_NOSR/7.3.0_1651070068563/spec.json @@ -0,0 +1,260 @@ +{ + "version" : "7.3.0", + "timestamp" : 1651070068563, + "path" : "query-validation-tests/join-with-custom-timestamp.json", + "schemas" : { + "CTAS_S1_JOIN_S2_0.KafkaTopic_Left.Source" : { + "schema" : "`ID` BIGINT KEY, `NAME` STRING, `TS` BIGINT", + "keyFormat" : { + "format" : "KAFKA" + }, + "valueFormat" : { + "format" : "PROTOBUF_NOSR" + } + }, + "CTAS_S1_JOIN_S2_0.Project" : { + "schema" : "`S1_ID` BIGINT KEY, `NAME` STRING, `TS` BIGINT, `F1` STRING, `F2` STRING", + "keyFormat" : { + "format" : "KAFKA" + }, + "valueFormat" : { + "format" : "PROTOBUF_NOSR" + } + }, + "CTAS_S1_JOIN_S2_0.PrependAliasLeft" : { + "schema" : "`S1_ID` BIGINT KEY, `S1_NAME` STRING, `S1_TS` BIGINT, `S1_ROWTIME` BIGINT, `S1_ROWPARTITION` INTEGER, `S1_ROWOFFSET` BIGINT, `S1_ID` BIGINT", + "keyFormat" : { + "format" : "KAFKA" + }, + "valueFormat" : { + "format" : "PROTOBUF_NOSR" + } + }, + "CTAS_S1_JOIN_S2_0.PrependAliasRight" : { + "schema" : "`S2_ID` BIGINT KEY, `S2_F1` STRING, `S2_F2` STRING, `S2_RTS` BIGINT, `S2_ROWTIME` BIGINT, `S2_ROWPARTITION` INTEGER, `S2_ROWOFFSET` BIGINT, `S2_ID` BIGINT", + "keyFormat" : { + "format" : "KAFKA" + }, + "valueFormat" : { + "format" : "PROTOBUF_NOSR" + } + }, + "CTAS_S1_JOIN_S2_0.KafkaTopic_Right.Source.Materialized" : { + "schema" : "`ID` BIGINT KEY, `F1` STRING, `F2` STRING, `RTS` BIGINT, `ROWPARTITION` INTEGER, `ROWOFFSET` BIGINT", + "keyFormat" : { + "format" : "KAFKA" + }, + "valueFormat" : { + "format" : "PROTOBUF_NOSR" + } + }, + "CTAS_S1_JOIN_S2_0.KafkaTopic_Left.Source.Materialized" : { + "schema" : "`ID` BIGINT KEY, `NAME` STRING, `TS` BIGINT, `ROWPARTITION` INTEGER, `ROWOFFSET` BIGINT", + "keyFormat" : { + "format" : "KAFKA" + }, + "valueFormat" : { + "format" : "PROTOBUF_NOSR" + } + }, + "CTAS_S1_JOIN_S2_0.KafkaTopic_Right.Source" : { + "schema" : "`ID` BIGINT KEY, `F1` STRING, `F2` STRING, `RTS` BIGINT", + "keyFormat" : { + "format" : "KAFKA" + }, + "valueFormat" : { + "format" : "PROTOBUF_NOSR" + } + }, + "CTAS_S1_JOIN_S2_0.S1_JOIN_S2" : { + "schema" : "`S1_ID` BIGINT KEY, `NAME` STRING, `TS` BIGINT, `F1` STRING, `F2` STRING", + "keyFormat" : { + "format" : "KAFKA" + }, + "valueFormat" : { + "format" : "PROTOBUF_NOSR" + } + } + }, + "testCase" : { + "name" : "table table inner join with ts extractor both sides - PROTOBUF_NOSR", + "inputs" : [ { + "topic" : "s1", + "key" : 0, + "value" : { + "NAME" : "zero", + "TS" : 0 + }, + "timestamp" : 0 + }, { + "topic" : "s2", + "key" : 0, + "value" : { + "F1" : "blah", + "F2" : "foo", + "RTS" : 10000 + }, + "timestamp" : 0 + }, { + "topic" : "s2", + "key" : 10, + "value" : { + "F1" : "foo", + "F2" : "bar", + "RTS" : 13000 + }, + "timestamp" : 0 + }, { + "topic" : "s1", + "key" : 10, + "value" : { + "NAME" : "100", + "TS" : 11000 + }, + "timestamp" : 0 + }, { + "topic" : "s1", + "key" : 0, + "value" : { + "NAME" : "jan", + "TS" : 8000 + }, + "timestamp" : 0 + } ], + "outputs" : [ { + "topic" : "S1_JOIN_S2", + "key" : 0, + "value" : { + "NAME" : "zero", + "TS" : 0, + "F1" : "blah", + "F2" : "foo" + }, + "timestamp" : 0 + }, { + "topic" : "S1_JOIN_S2", + "key" : 10, + "value" : { + "NAME" : "100", + "TS" : 11000, + "F1" : "foo", + "F2" : "bar" + }, + "timestamp" : 11000 + }, { + "topic" : "S1_JOIN_S2", + "key" : 0, + "value" : { + "NAME" : "jan", + "TS" : 8000, + "F1" : "blah", + "F2" : "foo" + }, + "timestamp" : 8000 + } ], + "topics" : [ { + "name" : "S1_JOIN_S2", + "replicas" : 1, + "numPartitions" : 4 + }, { + "name" : "s1", + "replicas" : 1, + "numPartitions" : 4 + }, { + "name" : "s2", + "replicas" : 1, + "numPartitions" : 4 + } ], + "statements" : [ "CREATE TABLE S1 (ID BIGINT PRIMARY KEY, NAME varchar, TS bigint) WITH (timestamp='TS', kafka_topic='s1', value_format='PROTOBUF_NOSR');", "CREATE TABLE S2 (ID BIGINT PRIMARY KEY, F1 varchar, F2 varchar, RTS bigint) WITH (timestamp='RTS', kafka_topic='s2', value_format='PROTOBUF_NOSR');", "CREATE TABLE S1_JOIN_S2 WITH(timestamp='TS') as SELECT S1.ID, S1.name as name, S1.ts as ts, s2.f1, s2.f2 from S1 join S2 ON s1.id = s2.id;" ], + "post" : { + "sources" : [ { + "name" : "S1", + "type" : "TABLE", + "schema" : "`ID` BIGINT KEY, `NAME` STRING, `TS` BIGINT", + "keyFormat" : { + "format" : "KAFKA" + }, + "valueFormat" : "PROTOBUF_NOSR", + "keyFeatures" : [ ], + "valueFeatures" : [ ], + "isSource" : false + }, { + "name" : "S1_JOIN_S2", + "type" : "TABLE", + "schema" : "`S1_ID` BIGINT KEY, `NAME` STRING, `TS` BIGINT, `F1` STRING, `F2` STRING", + "keyFormat" : { + "format" : "KAFKA" + }, + "valueFormat" : "PROTOBUF_NOSR", + "keyFeatures" : [ ], + "valueFeatures" : [ ], + "isSource" : false + }, { + "name" : "S2", + "type" : "TABLE", + "schema" : "`ID` BIGINT KEY, `F1` STRING, `F2` STRING, `RTS` BIGINT", + "keyFormat" : { + "format" : "KAFKA" + }, + "valueFormat" : "PROTOBUF_NOSR", + "keyFeatures" : [ ], + "valueFeatures" : [ ], + "isSource" : false + } ], + "topics" : { + "topics" : [ { + "name" : "s2", + "keyFormat" : { + "format" : "KAFKA" + }, + "valueFormat" : { + "format" : "PROTOBUF_NOSR" + }, + "partitions" : 4 + }, { + "name" : "s1", + "keyFormat" : { + "format" : "KAFKA" + }, + "valueFormat" : { + "format" : "PROTOBUF_NOSR" + }, + "partitions" : 4 + }, { + "name" : "S1_JOIN_S2", + "keyFormat" : { + "format" : "KAFKA" + }, + "valueFormat" : { + "format" : "PROTOBUF_NOSR" + }, + "partitions" : 4 + }, { + "name" : "_confluent-ksql-some.ksql.service.idquery_CTAS_S1_JOIN_S2_0-KafkaTopic_Right-Reduce-changelog", + "keyFormat" : { + "format" : "KAFKA" + }, + "valueFormat" : { + "format" : "PROTOBUF_NOSR" + } + }, { + "name" : "_confluent-ksql-some.ksql.service.idquery_CTAS_S1_JOIN_S2_0-KafkaTopic_Left-Reduce-changelog", + "keyFormat" : { + "format" : "KAFKA" + }, + "valueFormat" : { + "format" : "PROTOBUF_NOSR" + } + }, { + "name" : "_confluent-ksql-some.ksql.service.idquery_CTAS_S1_JOIN_S2_0-Project-Last-changelog", + "keyFormat" : { + "format" : "KAFKA" + }, + "valueFormat" : { + "format" : "PROTOBUF_NOSR" + } + } ] + } + } + } +} \ No newline at end of file diff --git a/ksqldb-functional-tests/src/test/resources/historical_plans/join-with-custom-timestamp_-_table_table_inner_join_with_ts_extractor_both_sides_-_PROTOBUF_NOSR/7.3.0_1651070068563/topology b/ksqldb-functional-tests/src/test/resources/historical_plans/join-with-custom-timestamp_-_table_table_inner_join_with_ts_extractor_both_sides_-_PROTOBUF_NOSR/7.3.0_1651070068563/topology new file mode 100644 index 000000000000..1ce7dcb9d4aa --- /dev/null +++ b/ksqldb-functional-tests/src/test/resources/historical_plans/join-with-custom-timestamp_-_table_table_inner_join_with_ts_extractor_both_sides_-_PROTOBUF_NOSR/7.3.0_1651070068563/topology @@ -0,0 +1,51 @@ +Topologies: + Sub-topology: 0 + Source: KSTREAM-SOURCE-0000000001 (topics: [s1]) + --> KTABLE-SOURCE-0000000002 + Source: KSTREAM-SOURCE-0000000007 (topics: [s2]) + --> KTABLE-SOURCE-0000000008 + Processor: KTABLE-SOURCE-0000000002 (stores: []) + --> KTABLE-TRANSFORMVALUES-0000000003 + <-- KSTREAM-SOURCE-0000000001 + Processor: KTABLE-SOURCE-0000000008 (stores: []) + --> KTABLE-TRANSFORMVALUES-0000000009 + <-- KSTREAM-SOURCE-0000000007 + Processor: KTABLE-TRANSFORMVALUES-0000000003 (stores: [KafkaTopic_Left-Reduce]) + --> KTABLE-TRANSFORMVALUES-0000000004 + <-- KTABLE-SOURCE-0000000002 + Processor: KTABLE-TRANSFORMVALUES-0000000009 (stores: [KafkaTopic_Right-Reduce]) + --> KTABLE-TRANSFORMVALUES-0000000010 + <-- KTABLE-SOURCE-0000000008 + Processor: KTABLE-TRANSFORMVALUES-0000000004 (stores: []) + --> PrependAliasLeft + <-- KTABLE-TRANSFORMVALUES-0000000003 + Processor: KTABLE-TRANSFORMVALUES-0000000010 (stores: []) + --> PrependAliasRight + <-- KTABLE-TRANSFORMVALUES-0000000009 + Processor: PrependAliasLeft (stores: []) + --> KTABLE-JOINTHIS-0000000013 + <-- KTABLE-TRANSFORMVALUES-0000000004 + Processor: PrependAliasRight (stores: []) + --> KTABLE-JOINOTHER-0000000014 + <-- KTABLE-TRANSFORMVALUES-0000000010 + Processor: KTABLE-JOINOTHER-0000000014 (stores: [KafkaTopic_Left-Reduce]) + --> KTABLE-MERGE-0000000012 + <-- PrependAliasRight + Processor: KTABLE-JOINTHIS-0000000013 (stores: [KafkaTopic_Right-Reduce]) + --> KTABLE-MERGE-0000000012 + <-- PrependAliasLeft + Processor: KTABLE-MERGE-0000000012 (stores: []) + --> KTABLE-TRANSFORMVALUES-0000000015 + <-- KTABLE-JOINTHIS-0000000013, KTABLE-JOINOTHER-0000000014 + Processor: KTABLE-TRANSFORMVALUES-0000000015 (stores: [Project-Last]) + --> KTABLE-TOSTREAM-0000000016 + <-- KTABLE-MERGE-0000000012 + Processor: KTABLE-TOSTREAM-0000000016 (stores: []) + --> ApplyTimestampTransform-S1_JOIN_S2 + <-- KTABLE-TRANSFORMVALUES-0000000015 + Processor: ApplyTimestampTransform-S1_JOIN_S2 (stores: []) + --> KSTREAM-SINK-0000000017 + <-- KTABLE-TOSTREAM-0000000016 + Sink: KSTREAM-SINK-0000000017 (topic: S1_JOIN_S2) + <-- ApplyTimestampTransform-S1_JOIN_S2 + diff --git a/ksqldb-functional-tests/src/test/resources/historical_plans/joins_-_on_BIGINT_column_-_KAFKA_-_PROTOBUF_NOSR/7.3.0_1651071425414/plan.json b/ksqldb-functional-tests/src/test/resources/historical_plans/joins_-_on_BIGINT_column_-_KAFKA_-_PROTOBUF_NOSR/7.3.0_1651071425414/plan.json new file mode 100644 index 000000000000..3cfe4ab600c5 --- /dev/null +++ b/ksqldb-functional-tests/src/test/resources/historical_plans/joins_-_on_BIGINT_column_-_KAFKA_-_PROTOBUF_NOSR/7.3.0_1651071425414/plan.json @@ -0,0 +1,310 @@ +{ + "plan" : [ { + "@type" : "ksqlPlanV1", + "statementText" : "CREATE STREAM L (ID STRING KEY, L0 BIGINT, L1 INTEGER) WITH (KAFKA_TOPIC='left_topic', KEY_FORMAT='KAFKA', VALUE_FORMAT='PROTOBUF_NOSR');", + "ddlCommand" : { + "@type" : "createStreamV1", + "sourceName" : "L", + "schema" : "`ID` STRING KEY, `L0` BIGINT, `L1` INTEGER", + "timestampColumn" : null, + "topicName" : "left_topic", + "formats" : { + "keyFormat" : { + "format" : "KAFKA", + "properties" : { } + }, + "valueFormat" : { + "format" : "PROTOBUF_NOSR", + "properties" : { } + } + }, + "windowInfo" : null, + "orReplace" : false, + "isSource" : false + }, + "queryPlan" : null + }, { + "@type" : "ksqlPlanV1", + "statementText" : "CREATE STREAM R (ID STRING KEY, R0 BIGINT, R1 INTEGER) WITH (KAFKA_TOPIC='right_topic', KEY_FORMAT='KAFKA', VALUE_FORMAT='PROTOBUF_NOSR');", + "ddlCommand" : { + "@type" : "createStreamV1", + "sourceName" : "R", + "schema" : "`ID` STRING KEY, `R0` BIGINT, `R1` INTEGER", + "timestampColumn" : null, + "topicName" : "right_topic", + "formats" : { + "keyFormat" : { + "format" : "KAFKA", + "properties" : { } + }, + "valueFormat" : { + "format" : "PROTOBUF_NOSR", + "properties" : { } + } + }, + "windowInfo" : null, + "orReplace" : false, + "isSource" : false + }, + "queryPlan" : null + }, { + "@type" : "ksqlPlanV1", + "statementText" : "CREATE STREAM OUTPUT AS SELECT\n L.L0 L0,\n L.ID L_ID,\n L.L1 L1,\n R.R1 R1\nFROM L L\nINNER JOIN R R WITHIN 11 SECONDS ON ((L.L0 = R.R0))\nEMIT CHANGES", + "ddlCommand" : { + "@type" : "createStreamV1", + "sourceName" : "OUTPUT", + "schema" : "`L0` BIGINT KEY, `L_ID` STRING, `L1` INTEGER, `R1` INTEGER", + "timestampColumn" : null, + "topicName" : "OUTPUT", + "formats" : { + "keyFormat" : { + "format" : "KAFKA", + "properties" : { } + }, + "valueFormat" : { + "format" : "PROTOBUF_NOSR", + "properties" : { } + } + }, + "windowInfo" : null, + "orReplace" : false, + "isSource" : false + }, + "queryPlan" : { + "sources" : [ "L", "R" ], + "sink" : "OUTPUT", + "physicalPlan" : { + "@type" : "streamSinkV1", + "properties" : { + "queryContext" : "OUTPUT" + }, + "source" : { + "@type" : "streamSelectV1", + "properties" : { + "queryContext" : "Project" + }, + "source" : { + "@type" : "streamStreamJoinV1", + "properties" : { + "queryContext" : "Join" + }, + "joinType" : "INNER", + "leftInternalFormats" : { + "keyFormat" : { + "format" : "KAFKA", + "properties" : { } + }, + "valueFormat" : { + "format" : "PROTOBUF_NOSR", + "properties" : { } + } + }, + "rightInternalFormats" : { + "keyFormat" : { + "format" : "KAFKA", + "properties" : { } + }, + "valueFormat" : { + "format" : "PROTOBUF_NOSR", + "properties" : { } + } + }, + "leftSource" : { + "@type" : "streamSelectV1", + "properties" : { + "queryContext" : "PrependAliasLeft" + }, + "source" : { + "@type" : "streamSelectKeyV2", + "properties" : { + "queryContext" : "LeftSourceKeyed" + }, + "source" : { + "@type" : "streamSourceV1", + "properties" : { + "queryContext" : "KafkaTopic_Left/Source" + }, + "topicName" : "left_topic", + "formats" : { + "keyFormat" : { + "format" : "KAFKA", + "properties" : { } + }, + "valueFormat" : { + "format" : "PROTOBUF_NOSR", + "properties" : { } + } + }, + "timestampColumn" : null, + "sourceSchema" : "`ID` STRING KEY, `L0` BIGINT, `L1` INTEGER", + "pseudoColumnVersion" : 1 + }, + "keyExpression" : [ "L0" ] + }, + "keyColumnNames" : [ "L_L0" ], + "selectedKeys" : null, + "selectExpressions" : [ "L0 AS L_L0", "L1 AS L_L1", "ROWTIME AS L_ROWTIME", "ROWPARTITION AS L_ROWPARTITION", "ROWOFFSET AS L_ROWOFFSET", "ID AS L_ID" ] + }, + "rightSource" : { + "@type" : "streamSelectV1", + "properties" : { + "queryContext" : "PrependAliasRight" + }, + "source" : { + "@type" : "streamSelectKeyV2", + "properties" : { + "queryContext" : "RightSourceKeyed" + }, + "source" : { + "@type" : "streamSourceV1", + "properties" : { + "queryContext" : "KafkaTopic_Right/Source" + }, + "topicName" : "right_topic", + "formats" : { + "keyFormat" : { + "format" : "KAFKA", + "properties" : { } + }, + "valueFormat" : { + "format" : "PROTOBUF_NOSR", + "properties" : { } + } + }, + "timestampColumn" : null, + "sourceSchema" : "`ID` STRING KEY, `R0` BIGINT, `R1` INTEGER", + "pseudoColumnVersion" : 1 + }, + "keyExpression" : [ "R0" ] + }, + "keyColumnNames" : [ "R_R0" ], + "selectedKeys" : null, + "selectExpressions" : [ "R0 AS R_R0", "R1 AS R_R1", "ROWTIME AS R_ROWTIME", "ROWPARTITION AS R_ROWPARTITION", "ROWOFFSET AS R_ROWOFFSET", "ID AS R_ID" ] + }, + "beforeMillis" : 11.000000000, + "afterMillis" : 11.000000000, + "graceMillis" : null, + "keyColName" : "L_L0" + }, + "keyColumnNames" : [ "L0" ], + "selectedKeys" : null, + "selectExpressions" : [ "L_ID AS L_ID", "L_L1 AS L1", "R_R1 AS R1" ] + }, + "formats" : { + "keyFormat" : { + "format" : "KAFKA", + "properties" : { } + }, + "valueFormat" : { + "format" : "PROTOBUF_NOSR", + "properties" : { } + } + }, + "topicName" : "OUTPUT", + "timestampColumn" : null + }, + "queryId" : "CSAS_OUTPUT_0", + "runtimeId" : null + } + } ], + "configs" : { + "ksql.extension.dir" : "ext", + "ksql.streams.cache.max.bytes.buffering" : "0", + "metric.reporters" : "", + "ksql.query.status.running.threshold.seconds" : "300", + "ksql.connect.basic.auth.credentials.reload" : "false", + "ksql.output.topic.name.prefix" : "", + "ksql.query.pull.stream.enabled" : "true", + "ksql.query.push.v2.interpreter.enabled" : "true", + "ksql.queryanonymizer.logs_enabled" : "true", + "ksql.variable.substitution.enable" : "true", + "ksql.streams.shutdown.timeout.ms" : "300000", + "ksql.query.pull.max.allowed.offset.lag" : "9223372036854775807", + "ksql.query.pull.max.qps" : "2147483647", + "ksql.access.validator.enable" : "auto", + "ksql.streams.bootstrap.servers" : "localhost:0", + "ksql.query.pull.metrics.enabled" : "true", + "ksql.metrics.extension" : null, + "ksql.query.push.v2.alos.enabled" : "true", + "ksql.query.push.v2.max.hourly.bandwidth.megabytes" : "2147483647", + "ksql.query.pull.range.scan.enabled" : "true", + "ksql.transient.query.cleanup.service.initial.delay.seconds" : "600", + "ksql.hidden.topics" : "_confluent.*,__confluent.*,_schemas,__consumer_offsets,__transaction_state,connect-configs,connect-offsets,connect-status,connect-statuses", + "ksql.lambdas.enabled" : "true", + "ksql.source.table.materialization.enabled" : "true", + "ksql.query.pull.max.hourly.bandwidth.megabytes" : "2147483647", + "ksql.sink.window.change.log.additional.retention" : "1000000", + "ksql.query.persistent.active.limit" : "2147483647", + "ksql.persistence.wrap.single.values" : null, + "ksql.query.transient.max.bytes.buffering.total" : "-1", + "ksql.connect.basic.auth.credentials.source" : "NONE", + "ksql.schema.registry.url" : "schema_registry.url:0", + "ksql.properties.overrides.denylist" : "", + "ksql.service.id" : "some.ksql.service.id", + "ksql.query.push.v2.max.catchup.consumers" : "5", + "ksql.query.push.v2.enabled" : "false", + "ksql.transient.query.cleanup.service.enable" : "true", + "ksql.query.push.v2.metrics.enabled" : "true", + "ksql.rowpartition.rowoffset.enabled" : "true", + "ksql.streams.commit.interval.ms" : "2000", + "ksql.query.pull.table.scan.enabled" : "true", + "ksql.streams.auto.commit.interval.ms" : "0", + "ksql.streams.topology.optimization" : "all", + "ksql.endpoint.migrate.query" : "true", + "ksql.query.push.v2.registry.installed" : "false", + "ksql.streams.num.stream.threads" : "4", + "ksql.metrics.tags.custom" : "", + "ksql.query.push.v2.catchup.consumer.msg.window" : "50", + "ksql.runtime.feature.shared.enabled" : "false", + "ksql.udf.collect.metrics" : "false", + "ksql.new.query.planner.enabled" : "false", + "ksql.connect.request.headers.plugin" : null, + "ksql.security.extension.class" : null, + "ksql.transient.prefix" : "transient_", + "ksql.headers.columns.enabled" : "true", + "ksql.streams.default.deserialization.exception.handler" : "io.confluent.ksql.errors.LogMetricAndContinueExceptionHandler", + "ksql.connect.request.timeout.ms" : "5000", + "ksql.query.pull.enable.standby.reads" : "false", + "ksql.persistence.default.format.key" : "KAFKA", + "ksql.query.persistent.max.bytes.buffering.total" : "-1", + "ksql.query.error.max.queue.size" : "10", + "ksql.query.cleanup.shutdown.timeout.ms" : "30000", + "ksql.internal.topic.min.insync.replicas" : "1", + "ksql.internal.topic.replicas" : "1", + "ksql.insert.into.values.enabled" : "true", + "ksql.queryanonymizer.cluster_namespace" : null, + "ksql.create.or.replace.enabled" : "true", + "ksql.shared.runtimes.count" : "2", + "ksql.cast.strings.preserve.nulls" : "true", + "ksql.authorization.cache.max.entries" : "10000", + "ksql.pull.queries.enable" : "true", + "ksql.transient.query.cleanup.service.period.seconds" : "600", + "ksql.suppress.enabled" : "false", + "ksql.readonly.topics" : "_confluent.*,__confluent.*,_schemas,__consumer_offsets,__transaction_state,connect-configs,connect-offsets,connect-status,connect-statuses", + "ksql.connect.basic.auth.credentials.file" : "", + "ksql.authorization.cache.expiry.time.secs" : "30", + "ksql.query.retry.backoff.initial.ms" : "15000", + "ksql.query.pull.max.concurrent.requests" : "2147483647", + "ksql.streams.auto.offset.reset" : "earliest", + "ksql.connect.url" : "http://localhost:8083", + "ksql.query.push.v2.new.latest.delay.ms" : "5000", + "ksql.query.push.v2.latest.reset.age.ms" : "30000", + "ksql.streams.default.production.exception.handler" : "io.confluent.ksql.errors.ProductionExceptionHandlerUtil$LogAndFailProductionExceptionHandler", + "ksql.query.pull.interpreter.enabled" : "true", + "ksql.query.pull.limit.clause.enabled" : "true", + "ksql.query.pull.router.thread.pool.size" : "50", + "ksql.query.push.v2.continuation.tokens.enabled" : "false", + "ksql.query.retry.backoff.max.ms" : "900000", + "ksql.timestamp.throw.on.invalid" : "false", + "ksql.persistence.default.format.value" : null, + "ksql.udfs.enabled" : "true", + "ksql.udf.enable.security.manager" : "true", + "ksql.connect.worker.config" : "", + "ksql.nested.error.set.null" : "true", + "ksql.query.pull.thread.pool.size" : "50", + "ksql.persistent.prefix" : "query_", + "ksql.metastore.backup.location" : "", + "ksql.error.classifier.regex" : "", + "ksql.suppress.buffer.size.bytes" : "-1" + } +} \ No newline at end of file diff --git a/ksqldb-functional-tests/src/test/resources/historical_plans/joins_-_on_BIGINT_column_-_KAFKA_-_PROTOBUF_NOSR/7.3.0_1651071425414/spec.json b/ksqldb-functional-tests/src/test/resources/historical_plans/joins_-_on_BIGINT_column_-_KAFKA_-_PROTOBUF_NOSR/7.3.0_1651071425414/spec.json new file mode 100644 index 000000000000..3c96ffb33bf1 --- /dev/null +++ b/ksqldb-functional-tests/src/test/resources/historical_plans/joins_-_on_BIGINT_column_-_KAFKA_-_PROTOBUF_NOSR/7.3.0_1651071425414/spec.json @@ -0,0 +1,194 @@ +{ + "version" : "7.3.0", + "timestamp" : 1651071425414, + "path" : "query-validation-tests/joins.json", + "schemas" : { + "CSAS_OUTPUT_0.KafkaTopic_Right.Source" : { + "schema" : "`ID` STRING KEY, `R0` BIGINT, `R1` INTEGER", + "keyFormat" : { + "format" : "KAFKA" + }, + "valueFormat" : { + "format" : "PROTOBUF_NOSR" + } + }, + "CSAS_OUTPUT_0.Join.Left" : { + "schema" : "`L_L0` BIGINT KEY, `L_L0` BIGINT, `L_L1` INTEGER, `L_ROWTIME` BIGINT, `L_ROWPARTITION` INTEGER, `L_ROWOFFSET` BIGINT, `L_ID` STRING", + "valueFormat" : { + "format" : "PROTOBUF_NOSR" + } + }, + "CSAS_OUTPUT_0.OUTPUT" : { + "schema" : "`L0` BIGINT KEY, `L_ID` STRING, `L1` INTEGER, `R1` INTEGER", + "keyFormat" : { + "format" : "KAFKA" + }, + "valueFormat" : { + "format" : "PROTOBUF_NOSR" + } + }, + "CSAS_OUTPUT_0.KafkaTopic_Left.Source" : { + "schema" : "`ID` STRING KEY, `L0` BIGINT, `L1` INTEGER", + "keyFormat" : { + "format" : "KAFKA" + }, + "valueFormat" : { + "format" : "PROTOBUF_NOSR" + } + }, + "CSAS_OUTPUT_0.Join" : { + "schema" : "`L_L0` BIGINT KEY, `L_L0` BIGINT, `L_L1` INTEGER, `L_ROWTIME` BIGINT, `L_ROWPARTITION` INTEGER, `L_ROWOFFSET` BIGINT, `L_ID` STRING", + "keyFormat" : { + "format" : "KAFKA" + } + }, + "CSAS_OUTPUT_0.Join.Right" : { + "schema" : "`R_R0` BIGINT KEY, `R_R0` BIGINT, `R_R1` INTEGER, `R_ROWTIME` BIGINT, `R_ROWPARTITION` INTEGER, `R_ROWOFFSET` BIGINT, `R_ID` STRING", + "valueFormat" : { + "format" : "PROTOBUF_NOSR" + } + } + }, + "testCase" : { + "name" : "on BIGINT column - KAFKA - PROTOBUF_NOSR", + "inputs" : [ { + "topic" : "left_topic", + "key" : "a", + "value" : { + "L0" : 1000000000, + "L1" : 1 + }, + "timestamp" : 0 + }, { + "topic" : "right_topic", + "key" : "b", + "value" : { + "R0" : 1000000000, + "R1" : 2 + }, + "timestamp" : 10000 + } ], + "outputs" : [ { + "topic" : "OUTPUT", + "key" : 1000000000, + "value" : { + "L_ID" : "a", + "L1" : 1, + "R1" : 2 + }, + "timestamp" : 10000 + } ], + "topics" : [ { + "name" : "right_topic", + "replicas" : 1, + "numPartitions" : 4 + }, { + "name" : "OUTPUT", + "replicas" : 1, + "numPartitions" : 4 + }, { + "name" : "left_topic", + "replicas" : 1, + "numPartitions" : 4 + } ], + "statements" : [ "CREATE STREAM L (ID STRING KEY, l0 BIGINT, l1 INT) WITH (kafka_topic='left_topic', value_format='PROTOBUF_NOSR');", "CREATE STREAM R (ID STRING KEY, r0 BIGINT, r1 INT) WITH (kafka_topic='right_topic', value_format='PROTOBUF_NOSR');", "CREATE STREAM OUTPUT as SELECT L.l0, L.ID, L1, R1 FROM L join R WITHIN 11 SECONDS ON L.l0 = R.r0;" ], + "post" : { + "sources" : [ { + "name" : "L", + "type" : "STREAM", + "schema" : "`ID` STRING KEY, `L0` BIGINT, `L1` INTEGER", + "keyFormat" : { + "format" : "KAFKA" + }, + "valueFormat" : "PROTOBUF_NOSR", + "keyFeatures" : [ ], + "valueFeatures" : [ ], + "isSource" : false + }, { + "name" : "OUTPUT", + "type" : "STREAM", + "schema" : "`L0` BIGINT KEY, `L_ID` STRING, `L1` INTEGER, `R1` INTEGER", + "keyFormat" : { + "format" : "KAFKA" + }, + "valueFormat" : "PROTOBUF_NOSR", + "keyFeatures" : [ ], + "valueFeatures" : [ ], + "isSource" : false + }, { + "name" : "R", + "type" : "STREAM", + "schema" : "`ID` STRING KEY, `R0` BIGINT, `R1` INTEGER", + "keyFormat" : { + "format" : "KAFKA" + }, + "valueFormat" : "PROTOBUF_NOSR", + "keyFeatures" : [ ], + "valueFeatures" : [ ], + "isSource" : false + } ], + "topics" : { + "topics" : [ { + "name" : "_confluent-ksql-some.ksql.service.idquery_CSAS_OUTPUT_0-Join-left-repartition", + "keyFormat" : { + "format" : "KAFKA" + }, + "valueFormat" : { + "format" : "PROTOBUF_NOSR" + } + }, { + "name" : "_confluent-ksql-some.ksql.service.idquery_CSAS_OUTPUT_0-Join-right-repartition", + "keyFormat" : { + "format" : "KAFKA" + }, + "valueFormat" : { + "format" : "PROTOBUF_NOSR" + } + }, { + "name" : "_confluent-ksql-some.ksql.service.idquery_CSAS_OUTPUT_0-KSTREAM-JOINTHIS-0000000016-store-changelog", + "keyFormat" : { + "format" : "KAFKA" + }, + "valueFormat" : { + "format" : "PROTOBUF_NOSR" + } + }, { + "name" : "_confluent-ksql-some.ksql.service.idquery_CSAS_OUTPUT_0-KSTREAM-JOINOTHER-0000000017-store-changelog", + "keyFormat" : { + "format" : "KAFKA" + }, + "valueFormat" : { + "format" : "PROTOBUF_NOSR" + } + }, { + "name" : "left_topic", + "keyFormat" : { + "format" : "KAFKA" + }, + "valueFormat" : { + "format" : "PROTOBUF_NOSR" + }, + "partitions" : 4 + }, { + "name" : "OUTPUT", + "keyFormat" : { + "format" : "KAFKA" + }, + "valueFormat" : { + "format" : "PROTOBUF_NOSR" + }, + "partitions" : 4 + }, { + "name" : "right_topic", + "keyFormat" : { + "format" : "KAFKA" + }, + "valueFormat" : { + "format" : "PROTOBUF_NOSR" + }, + "partitions" : 4 + } ] + } + } + } +} \ No newline at end of file diff --git a/ksqldb-functional-tests/src/test/resources/historical_plans/joins_-_on_BIGINT_column_-_KAFKA_-_PROTOBUF_NOSR/7.3.0_1651071425414/topology b/ksqldb-functional-tests/src/test/resources/historical_plans/joins_-_on_BIGINT_column_-_KAFKA_-_PROTOBUF_NOSR/7.3.0_1651071425414/topology new file mode 100644 index 000000000000..e6d9bd120b1e --- /dev/null +++ b/ksqldb-functional-tests/src/test/resources/historical_plans/joins_-_on_BIGINT_column_-_KAFKA_-_PROTOBUF_NOSR/7.3.0_1651071425414/topology @@ -0,0 +1,63 @@ +Topologies: + Sub-topology: 0 + Source: KSTREAM-SOURCE-0000000000 (topics: [left_topic]) + --> KSTREAM-TRANSFORMVALUES-0000000001 + Processor: KSTREAM-TRANSFORMVALUES-0000000001 (stores: []) + --> LeftSourceKeyed-SelectKey + <-- KSTREAM-SOURCE-0000000000 + Processor: LeftSourceKeyed-SelectKey (stores: []) + --> PrependAliasLeft + <-- KSTREAM-TRANSFORMVALUES-0000000001 + Processor: PrependAliasLeft (stores: []) + --> Join-left-repartition-filter + <-- LeftSourceKeyed-SelectKey + Processor: Join-left-repartition-filter (stores: []) + --> Join-left-repartition-sink + <-- PrependAliasLeft + Sink: Join-left-repartition-sink (topic: Join-left-repartition) + <-- Join-left-repartition-filter + + Sub-topology: 1 + Source: KSTREAM-SOURCE-0000000004 (topics: [right_topic]) + --> KSTREAM-TRANSFORMVALUES-0000000005 + Processor: KSTREAM-TRANSFORMVALUES-0000000005 (stores: []) + --> RightSourceKeyed-SelectKey + <-- KSTREAM-SOURCE-0000000004 + Processor: RightSourceKeyed-SelectKey (stores: []) + --> PrependAliasRight + <-- KSTREAM-TRANSFORMVALUES-0000000005 + Processor: PrependAliasRight (stores: []) + --> Join-right-repartition-filter + <-- RightSourceKeyed-SelectKey + Processor: Join-right-repartition-filter (stores: []) + --> Join-right-repartition-sink + <-- PrependAliasRight + Sink: Join-right-repartition-sink (topic: Join-right-repartition) + <-- Join-right-repartition-filter + + Sub-topology: 2 + Source: Join-left-repartition-source (topics: [Join-left-repartition]) + --> Join-this-windowed + Source: Join-right-repartition-source (topics: [Join-right-repartition]) + --> Join-other-windowed + Processor: Join-other-windowed (stores: [KSTREAM-JOINOTHER-0000000017-store]) + --> Join-other-join + <-- Join-right-repartition-source + Processor: Join-this-windowed (stores: [KSTREAM-JOINTHIS-0000000016-store]) + --> Join-this-join + <-- Join-left-repartition-source + Processor: Join-other-join (stores: [KSTREAM-JOINTHIS-0000000016-store]) + --> Join-merge + <-- Join-other-windowed + Processor: Join-this-join (stores: [KSTREAM-JOINOTHER-0000000017-store]) + --> Join-merge + <-- Join-this-windowed + Processor: Join-merge (stores: []) + --> Project + <-- Join-this-join, Join-other-join + Processor: Project (stores: []) + --> KSTREAM-SINK-0000000020 + <-- Join-merge + Sink: KSTREAM-SINK-0000000020 (topic: OUTPUT) + <-- Project + diff --git a/ksqldb-functional-tests/src/test/resources/historical_plans/joins_-_on_DOUBLE_column_=_KAFKA_-_PROTOBUF_NOSR/7.3.0_1651071428133/plan.json b/ksqldb-functional-tests/src/test/resources/historical_plans/joins_-_on_DOUBLE_column_=_KAFKA_-_PROTOBUF_NOSR/7.3.0_1651071428133/plan.json new file mode 100644 index 000000000000..457fecdd20e1 --- /dev/null +++ b/ksqldb-functional-tests/src/test/resources/historical_plans/joins_-_on_DOUBLE_column_=_KAFKA_-_PROTOBUF_NOSR/7.3.0_1651071428133/plan.json @@ -0,0 +1,310 @@ +{ + "plan" : [ { + "@type" : "ksqlPlanV1", + "statementText" : "CREATE STREAM L (ID STRING KEY, L0 DOUBLE, L1 INTEGER) WITH (KAFKA_TOPIC='left_topic', KEY_FORMAT='KAFKA', VALUE_FORMAT='PROTOBUF_NOSR');", + "ddlCommand" : { + "@type" : "createStreamV1", + "sourceName" : "L", + "schema" : "`ID` STRING KEY, `L0` DOUBLE, `L1` INTEGER", + "timestampColumn" : null, + "topicName" : "left_topic", + "formats" : { + "keyFormat" : { + "format" : "KAFKA", + "properties" : { } + }, + "valueFormat" : { + "format" : "PROTOBUF_NOSR", + "properties" : { } + } + }, + "windowInfo" : null, + "orReplace" : false, + "isSource" : false + }, + "queryPlan" : null + }, { + "@type" : "ksqlPlanV1", + "statementText" : "CREATE STREAM R (ID STRING KEY, R0 DOUBLE, R1 INTEGER) WITH (KAFKA_TOPIC='right_topic', KEY_FORMAT='KAFKA', VALUE_FORMAT='PROTOBUF_NOSR');", + "ddlCommand" : { + "@type" : "createStreamV1", + "sourceName" : "R", + "schema" : "`ID` STRING KEY, `R0` DOUBLE, `R1` INTEGER", + "timestampColumn" : null, + "topicName" : "right_topic", + "formats" : { + "keyFormat" : { + "format" : "KAFKA", + "properties" : { } + }, + "valueFormat" : { + "format" : "PROTOBUF_NOSR", + "properties" : { } + } + }, + "windowInfo" : null, + "orReplace" : false, + "isSource" : false + }, + "queryPlan" : null + }, { + "@type" : "ksqlPlanV1", + "statementText" : "CREATE STREAM OUTPUT AS SELECT\n L.L0 L0,\n L.ID L_ID,\n L.L1 L1,\n R.R1 R1\nFROM L L\nINNER JOIN R R WITHIN 11 SECONDS ON ((L.L0 = R.R0))\nEMIT CHANGES", + "ddlCommand" : { + "@type" : "createStreamV1", + "sourceName" : "OUTPUT", + "schema" : "`L0` DOUBLE KEY, `L_ID` STRING, `L1` INTEGER, `R1` INTEGER", + "timestampColumn" : null, + "topicName" : "OUTPUT", + "formats" : { + "keyFormat" : { + "format" : "KAFKA", + "properties" : { } + }, + "valueFormat" : { + "format" : "PROTOBUF_NOSR", + "properties" : { } + } + }, + "windowInfo" : null, + "orReplace" : false, + "isSource" : false + }, + "queryPlan" : { + "sources" : [ "L", "R" ], + "sink" : "OUTPUT", + "physicalPlan" : { + "@type" : "streamSinkV1", + "properties" : { + "queryContext" : "OUTPUT" + }, + "source" : { + "@type" : "streamSelectV1", + "properties" : { + "queryContext" : "Project" + }, + "source" : { + "@type" : "streamStreamJoinV1", + "properties" : { + "queryContext" : "Join" + }, + "joinType" : "INNER", + "leftInternalFormats" : { + "keyFormat" : { + "format" : "KAFKA", + "properties" : { } + }, + "valueFormat" : { + "format" : "PROTOBUF_NOSR", + "properties" : { } + } + }, + "rightInternalFormats" : { + "keyFormat" : { + "format" : "KAFKA", + "properties" : { } + }, + "valueFormat" : { + "format" : "PROTOBUF_NOSR", + "properties" : { } + } + }, + "leftSource" : { + "@type" : "streamSelectV1", + "properties" : { + "queryContext" : "PrependAliasLeft" + }, + "source" : { + "@type" : "streamSelectKeyV2", + "properties" : { + "queryContext" : "LeftSourceKeyed" + }, + "source" : { + "@type" : "streamSourceV1", + "properties" : { + "queryContext" : "KafkaTopic_Left/Source" + }, + "topicName" : "left_topic", + "formats" : { + "keyFormat" : { + "format" : "KAFKA", + "properties" : { } + }, + "valueFormat" : { + "format" : "PROTOBUF_NOSR", + "properties" : { } + } + }, + "timestampColumn" : null, + "sourceSchema" : "`ID` STRING KEY, `L0` DOUBLE, `L1` INTEGER", + "pseudoColumnVersion" : 1 + }, + "keyExpression" : [ "L0" ] + }, + "keyColumnNames" : [ "L_L0" ], + "selectedKeys" : null, + "selectExpressions" : [ "L0 AS L_L0", "L1 AS L_L1", "ROWTIME AS L_ROWTIME", "ROWPARTITION AS L_ROWPARTITION", "ROWOFFSET AS L_ROWOFFSET", "ID AS L_ID" ] + }, + "rightSource" : { + "@type" : "streamSelectV1", + "properties" : { + "queryContext" : "PrependAliasRight" + }, + "source" : { + "@type" : "streamSelectKeyV2", + "properties" : { + "queryContext" : "RightSourceKeyed" + }, + "source" : { + "@type" : "streamSourceV1", + "properties" : { + "queryContext" : "KafkaTopic_Right/Source" + }, + "topicName" : "right_topic", + "formats" : { + "keyFormat" : { + "format" : "KAFKA", + "properties" : { } + }, + "valueFormat" : { + "format" : "PROTOBUF_NOSR", + "properties" : { } + } + }, + "timestampColumn" : null, + "sourceSchema" : "`ID` STRING KEY, `R0` DOUBLE, `R1` INTEGER", + "pseudoColumnVersion" : 1 + }, + "keyExpression" : [ "R0" ] + }, + "keyColumnNames" : [ "R_R0" ], + "selectedKeys" : null, + "selectExpressions" : [ "R0 AS R_R0", "R1 AS R_R1", "ROWTIME AS R_ROWTIME", "ROWPARTITION AS R_ROWPARTITION", "ROWOFFSET AS R_ROWOFFSET", "ID AS R_ID" ] + }, + "beforeMillis" : 11.000000000, + "afterMillis" : 11.000000000, + "graceMillis" : null, + "keyColName" : "L_L0" + }, + "keyColumnNames" : [ "L0" ], + "selectedKeys" : null, + "selectExpressions" : [ "L_ID AS L_ID", "L_L1 AS L1", "R_R1 AS R1" ] + }, + "formats" : { + "keyFormat" : { + "format" : "KAFKA", + "properties" : { } + }, + "valueFormat" : { + "format" : "PROTOBUF_NOSR", + "properties" : { } + } + }, + "topicName" : "OUTPUT", + "timestampColumn" : null + }, + "queryId" : "CSAS_OUTPUT_0", + "runtimeId" : null + } + } ], + "configs" : { + "ksql.extension.dir" : "ext", + "ksql.streams.cache.max.bytes.buffering" : "0", + "metric.reporters" : "", + "ksql.query.status.running.threshold.seconds" : "300", + "ksql.connect.basic.auth.credentials.reload" : "false", + "ksql.output.topic.name.prefix" : "", + "ksql.query.pull.stream.enabled" : "true", + "ksql.query.push.v2.interpreter.enabled" : "true", + "ksql.queryanonymizer.logs_enabled" : "true", + "ksql.variable.substitution.enable" : "true", + "ksql.streams.shutdown.timeout.ms" : "300000", + "ksql.query.pull.max.allowed.offset.lag" : "9223372036854775807", + "ksql.query.pull.max.qps" : "2147483647", + "ksql.access.validator.enable" : "auto", + "ksql.streams.bootstrap.servers" : "localhost:0", + "ksql.query.pull.metrics.enabled" : "true", + "ksql.metrics.extension" : null, + "ksql.query.push.v2.alos.enabled" : "true", + "ksql.query.push.v2.max.hourly.bandwidth.megabytes" : "2147483647", + "ksql.query.pull.range.scan.enabled" : "true", + "ksql.transient.query.cleanup.service.initial.delay.seconds" : "600", + "ksql.hidden.topics" : "_confluent.*,__confluent.*,_schemas,__consumer_offsets,__transaction_state,connect-configs,connect-offsets,connect-status,connect-statuses", + "ksql.lambdas.enabled" : "true", + "ksql.source.table.materialization.enabled" : "true", + "ksql.query.pull.max.hourly.bandwidth.megabytes" : "2147483647", + "ksql.sink.window.change.log.additional.retention" : "1000000", + "ksql.query.persistent.active.limit" : "2147483647", + "ksql.persistence.wrap.single.values" : null, + "ksql.query.transient.max.bytes.buffering.total" : "-1", + "ksql.connect.basic.auth.credentials.source" : "NONE", + "ksql.schema.registry.url" : "schema_registry.url:0", + "ksql.properties.overrides.denylist" : "", + "ksql.service.id" : "some.ksql.service.id", + "ksql.query.push.v2.max.catchup.consumers" : "5", + "ksql.query.push.v2.enabled" : "false", + "ksql.transient.query.cleanup.service.enable" : "true", + "ksql.query.push.v2.metrics.enabled" : "true", + "ksql.rowpartition.rowoffset.enabled" : "true", + "ksql.streams.commit.interval.ms" : "2000", + "ksql.query.pull.table.scan.enabled" : "true", + "ksql.streams.auto.commit.interval.ms" : "0", + "ksql.streams.topology.optimization" : "all", + "ksql.endpoint.migrate.query" : "true", + "ksql.query.push.v2.registry.installed" : "false", + "ksql.streams.num.stream.threads" : "4", + "ksql.metrics.tags.custom" : "", + "ksql.query.push.v2.catchup.consumer.msg.window" : "50", + "ksql.runtime.feature.shared.enabled" : "false", + "ksql.udf.collect.metrics" : "false", + "ksql.new.query.planner.enabled" : "false", + "ksql.connect.request.headers.plugin" : null, + "ksql.security.extension.class" : null, + "ksql.transient.prefix" : "transient_", + "ksql.headers.columns.enabled" : "true", + "ksql.streams.default.deserialization.exception.handler" : "io.confluent.ksql.errors.LogMetricAndContinueExceptionHandler", + "ksql.connect.request.timeout.ms" : "5000", + "ksql.query.pull.enable.standby.reads" : "false", + "ksql.persistence.default.format.key" : "KAFKA", + "ksql.query.persistent.max.bytes.buffering.total" : "-1", + "ksql.query.error.max.queue.size" : "10", + "ksql.query.cleanup.shutdown.timeout.ms" : "30000", + "ksql.internal.topic.min.insync.replicas" : "1", + "ksql.internal.topic.replicas" : "1", + "ksql.insert.into.values.enabled" : "true", + "ksql.queryanonymizer.cluster_namespace" : null, + "ksql.create.or.replace.enabled" : "true", + "ksql.shared.runtimes.count" : "2", + "ksql.cast.strings.preserve.nulls" : "true", + "ksql.authorization.cache.max.entries" : "10000", + "ksql.pull.queries.enable" : "true", + "ksql.transient.query.cleanup.service.period.seconds" : "600", + "ksql.suppress.enabled" : "false", + "ksql.readonly.topics" : "_confluent.*,__confluent.*,_schemas,__consumer_offsets,__transaction_state,connect-configs,connect-offsets,connect-status,connect-statuses", + "ksql.connect.basic.auth.credentials.file" : "", + "ksql.authorization.cache.expiry.time.secs" : "30", + "ksql.query.retry.backoff.initial.ms" : "15000", + "ksql.query.pull.max.concurrent.requests" : "2147483647", + "ksql.streams.auto.offset.reset" : "earliest", + "ksql.connect.url" : "http://localhost:8083", + "ksql.query.push.v2.new.latest.delay.ms" : "5000", + "ksql.query.push.v2.latest.reset.age.ms" : "30000", + "ksql.streams.default.production.exception.handler" : "io.confluent.ksql.errors.ProductionExceptionHandlerUtil$LogAndFailProductionExceptionHandler", + "ksql.query.pull.interpreter.enabled" : "true", + "ksql.query.pull.limit.clause.enabled" : "true", + "ksql.query.pull.router.thread.pool.size" : "50", + "ksql.query.push.v2.continuation.tokens.enabled" : "false", + "ksql.query.retry.backoff.max.ms" : "900000", + "ksql.timestamp.throw.on.invalid" : "false", + "ksql.persistence.default.format.value" : null, + "ksql.udfs.enabled" : "true", + "ksql.udf.enable.security.manager" : "true", + "ksql.connect.worker.config" : "", + "ksql.nested.error.set.null" : "true", + "ksql.query.pull.thread.pool.size" : "50", + "ksql.persistent.prefix" : "query_", + "ksql.metastore.backup.location" : "", + "ksql.error.classifier.regex" : "", + "ksql.suppress.buffer.size.bytes" : "-1" + } +} \ No newline at end of file diff --git a/ksqldb-functional-tests/src/test/resources/historical_plans/joins_-_on_DOUBLE_column_=_KAFKA_-_PROTOBUF_NOSR/7.3.0_1651071428133/spec.json b/ksqldb-functional-tests/src/test/resources/historical_plans/joins_-_on_DOUBLE_column_=_KAFKA_-_PROTOBUF_NOSR/7.3.0_1651071428133/spec.json new file mode 100644 index 000000000000..45b615d5e178 --- /dev/null +++ b/ksqldb-functional-tests/src/test/resources/historical_plans/joins_-_on_DOUBLE_column_=_KAFKA_-_PROTOBUF_NOSR/7.3.0_1651071428133/spec.json @@ -0,0 +1,194 @@ +{ + "version" : "7.3.0", + "timestamp" : 1651071428133, + "path" : "query-validation-tests/joins.json", + "schemas" : { + "CSAS_OUTPUT_0.KafkaTopic_Right.Source" : { + "schema" : "`ID` STRING KEY, `R0` DOUBLE, `R1` INTEGER", + "keyFormat" : { + "format" : "KAFKA" + }, + "valueFormat" : { + "format" : "PROTOBUF_NOSR" + } + }, + "CSAS_OUTPUT_0.Join.Left" : { + "schema" : "`L_L0` DOUBLE KEY, `L_L0` DOUBLE, `L_L1` INTEGER, `L_ROWTIME` BIGINT, `L_ROWPARTITION` INTEGER, `L_ROWOFFSET` BIGINT, `L_ID` STRING", + "valueFormat" : { + "format" : "PROTOBUF_NOSR" + } + }, + "CSAS_OUTPUT_0.OUTPUT" : { + "schema" : "`L0` DOUBLE KEY, `L_ID` STRING, `L1` INTEGER, `R1` INTEGER", + "keyFormat" : { + "format" : "KAFKA" + }, + "valueFormat" : { + "format" : "PROTOBUF_NOSR" + } + }, + "CSAS_OUTPUT_0.KafkaTopic_Left.Source" : { + "schema" : "`ID` STRING KEY, `L0` DOUBLE, `L1` INTEGER", + "keyFormat" : { + "format" : "KAFKA" + }, + "valueFormat" : { + "format" : "PROTOBUF_NOSR" + } + }, + "CSAS_OUTPUT_0.Join" : { + "schema" : "`L_L0` DOUBLE KEY, `L_L0` DOUBLE, `L_L1` INTEGER, `L_ROWTIME` BIGINT, `L_ROWPARTITION` INTEGER, `L_ROWOFFSET` BIGINT, `L_ID` STRING", + "keyFormat" : { + "format" : "KAFKA" + } + }, + "CSAS_OUTPUT_0.Join.Right" : { + "schema" : "`R_R0` DOUBLE KEY, `R_R0` DOUBLE, `R_R1` INTEGER, `R_ROWTIME` BIGINT, `R_ROWPARTITION` INTEGER, `R_ROWOFFSET` BIGINT, `R_ID` STRING", + "valueFormat" : { + "format" : "PROTOBUF_NOSR" + } + } + }, + "testCase" : { + "name" : "on DOUBLE column = KAFKA - PROTOBUF_NOSR", + "inputs" : [ { + "topic" : "left_topic", + "key" : "a", + "value" : { + "L0" : 1.23, + "L1" : 1 + }, + "timestamp" : 0 + }, { + "topic" : "right_topic", + "key" : "b", + "value" : { + "R0" : 1.23, + "R1" : 2 + }, + "timestamp" : 10000 + } ], + "outputs" : [ { + "topic" : "OUTPUT", + "key" : 1.23, + "value" : { + "L_ID" : "a", + "L1" : 1, + "R1" : 2 + }, + "timestamp" : 10000 + } ], + "topics" : [ { + "name" : "right_topic", + "replicas" : 1, + "numPartitions" : 4 + }, { + "name" : "OUTPUT", + "replicas" : 1, + "numPartitions" : 4 + }, { + "name" : "left_topic", + "replicas" : 1, + "numPartitions" : 4 + } ], + "statements" : [ "CREATE STREAM L (ID STRING KEY, l0 DOUBLE, l1 INT) WITH (kafka_topic='left_topic', value_format='PROTOBUF_NOSR');", "CREATE STREAM R (ID STRING KEY, r0 DOUBLE, r1 INT) WITH (kafka_topic='right_topic', value_format='PROTOBUF_NOSR');", "CREATE STREAM OUTPUT as SELECT L.l0, L.ID, L1, R1 FROM L join R WITHIN 11 SECONDS ON L.l0 = R.r0;" ], + "post" : { + "sources" : [ { + "name" : "L", + "type" : "STREAM", + "schema" : "`ID` STRING KEY, `L0` DOUBLE, `L1` INTEGER", + "keyFormat" : { + "format" : "KAFKA" + }, + "valueFormat" : "PROTOBUF_NOSR", + "keyFeatures" : [ ], + "valueFeatures" : [ ], + "isSource" : false + }, { + "name" : "OUTPUT", + "type" : "STREAM", + "schema" : "`L0` DOUBLE KEY, `L_ID` STRING, `L1` INTEGER, `R1` INTEGER", + "keyFormat" : { + "format" : "KAFKA" + }, + "valueFormat" : "PROTOBUF_NOSR", + "keyFeatures" : [ ], + "valueFeatures" : [ ], + "isSource" : false + }, { + "name" : "R", + "type" : "STREAM", + "schema" : "`ID` STRING KEY, `R0` DOUBLE, `R1` INTEGER", + "keyFormat" : { + "format" : "KAFKA" + }, + "valueFormat" : "PROTOBUF_NOSR", + "keyFeatures" : [ ], + "valueFeatures" : [ ], + "isSource" : false + } ], + "topics" : { + "topics" : [ { + "name" : "_confluent-ksql-some.ksql.service.idquery_CSAS_OUTPUT_0-Join-left-repartition", + "keyFormat" : { + "format" : "KAFKA" + }, + "valueFormat" : { + "format" : "PROTOBUF_NOSR" + } + }, { + "name" : "_confluent-ksql-some.ksql.service.idquery_CSAS_OUTPUT_0-Join-right-repartition", + "keyFormat" : { + "format" : "KAFKA" + }, + "valueFormat" : { + "format" : "PROTOBUF_NOSR" + } + }, { + "name" : "_confluent-ksql-some.ksql.service.idquery_CSAS_OUTPUT_0-KSTREAM-JOINTHIS-0000000016-store-changelog", + "keyFormat" : { + "format" : "KAFKA" + }, + "valueFormat" : { + "format" : "PROTOBUF_NOSR" + } + }, { + "name" : "_confluent-ksql-some.ksql.service.idquery_CSAS_OUTPUT_0-KSTREAM-JOINOTHER-0000000017-store-changelog", + "keyFormat" : { + "format" : "KAFKA" + }, + "valueFormat" : { + "format" : "PROTOBUF_NOSR" + } + }, { + "name" : "left_topic", + "keyFormat" : { + "format" : "KAFKA" + }, + "valueFormat" : { + "format" : "PROTOBUF_NOSR" + }, + "partitions" : 4 + }, { + "name" : "OUTPUT", + "keyFormat" : { + "format" : "KAFKA" + }, + "valueFormat" : { + "format" : "PROTOBUF_NOSR" + }, + "partitions" : 4 + }, { + "name" : "right_topic", + "keyFormat" : { + "format" : "KAFKA" + }, + "valueFormat" : { + "format" : "PROTOBUF_NOSR" + }, + "partitions" : 4 + } ] + } + } + } +} \ No newline at end of file diff --git a/ksqldb-functional-tests/src/test/resources/historical_plans/joins_-_on_DOUBLE_column_=_KAFKA_-_PROTOBUF_NOSR/7.3.0_1651071428133/topology b/ksqldb-functional-tests/src/test/resources/historical_plans/joins_-_on_DOUBLE_column_=_KAFKA_-_PROTOBUF_NOSR/7.3.0_1651071428133/topology new file mode 100644 index 000000000000..e6d9bd120b1e --- /dev/null +++ b/ksqldb-functional-tests/src/test/resources/historical_plans/joins_-_on_DOUBLE_column_=_KAFKA_-_PROTOBUF_NOSR/7.3.0_1651071428133/topology @@ -0,0 +1,63 @@ +Topologies: + Sub-topology: 0 + Source: KSTREAM-SOURCE-0000000000 (topics: [left_topic]) + --> KSTREAM-TRANSFORMVALUES-0000000001 + Processor: KSTREAM-TRANSFORMVALUES-0000000001 (stores: []) + --> LeftSourceKeyed-SelectKey + <-- KSTREAM-SOURCE-0000000000 + Processor: LeftSourceKeyed-SelectKey (stores: []) + --> PrependAliasLeft + <-- KSTREAM-TRANSFORMVALUES-0000000001 + Processor: PrependAliasLeft (stores: []) + --> Join-left-repartition-filter + <-- LeftSourceKeyed-SelectKey + Processor: Join-left-repartition-filter (stores: []) + --> Join-left-repartition-sink + <-- PrependAliasLeft + Sink: Join-left-repartition-sink (topic: Join-left-repartition) + <-- Join-left-repartition-filter + + Sub-topology: 1 + Source: KSTREAM-SOURCE-0000000004 (topics: [right_topic]) + --> KSTREAM-TRANSFORMVALUES-0000000005 + Processor: KSTREAM-TRANSFORMVALUES-0000000005 (stores: []) + --> RightSourceKeyed-SelectKey + <-- KSTREAM-SOURCE-0000000004 + Processor: RightSourceKeyed-SelectKey (stores: []) + --> PrependAliasRight + <-- KSTREAM-TRANSFORMVALUES-0000000005 + Processor: PrependAliasRight (stores: []) + --> Join-right-repartition-filter + <-- RightSourceKeyed-SelectKey + Processor: Join-right-repartition-filter (stores: []) + --> Join-right-repartition-sink + <-- PrependAliasRight + Sink: Join-right-repartition-sink (topic: Join-right-repartition) + <-- Join-right-repartition-filter + + Sub-topology: 2 + Source: Join-left-repartition-source (topics: [Join-left-repartition]) + --> Join-this-windowed + Source: Join-right-repartition-source (topics: [Join-right-repartition]) + --> Join-other-windowed + Processor: Join-other-windowed (stores: [KSTREAM-JOINOTHER-0000000017-store]) + --> Join-other-join + <-- Join-right-repartition-source + Processor: Join-this-windowed (stores: [KSTREAM-JOINTHIS-0000000016-store]) + --> Join-this-join + <-- Join-left-repartition-source + Processor: Join-other-join (stores: [KSTREAM-JOINTHIS-0000000016-store]) + --> Join-merge + <-- Join-other-windowed + Processor: Join-this-join (stores: [KSTREAM-JOINOTHER-0000000017-store]) + --> Join-merge + <-- Join-this-windowed + Processor: Join-merge (stores: []) + --> Project + <-- Join-this-join, Join-other-join + Processor: Project (stores: []) + --> KSTREAM-SINK-0000000020 + <-- Join-merge + Sink: KSTREAM-SINK-0000000020 (topic: OUTPUT) + <-- Project + diff --git a/ksqldb-functional-tests/src/test/resources/historical_plans/joins_-_on_INT_column_-_KAFKA_-_PROTOBUF_NOSR/7.3.0_1651071422470/plan.json b/ksqldb-functional-tests/src/test/resources/historical_plans/joins_-_on_INT_column_-_KAFKA_-_PROTOBUF_NOSR/7.3.0_1651071422470/plan.json new file mode 100644 index 000000000000..c60bbbde4c2a --- /dev/null +++ b/ksqldb-functional-tests/src/test/resources/historical_plans/joins_-_on_INT_column_-_KAFKA_-_PROTOBUF_NOSR/7.3.0_1651071422470/plan.json @@ -0,0 +1,310 @@ +{ + "plan" : [ { + "@type" : "ksqlPlanV1", + "statementText" : "CREATE STREAM L (ID STRING KEY, L0 INTEGER, L1 INTEGER) WITH (KAFKA_TOPIC='left_topic', KEY_FORMAT='KAFKA', VALUE_FORMAT='PROTOBUF_NOSR');", + "ddlCommand" : { + "@type" : "createStreamV1", + "sourceName" : "L", + "schema" : "`ID` STRING KEY, `L0` INTEGER, `L1` INTEGER", + "timestampColumn" : null, + "topicName" : "left_topic", + "formats" : { + "keyFormat" : { + "format" : "KAFKA", + "properties" : { } + }, + "valueFormat" : { + "format" : "PROTOBUF_NOSR", + "properties" : { } + } + }, + "windowInfo" : null, + "orReplace" : false, + "isSource" : false + }, + "queryPlan" : null + }, { + "@type" : "ksqlPlanV1", + "statementText" : "CREATE STREAM R (ID STRING KEY, R0 INTEGER, R1 INTEGER) WITH (KAFKA_TOPIC='right_topic', KEY_FORMAT='KAFKA', VALUE_FORMAT='PROTOBUF_NOSR');", + "ddlCommand" : { + "@type" : "createStreamV1", + "sourceName" : "R", + "schema" : "`ID` STRING KEY, `R0` INTEGER, `R1` INTEGER", + "timestampColumn" : null, + "topicName" : "right_topic", + "formats" : { + "keyFormat" : { + "format" : "KAFKA", + "properties" : { } + }, + "valueFormat" : { + "format" : "PROTOBUF_NOSR", + "properties" : { } + } + }, + "windowInfo" : null, + "orReplace" : false, + "isSource" : false + }, + "queryPlan" : null + }, { + "@type" : "ksqlPlanV1", + "statementText" : "CREATE STREAM OUTPUT AS SELECT\n L.L0 L0,\n L.ID L_ID,\n L.L1 L1,\n R.R1 R1\nFROM L L\nINNER JOIN R R WITHIN 11 SECONDS ON ((L.L0 = R.R0))\nEMIT CHANGES", + "ddlCommand" : { + "@type" : "createStreamV1", + "sourceName" : "OUTPUT", + "schema" : "`L0` INTEGER KEY, `L_ID` STRING, `L1` INTEGER, `R1` INTEGER", + "timestampColumn" : null, + "topicName" : "OUTPUT", + "formats" : { + "keyFormat" : { + "format" : "KAFKA", + "properties" : { } + }, + "valueFormat" : { + "format" : "PROTOBUF_NOSR", + "properties" : { } + } + }, + "windowInfo" : null, + "orReplace" : false, + "isSource" : false + }, + "queryPlan" : { + "sources" : [ "L", "R" ], + "sink" : "OUTPUT", + "physicalPlan" : { + "@type" : "streamSinkV1", + "properties" : { + "queryContext" : "OUTPUT" + }, + "source" : { + "@type" : "streamSelectV1", + "properties" : { + "queryContext" : "Project" + }, + "source" : { + "@type" : "streamStreamJoinV1", + "properties" : { + "queryContext" : "Join" + }, + "joinType" : "INNER", + "leftInternalFormats" : { + "keyFormat" : { + "format" : "KAFKA", + "properties" : { } + }, + "valueFormat" : { + "format" : "PROTOBUF_NOSR", + "properties" : { } + } + }, + "rightInternalFormats" : { + "keyFormat" : { + "format" : "KAFKA", + "properties" : { } + }, + "valueFormat" : { + "format" : "PROTOBUF_NOSR", + "properties" : { } + } + }, + "leftSource" : { + "@type" : "streamSelectV1", + "properties" : { + "queryContext" : "PrependAliasLeft" + }, + "source" : { + "@type" : "streamSelectKeyV2", + "properties" : { + "queryContext" : "LeftSourceKeyed" + }, + "source" : { + "@type" : "streamSourceV1", + "properties" : { + "queryContext" : "KafkaTopic_Left/Source" + }, + "topicName" : "left_topic", + "formats" : { + "keyFormat" : { + "format" : "KAFKA", + "properties" : { } + }, + "valueFormat" : { + "format" : "PROTOBUF_NOSR", + "properties" : { } + } + }, + "timestampColumn" : null, + "sourceSchema" : "`ID` STRING KEY, `L0` INTEGER, `L1` INTEGER", + "pseudoColumnVersion" : 1 + }, + "keyExpression" : [ "L0" ] + }, + "keyColumnNames" : [ "L_L0" ], + "selectedKeys" : null, + "selectExpressions" : [ "L0 AS L_L0", "L1 AS L_L1", "ROWTIME AS L_ROWTIME", "ROWPARTITION AS L_ROWPARTITION", "ROWOFFSET AS L_ROWOFFSET", "ID AS L_ID" ] + }, + "rightSource" : { + "@type" : "streamSelectV1", + "properties" : { + "queryContext" : "PrependAliasRight" + }, + "source" : { + "@type" : "streamSelectKeyV2", + "properties" : { + "queryContext" : "RightSourceKeyed" + }, + "source" : { + "@type" : "streamSourceV1", + "properties" : { + "queryContext" : "KafkaTopic_Right/Source" + }, + "topicName" : "right_topic", + "formats" : { + "keyFormat" : { + "format" : "KAFKA", + "properties" : { } + }, + "valueFormat" : { + "format" : "PROTOBUF_NOSR", + "properties" : { } + } + }, + "timestampColumn" : null, + "sourceSchema" : "`ID` STRING KEY, `R0` INTEGER, `R1` INTEGER", + "pseudoColumnVersion" : 1 + }, + "keyExpression" : [ "R0" ] + }, + "keyColumnNames" : [ "R_R0" ], + "selectedKeys" : null, + "selectExpressions" : [ "R0 AS R_R0", "R1 AS R_R1", "ROWTIME AS R_ROWTIME", "ROWPARTITION AS R_ROWPARTITION", "ROWOFFSET AS R_ROWOFFSET", "ID AS R_ID" ] + }, + "beforeMillis" : 11.000000000, + "afterMillis" : 11.000000000, + "graceMillis" : null, + "keyColName" : "L_L0" + }, + "keyColumnNames" : [ "L0" ], + "selectedKeys" : null, + "selectExpressions" : [ "L_ID AS L_ID", "L_L1 AS L1", "R_R1 AS R1" ] + }, + "formats" : { + "keyFormat" : { + "format" : "KAFKA", + "properties" : { } + }, + "valueFormat" : { + "format" : "PROTOBUF_NOSR", + "properties" : { } + } + }, + "topicName" : "OUTPUT", + "timestampColumn" : null + }, + "queryId" : "CSAS_OUTPUT_0", + "runtimeId" : null + } + } ], + "configs" : { + "ksql.extension.dir" : "ext", + "ksql.streams.cache.max.bytes.buffering" : "0", + "metric.reporters" : "", + "ksql.query.status.running.threshold.seconds" : "300", + "ksql.connect.basic.auth.credentials.reload" : "false", + "ksql.output.topic.name.prefix" : "", + "ksql.query.pull.stream.enabled" : "true", + "ksql.query.push.v2.interpreter.enabled" : "true", + "ksql.queryanonymizer.logs_enabled" : "true", + "ksql.variable.substitution.enable" : "true", + "ksql.streams.shutdown.timeout.ms" : "300000", + "ksql.query.pull.max.allowed.offset.lag" : "9223372036854775807", + "ksql.query.pull.max.qps" : "2147483647", + "ksql.access.validator.enable" : "auto", + "ksql.streams.bootstrap.servers" : "localhost:0", + "ksql.query.pull.metrics.enabled" : "true", + "ksql.metrics.extension" : null, + "ksql.query.push.v2.alos.enabled" : "true", + "ksql.query.push.v2.max.hourly.bandwidth.megabytes" : "2147483647", + "ksql.query.pull.range.scan.enabled" : "true", + "ksql.transient.query.cleanup.service.initial.delay.seconds" : "600", + "ksql.hidden.topics" : "_confluent.*,__confluent.*,_schemas,__consumer_offsets,__transaction_state,connect-configs,connect-offsets,connect-status,connect-statuses", + "ksql.lambdas.enabled" : "true", + "ksql.source.table.materialization.enabled" : "true", + "ksql.query.pull.max.hourly.bandwidth.megabytes" : "2147483647", + "ksql.sink.window.change.log.additional.retention" : "1000000", + "ksql.query.persistent.active.limit" : "2147483647", + "ksql.persistence.wrap.single.values" : null, + "ksql.query.transient.max.bytes.buffering.total" : "-1", + "ksql.connect.basic.auth.credentials.source" : "NONE", + "ksql.schema.registry.url" : "schema_registry.url:0", + "ksql.properties.overrides.denylist" : "", + "ksql.service.id" : "some.ksql.service.id", + "ksql.query.push.v2.max.catchup.consumers" : "5", + "ksql.query.push.v2.enabled" : "false", + "ksql.transient.query.cleanup.service.enable" : "true", + "ksql.query.push.v2.metrics.enabled" : "true", + "ksql.rowpartition.rowoffset.enabled" : "true", + "ksql.streams.commit.interval.ms" : "2000", + "ksql.query.pull.table.scan.enabled" : "true", + "ksql.streams.auto.commit.interval.ms" : "0", + "ksql.streams.topology.optimization" : "all", + "ksql.endpoint.migrate.query" : "true", + "ksql.query.push.v2.registry.installed" : "false", + "ksql.streams.num.stream.threads" : "4", + "ksql.metrics.tags.custom" : "", + "ksql.query.push.v2.catchup.consumer.msg.window" : "50", + "ksql.runtime.feature.shared.enabled" : "false", + "ksql.udf.collect.metrics" : "false", + "ksql.new.query.planner.enabled" : "false", + "ksql.connect.request.headers.plugin" : null, + "ksql.security.extension.class" : null, + "ksql.transient.prefix" : "transient_", + "ksql.headers.columns.enabled" : "true", + "ksql.streams.default.deserialization.exception.handler" : "io.confluent.ksql.errors.LogMetricAndContinueExceptionHandler", + "ksql.connect.request.timeout.ms" : "5000", + "ksql.query.pull.enable.standby.reads" : "false", + "ksql.persistence.default.format.key" : "KAFKA", + "ksql.query.persistent.max.bytes.buffering.total" : "-1", + "ksql.query.error.max.queue.size" : "10", + "ksql.query.cleanup.shutdown.timeout.ms" : "30000", + "ksql.internal.topic.min.insync.replicas" : "1", + "ksql.internal.topic.replicas" : "1", + "ksql.insert.into.values.enabled" : "true", + "ksql.queryanonymizer.cluster_namespace" : null, + "ksql.create.or.replace.enabled" : "true", + "ksql.shared.runtimes.count" : "2", + "ksql.cast.strings.preserve.nulls" : "true", + "ksql.authorization.cache.max.entries" : "10000", + "ksql.pull.queries.enable" : "true", + "ksql.transient.query.cleanup.service.period.seconds" : "600", + "ksql.suppress.enabled" : "false", + "ksql.readonly.topics" : "_confluent.*,__confluent.*,_schemas,__consumer_offsets,__transaction_state,connect-configs,connect-offsets,connect-status,connect-statuses", + "ksql.connect.basic.auth.credentials.file" : "", + "ksql.authorization.cache.expiry.time.secs" : "30", + "ksql.query.retry.backoff.initial.ms" : "15000", + "ksql.query.pull.max.concurrent.requests" : "2147483647", + "ksql.streams.auto.offset.reset" : "earliest", + "ksql.connect.url" : "http://localhost:8083", + "ksql.query.push.v2.new.latest.delay.ms" : "5000", + "ksql.query.push.v2.latest.reset.age.ms" : "30000", + "ksql.streams.default.production.exception.handler" : "io.confluent.ksql.errors.ProductionExceptionHandlerUtil$LogAndFailProductionExceptionHandler", + "ksql.query.pull.interpreter.enabled" : "true", + "ksql.query.pull.limit.clause.enabled" : "true", + "ksql.query.pull.router.thread.pool.size" : "50", + "ksql.query.push.v2.continuation.tokens.enabled" : "false", + "ksql.query.retry.backoff.max.ms" : "900000", + "ksql.timestamp.throw.on.invalid" : "false", + "ksql.persistence.default.format.value" : null, + "ksql.udfs.enabled" : "true", + "ksql.udf.enable.security.manager" : "true", + "ksql.connect.worker.config" : "", + "ksql.nested.error.set.null" : "true", + "ksql.query.pull.thread.pool.size" : "50", + "ksql.persistent.prefix" : "query_", + "ksql.metastore.backup.location" : "", + "ksql.error.classifier.regex" : "", + "ksql.suppress.buffer.size.bytes" : "-1" + } +} \ No newline at end of file diff --git a/ksqldb-functional-tests/src/test/resources/historical_plans/joins_-_on_INT_column_-_KAFKA_-_PROTOBUF_NOSR/7.3.0_1651071422470/spec.json b/ksqldb-functional-tests/src/test/resources/historical_plans/joins_-_on_INT_column_-_KAFKA_-_PROTOBUF_NOSR/7.3.0_1651071422470/spec.json new file mode 100644 index 000000000000..2490782b9f7b --- /dev/null +++ b/ksqldb-functional-tests/src/test/resources/historical_plans/joins_-_on_INT_column_-_KAFKA_-_PROTOBUF_NOSR/7.3.0_1651071422470/spec.json @@ -0,0 +1,194 @@ +{ + "version" : "7.3.0", + "timestamp" : 1651071422470, + "path" : "query-validation-tests/joins.json", + "schemas" : { + "CSAS_OUTPUT_0.KafkaTopic_Right.Source" : { + "schema" : "`ID` STRING KEY, `R0` INTEGER, `R1` INTEGER", + "keyFormat" : { + "format" : "KAFKA" + }, + "valueFormat" : { + "format" : "PROTOBUF_NOSR" + } + }, + "CSAS_OUTPUT_0.Join.Left" : { + "schema" : "`L_L0` INTEGER KEY, `L_L0` INTEGER, `L_L1` INTEGER, `L_ROWTIME` BIGINT, `L_ROWPARTITION` INTEGER, `L_ROWOFFSET` BIGINT, `L_ID` STRING", + "valueFormat" : { + "format" : "PROTOBUF_NOSR" + } + }, + "CSAS_OUTPUT_0.OUTPUT" : { + "schema" : "`L0` INTEGER KEY, `L_ID` STRING, `L1` INTEGER, `R1` INTEGER", + "keyFormat" : { + "format" : "KAFKA" + }, + "valueFormat" : { + "format" : "PROTOBUF_NOSR" + } + }, + "CSAS_OUTPUT_0.KafkaTopic_Left.Source" : { + "schema" : "`ID` STRING KEY, `L0` INTEGER, `L1` INTEGER", + "keyFormat" : { + "format" : "KAFKA" + }, + "valueFormat" : { + "format" : "PROTOBUF_NOSR" + } + }, + "CSAS_OUTPUT_0.Join" : { + "schema" : "`L_L0` INTEGER KEY, `L_L0` INTEGER, `L_L1` INTEGER, `L_ROWTIME` BIGINT, `L_ROWPARTITION` INTEGER, `L_ROWOFFSET` BIGINT, `L_ID` STRING", + "keyFormat" : { + "format" : "KAFKA" + } + }, + "CSAS_OUTPUT_0.Join.Right" : { + "schema" : "`R_R0` INTEGER KEY, `R_R0` INTEGER, `R_R1` INTEGER, `R_ROWTIME` BIGINT, `R_ROWPARTITION` INTEGER, `R_ROWOFFSET` BIGINT, `R_ID` STRING", + "valueFormat" : { + "format" : "PROTOBUF_NOSR" + } + } + }, + "testCase" : { + "name" : "on INT column - KAFKA - PROTOBUF_NOSR", + "inputs" : [ { + "topic" : "left_topic", + "key" : "a", + "value" : { + "L0" : 10, + "L1" : 1 + }, + "timestamp" : 0 + }, { + "topic" : "right_topic", + "key" : "b", + "value" : { + "R0" : 10, + "R1" : 2 + }, + "timestamp" : 10000 + } ], + "outputs" : [ { + "topic" : "OUTPUT", + "key" : 10, + "value" : { + "L_ID" : "a", + "L1" : 1, + "R1" : 2 + }, + "timestamp" : 10000 + } ], + "topics" : [ { + "name" : "right_topic", + "replicas" : 1, + "numPartitions" : 4 + }, { + "name" : "OUTPUT", + "replicas" : 1, + "numPartitions" : 4 + }, { + "name" : "left_topic", + "replicas" : 1, + "numPartitions" : 4 + } ], + "statements" : [ "CREATE STREAM L (ID STRING KEY, l0 INT, l1 INT) WITH (kafka_topic='left_topic', value_format='PROTOBUF_NOSR');", "CREATE STREAM R (ID STRING KEY, r0 INT, r1 INT) WITH (kafka_topic='right_topic', value_format='PROTOBUF_NOSR');", "CREATE STREAM OUTPUT as SELECT L.l0, L.ID, L1, R1 FROM L join R WITHIN 11 SECONDS ON L.l0 = R.r0;" ], + "post" : { + "sources" : [ { + "name" : "L", + "type" : "STREAM", + "schema" : "`ID` STRING KEY, `L0` INTEGER, `L1` INTEGER", + "keyFormat" : { + "format" : "KAFKA" + }, + "valueFormat" : "PROTOBUF_NOSR", + "keyFeatures" : [ ], + "valueFeatures" : [ ], + "isSource" : false + }, { + "name" : "OUTPUT", + "type" : "STREAM", + "schema" : "`L0` INTEGER KEY, `L_ID` STRING, `L1` INTEGER, `R1` INTEGER", + "keyFormat" : { + "format" : "KAFKA" + }, + "valueFormat" : "PROTOBUF_NOSR", + "keyFeatures" : [ ], + "valueFeatures" : [ ], + "isSource" : false + }, { + "name" : "R", + "type" : "STREAM", + "schema" : "`ID` STRING KEY, `R0` INTEGER, `R1` INTEGER", + "keyFormat" : { + "format" : "KAFKA" + }, + "valueFormat" : "PROTOBUF_NOSR", + "keyFeatures" : [ ], + "valueFeatures" : [ ], + "isSource" : false + } ], + "topics" : { + "topics" : [ { + "name" : "_confluent-ksql-some.ksql.service.idquery_CSAS_OUTPUT_0-Join-left-repartition", + "keyFormat" : { + "format" : "KAFKA" + }, + "valueFormat" : { + "format" : "PROTOBUF_NOSR" + } + }, { + "name" : "_confluent-ksql-some.ksql.service.idquery_CSAS_OUTPUT_0-Join-right-repartition", + "keyFormat" : { + "format" : "KAFKA" + }, + "valueFormat" : { + "format" : "PROTOBUF_NOSR" + } + }, { + "name" : "_confluent-ksql-some.ksql.service.idquery_CSAS_OUTPUT_0-KSTREAM-JOINTHIS-0000000016-store-changelog", + "keyFormat" : { + "format" : "KAFKA" + }, + "valueFormat" : { + "format" : "PROTOBUF_NOSR" + } + }, { + "name" : "_confluent-ksql-some.ksql.service.idquery_CSAS_OUTPUT_0-KSTREAM-JOINOTHER-0000000017-store-changelog", + "keyFormat" : { + "format" : "KAFKA" + }, + "valueFormat" : { + "format" : "PROTOBUF_NOSR" + } + }, { + "name" : "left_topic", + "keyFormat" : { + "format" : "KAFKA" + }, + "valueFormat" : { + "format" : "PROTOBUF_NOSR" + }, + "partitions" : 4 + }, { + "name" : "OUTPUT", + "keyFormat" : { + "format" : "KAFKA" + }, + "valueFormat" : { + "format" : "PROTOBUF_NOSR" + }, + "partitions" : 4 + }, { + "name" : "right_topic", + "keyFormat" : { + "format" : "KAFKA" + }, + "valueFormat" : { + "format" : "PROTOBUF_NOSR" + }, + "partitions" : 4 + } ] + } + } + } +} \ No newline at end of file diff --git a/ksqldb-functional-tests/src/test/resources/historical_plans/joins_-_on_INT_column_-_KAFKA_-_PROTOBUF_NOSR/7.3.0_1651071422470/topology b/ksqldb-functional-tests/src/test/resources/historical_plans/joins_-_on_INT_column_-_KAFKA_-_PROTOBUF_NOSR/7.3.0_1651071422470/topology new file mode 100644 index 000000000000..e6d9bd120b1e --- /dev/null +++ b/ksqldb-functional-tests/src/test/resources/historical_plans/joins_-_on_INT_column_-_KAFKA_-_PROTOBUF_NOSR/7.3.0_1651071422470/topology @@ -0,0 +1,63 @@ +Topologies: + Sub-topology: 0 + Source: KSTREAM-SOURCE-0000000000 (topics: [left_topic]) + --> KSTREAM-TRANSFORMVALUES-0000000001 + Processor: KSTREAM-TRANSFORMVALUES-0000000001 (stores: []) + --> LeftSourceKeyed-SelectKey + <-- KSTREAM-SOURCE-0000000000 + Processor: LeftSourceKeyed-SelectKey (stores: []) + --> PrependAliasLeft + <-- KSTREAM-TRANSFORMVALUES-0000000001 + Processor: PrependAliasLeft (stores: []) + --> Join-left-repartition-filter + <-- LeftSourceKeyed-SelectKey + Processor: Join-left-repartition-filter (stores: []) + --> Join-left-repartition-sink + <-- PrependAliasLeft + Sink: Join-left-repartition-sink (topic: Join-left-repartition) + <-- Join-left-repartition-filter + + Sub-topology: 1 + Source: KSTREAM-SOURCE-0000000004 (topics: [right_topic]) + --> KSTREAM-TRANSFORMVALUES-0000000005 + Processor: KSTREAM-TRANSFORMVALUES-0000000005 (stores: []) + --> RightSourceKeyed-SelectKey + <-- KSTREAM-SOURCE-0000000004 + Processor: RightSourceKeyed-SelectKey (stores: []) + --> PrependAliasRight + <-- KSTREAM-TRANSFORMVALUES-0000000005 + Processor: PrependAliasRight (stores: []) + --> Join-right-repartition-filter + <-- RightSourceKeyed-SelectKey + Processor: Join-right-repartition-filter (stores: []) + --> Join-right-repartition-sink + <-- PrependAliasRight + Sink: Join-right-repartition-sink (topic: Join-right-repartition) + <-- Join-right-repartition-filter + + Sub-topology: 2 + Source: Join-left-repartition-source (topics: [Join-left-repartition]) + --> Join-this-windowed + Source: Join-right-repartition-source (topics: [Join-right-repartition]) + --> Join-other-windowed + Processor: Join-other-windowed (stores: [KSTREAM-JOINOTHER-0000000017-store]) + --> Join-other-join + <-- Join-right-repartition-source + Processor: Join-this-windowed (stores: [KSTREAM-JOINTHIS-0000000016-store]) + --> Join-this-join + <-- Join-left-repartition-source + Processor: Join-other-join (stores: [KSTREAM-JOINTHIS-0000000016-store]) + --> Join-merge + <-- Join-other-windowed + Processor: Join-this-join (stores: [KSTREAM-JOINOTHER-0000000017-store]) + --> Join-merge + <-- Join-this-windowed + Processor: Join-merge (stores: []) + --> Project + <-- Join-this-join, Join-other-join + Processor: Project (stores: []) + --> KSTREAM-SINK-0000000020 + <-- Join-merge + Sink: KSTREAM-SINK-0000000020 (topic: OUTPUT) + <-- Project + diff --git a/ksqldb-functional-tests/src/test/resources/historical_plans/joins_-_on_STRING_column_-_KAFKA_-_PROTOBUF_NOSR/7.3.0_1651071431375/plan.json b/ksqldb-functional-tests/src/test/resources/historical_plans/joins_-_on_STRING_column_-_KAFKA_-_PROTOBUF_NOSR/7.3.0_1651071431375/plan.json new file mode 100644 index 000000000000..085d4308cf22 --- /dev/null +++ b/ksqldb-functional-tests/src/test/resources/historical_plans/joins_-_on_STRING_column_-_KAFKA_-_PROTOBUF_NOSR/7.3.0_1651071431375/plan.json @@ -0,0 +1,310 @@ +{ + "plan" : [ { + "@type" : "ksqlPlanV1", + "statementText" : "CREATE STREAM L (ID STRING KEY, L0 STRING, L1 INTEGER) WITH (KAFKA_TOPIC='left_topic', KEY_FORMAT='KAFKA', VALUE_FORMAT='PROTOBUF_NOSR');", + "ddlCommand" : { + "@type" : "createStreamV1", + "sourceName" : "L", + "schema" : "`ID` STRING KEY, `L0` STRING, `L1` INTEGER", + "timestampColumn" : null, + "topicName" : "left_topic", + "formats" : { + "keyFormat" : { + "format" : "KAFKA", + "properties" : { } + }, + "valueFormat" : { + "format" : "PROTOBUF_NOSR", + "properties" : { } + } + }, + "windowInfo" : null, + "orReplace" : false, + "isSource" : false + }, + "queryPlan" : null + }, { + "@type" : "ksqlPlanV1", + "statementText" : "CREATE STREAM R (ID STRING KEY, R0 STRING, R1 INTEGER) WITH (KAFKA_TOPIC='right_topic', KEY_FORMAT='KAFKA', VALUE_FORMAT='PROTOBUF_NOSR');", + "ddlCommand" : { + "@type" : "createStreamV1", + "sourceName" : "R", + "schema" : "`ID` STRING KEY, `R0` STRING, `R1` INTEGER", + "timestampColumn" : null, + "topicName" : "right_topic", + "formats" : { + "keyFormat" : { + "format" : "KAFKA", + "properties" : { } + }, + "valueFormat" : { + "format" : "PROTOBUF_NOSR", + "properties" : { } + } + }, + "windowInfo" : null, + "orReplace" : false, + "isSource" : false + }, + "queryPlan" : null + }, { + "@type" : "ksqlPlanV1", + "statementText" : "CREATE STREAM OUTPUT AS SELECT\n L.L0 L0,\n L.ID L_ID,\n L.L1 L1,\n R.R1 R1\nFROM L L\nINNER JOIN R R WITHIN 11 SECONDS ON ((L.L0 = R.R0))\nEMIT CHANGES", + "ddlCommand" : { + "@type" : "createStreamV1", + "sourceName" : "OUTPUT", + "schema" : "`L0` STRING KEY, `L_ID` STRING, `L1` INTEGER, `R1` INTEGER", + "timestampColumn" : null, + "topicName" : "OUTPUT", + "formats" : { + "keyFormat" : { + "format" : "KAFKA", + "properties" : { } + }, + "valueFormat" : { + "format" : "PROTOBUF_NOSR", + "properties" : { } + } + }, + "windowInfo" : null, + "orReplace" : false, + "isSource" : false + }, + "queryPlan" : { + "sources" : [ "L", "R" ], + "sink" : "OUTPUT", + "physicalPlan" : { + "@type" : "streamSinkV1", + "properties" : { + "queryContext" : "OUTPUT" + }, + "source" : { + "@type" : "streamSelectV1", + "properties" : { + "queryContext" : "Project" + }, + "source" : { + "@type" : "streamStreamJoinV1", + "properties" : { + "queryContext" : "Join" + }, + "joinType" : "INNER", + "leftInternalFormats" : { + "keyFormat" : { + "format" : "KAFKA", + "properties" : { } + }, + "valueFormat" : { + "format" : "PROTOBUF_NOSR", + "properties" : { } + } + }, + "rightInternalFormats" : { + "keyFormat" : { + "format" : "KAFKA", + "properties" : { } + }, + "valueFormat" : { + "format" : "PROTOBUF_NOSR", + "properties" : { } + } + }, + "leftSource" : { + "@type" : "streamSelectV1", + "properties" : { + "queryContext" : "PrependAliasLeft" + }, + "source" : { + "@type" : "streamSelectKeyV2", + "properties" : { + "queryContext" : "LeftSourceKeyed" + }, + "source" : { + "@type" : "streamSourceV1", + "properties" : { + "queryContext" : "KafkaTopic_Left/Source" + }, + "topicName" : "left_topic", + "formats" : { + "keyFormat" : { + "format" : "KAFKA", + "properties" : { } + }, + "valueFormat" : { + "format" : "PROTOBUF_NOSR", + "properties" : { } + } + }, + "timestampColumn" : null, + "sourceSchema" : "`ID` STRING KEY, `L0` STRING, `L1` INTEGER", + "pseudoColumnVersion" : 1 + }, + "keyExpression" : [ "L0" ] + }, + "keyColumnNames" : [ "L_L0" ], + "selectedKeys" : null, + "selectExpressions" : [ "L0 AS L_L0", "L1 AS L_L1", "ROWTIME AS L_ROWTIME", "ROWPARTITION AS L_ROWPARTITION", "ROWOFFSET AS L_ROWOFFSET", "ID AS L_ID" ] + }, + "rightSource" : { + "@type" : "streamSelectV1", + "properties" : { + "queryContext" : "PrependAliasRight" + }, + "source" : { + "@type" : "streamSelectKeyV2", + "properties" : { + "queryContext" : "RightSourceKeyed" + }, + "source" : { + "@type" : "streamSourceV1", + "properties" : { + "queryContext" : "KafkaTopic_Right/Source" + }, + "topicName" : "right_topic", + "formats" : { + "keyFormat" : { + "format" : "KAFKA", + "properties" : { } + }, + "valueFormat" : { + "format" : "PROTOBUF_NOSR", + "properties" : { } + } + }, + "timestampColumn" : null, + "sourceSchema" : "`ID` STRING KEY, `R0` STRING, `R1` INTEGER", + "pseudoColumnVersion" : 1 + }, + "keyExpression" : [ "R0" ] + }, + "keyColumnNames" : [ "R_R0" ], + "selectedKeys" : null, + "selectExpressions" : [ "R0 AS R_R0", "R1 AS R_R1", "ROWTIME AS R_ROWTIME", "ROWPARTITION AS R_ROWPARTITION", "ROWOFFSET AS R_ROWOFFSET", "ID AS R_ID" ] + }, + "beforeMillis" : 11.000000000, + "afterMillis" : 11.000000000, + "graceMillis" : null, + "keyColName" : "L_L0" + }, + "keyColumnNames" : [ "L0" ], + "selectedKeys" : null, + "selectExpressions" : [ "L_ID AS L_ID", "L_L1 AS L1", "R_R1 AS R1" ] + }, + "formats" : { + "keyFormat" : { + "format" : "KAFKA", + "properties" : { } + }, + "valueFormat" : { + "format" : "PROTOBUF_NOSR", + "properties" : { } + } + }, + "topicName" : "OUTPUT", + "timestampColumn" : null + }, + "queryId" : "CSAS_OUTPUT_0", + "runtimeId" : null + } + } ], + "configs" : { + "ksql.extension.dir" : "ext", + "ksql.streams.cache.max.bytes.buffering" : "0", + "metric.reporters" : "", + "ksql.query.status.running.threshold.seconds" : "300", + "ksql.connect.basic.auth.credentials.reload" : "false", + "ksql.output.topic.name.prefix" : "", + "ksql.query.pull.stream.enabled" : "true", + "ksql.query.push.v2.interpreter.enabled" : "true", + "ksql.queryanonymizer.logs_enabled" : "true", + "ksql.variable.substitution.enable" : "true", + "ksql.streams.shutdown.timeout.ms" : "300000", + "ksql.query.pull.max.allowed.offset.lag" : "9223372036854775807", + "ksql.query.pull.max.qps" : "2147483647", + "ksql.access.validator.enable" : "auto", + "ksql.streams.bootstrap.servers" : "localhost:0", + "ksql.query.pull.metrics.enabled" : "true", + "ksql.metrics.extension" : null, + "ksql.query.push.v2.alos.enabled" : "true", + "ksql.query.push.v2.max.hourly.bandwidth.megabytes" : "2147483647", + "ksql.query.pull.range.scan.enabled" : "true", + "ksql.transient.query.cleanup.service.initial.delay.seconds" : "600", + "ksql.hidden.topics" : "_confluent.*,__confluent.*,_schemas,__consumer_offsets,__transaction_state,connect-configs,connect-offsets,connect-status,connect-statuses", + "ksql.lambdas.enabled" : "true", + "ksql.source.table.materialization.enabled" : "true", + "ksql.query.pull.max.hourly.bandwidth.megabytes" : "2147483647", + "ksql.sink.window.change.log.additional.retention" : "1000000", + "ksql.query.persistent.active.limit" : "2147483647", + "ksql.persistence.wrap.single.values" : null, + "ksql.query.transient.max.bytes.buffering.total" : "-1", + "ksql.connect.basic.auth.credentials.source" : "NONE", + "ksql.schema.registry.url" : "schema_registry.url:0", + "ksql.properties.overrides.denylist" : "", + "ksql.service.id" : "some.ksql.service.id", + "ksql.query.push.v2.max.catchup.consumers" : "5", + "ksql.query.push.v2.enabled" : "false", + "ksql.transient.query.cleanup.service.enable" : "true", + "ksql.query.push.v2.metrics.enabled" : "true", + "ksql.rowpartition.rowoffset.enabled" : "true", + "ksql.streams.commit.interval.ms" : "2000", + "ksql.query.pull.table.scan.enabled" : "true", + "ksql.streams.auto.commit.interval.ms" : "0", + "ksql.streams.topology.optimization" : "all", + "ksql.endpoint.migrate.query" : "true", + "ksql.query.push.v2.registry.installed" : "false", + "ksql.streams.num.stream.threads" : "4", + "ksql.metrics.tags.custom" : "", + "ksql.query.push.v2.catchup.consumer.msg.window" : "50", + "ksql.runtime.feature.shared.enabled" : "false", + "ksql.udf.collect.metrics" : "false", + "ksql.new.query.planner.enabled" : "false", + "ksql.connect.request.headers.plugin" : null, + "ksql.security.extension.class" : null, + "ksql.transient.prefix" : "transient_", + "ksql.headers.columns.enabled" : "true", + "ksql.streams.default.deserialization.exception.handler" : "io.confluent.ksql.errors.LogMetricAndContinueExceptionHandler", + "ksql.connect.request.timeout.ms" : "5000", + "ksql.query.pull.enable.standby.reads" : "false", + "ksql.persistence.default.format.key" : "KAFKA", + "ksql.query.persistent.max.bytes.buffering.total" : "-1", + "ksql.query.error.max.queue.size" : "10", + "ksql.query.cleanup.shutdown.timeout.ms" : "30000", + "ksql.internal.topic.min.insync.replicas" : "1", + "ksql.internal.topic.replicas" : "1", + "ksql.insert.into.values.enabled" : "true", + "ksql.queryanonymizer.cluster_namespace" : null, + "ksql.create.or.replace.enabled" : "true", + "ksql.shared.runtimes.count" : "2", + "ksql.cast.strings.preserve.nulls" : "true", + "ksql.authorization.cache.max.entries" : "10000", + "ksql.pull.queries.enable" : "true", + "ksql.transient.query.cleanup.service.period.seconds" : "600", + "ksql.suppress.enabled" : "false", + "ksql.readonly.topics" : "_confluent.*,__confluent.*,_schemas,__consumer_offsets,__transaction_state,connect-configs,connect-offsets,connect-status,connect-statuses", + "ksql.connect.basic.auth.credentials.file" : "", + "ksql.authorization.cache.expiry.time.secs" : "30", + "ksql.query.retry.backoff.initial.ms" : "15000", + "ksql.query.pull.max.concurrent.requests" : "2147483647", + "ksql.streams.auto.offset.reset" : "earliest", + "ksql.connect.url" : "http://localhost:8083", + "ksql.query.push.v2.new.latest.delay.ms" : "5000", + "ksql.query.push.v2.latest.reset.age.ms" : "30000", + "ksql.streams.default.production.exception.handler" : "io.confluent.ksql.errors.ProductionExceptionHandlerUtil$LogAndFailProductionExceptionHandler", + "ksql.query.pull.interpreter.enabled" : "true", + "ksql.query.pull.limit.clause.enabled" : "true", + "ksql.query.pull.router.thread.pool.size" : "50", + "ksql.query.push.v2.continuation.tokens.enabled" : "false", + "ksql.query.retry.backoff.max.ms" : "900000", + "ksql.timestamp.throw.on.invalid" : "false", + "ksql.persistence.default.format.value" : null, + "ksql.udfs.enabled" : "true", + "ksql.udf.enable.security.manager" : "true", + "ksql.connect.worker.config" : "", + "ksql.nested.error.set.null" : "true", + "ksql.query.pull.thread.pool.size" : "50", + "ksql.persistent.prefix" : "query_", + "ksql.metastore.backup.location" : "", + "ksql.error.classifier.regex" : "", + "ksql.suppress.buffer.size.bytes" : "-1" + } +} \ No newline at end of file diff --git a/ksqldb-functional-tests/src/test/resources/historical_plans/joins_-_on_STRING_column_-_KAFKA_-_PROTOBUF_NOSR/7.3.0_1651071431375/spec.json b/ksqldb-functional-tests/src/test/resources/historical_plans/joins_-_on_STRING_column_-_KAFKA_-_PROTOBUF_NOSR/7.3.0_1651071431375/spec.json new file mode 100644 index 000000000000..d819e4007e92 --- /dev/null +++ b/ksqldb-functional-tests/src/test/resources/historical_plans/joins_-_on_STRING_column_-_KAFKA_-_PROTOBUF_NOSR/7.3.0_1651071431375/spec.json @@ -0,0 +1,194 @@ +{ + "version" : "7.3.0", + "timestamp" : 1651071431375, + "path" : "query-validation-tests/joins.json", + "schemas" : { + "CSAS_OUTPUT_0.KafkaTopic_Right.Source" : { + "schema" : "`ID` STRING KEY, `R0` STRING, `R1` INTEGER", + "keyFormat" : { + "format" : "KAFKA" + }, + "valueFormat" : { + "format" : "PROTOBUF_NOSR" + } + }, + "CSAS_OUTPUT_0.Join.Left" : { + "schema" : "`L_L0` STRING KEY, `L_L0` STRING, `L_L1` INTEGER, `L_ROWTIME` BIGINT, `L_ROWPARTITION` INTEGER, `L_ROWOFFSET` BIGINT, `L_ID` STRING", + "valueFormat" : { + "format" : "PROTOBUF_NOSR" + } + }, + "CSAS_OUTPUT_0.OUTPUT" : { + "schema" : "`L0` STRING KEY, `L_ID` STRING, `L1` INTEGER, `R1` INTEGER", + "keyFormat" : { + "format" : "KAFKA" + }, + "valueFormat" : { + "format" : "PROTOBUF_NOSR" + } + }, + "CSAS_OUTPUT_0.KafkaTopic_Left.Source" : { + "schema" : "`ID` STRING KEY, `L0` STRING, `L1` INTEGER", + "keyFormat" : { + "format" : "KAFKA" + }, + "valueFormat" : { + "format" : "PROTOBUF_NOSR" + } + }, + "CSAS_OUTPUT_0.Join" : { + "schema" : "`L_L0` STRING KEY, `L_L0` STRING, `L_L1` INTEGER, `L_ROWTIME` BIGINT, `L_ROWPARTITION` INTEGER, `L_ROWOFFSET` BIGINT, `L_ID` STRING", + "keyFormat" : { + "format" : "KAFKA" + } + }, + "CSAS_OUTPUT_0.Join.Right" : { + "schema" : "`R_R0` STRING KEY, `R_R0` STRING, `R_R1` INTEGER, `R_ROWTIME` BIGINT, `R_ROWPARTITION` INTEGER, `R_ROWOFFSET` BIGINT, `R_ID` STRING", + "valueFormat" : { + "format" : "PROTOBUF_NOSR" + } + } + }, + "testCase" : { + "name" : "on STRING column - KAFKA - PROTOBUF_NOSR", + "inputs" : [ { + "topic" : "left_topic", + "key" : "a", + "value" : { + "L0" : "x", + "L1" : 1 + }, + "timestamp" : 0 + }, { + "topic" : "right_topic", + "key" : "b", + "value" : { + "R0" : "x", + "R1" : 2 + }, + "timestamp" : 10000 + } ], + "outputs" : [ { + "topic" : "OUTPUT", + "key" : "x", + "value" : { + "L_ID" : "a", + "L1" : 1, + "R1" : 2 + }, + "timestamp" : 10000 + } ], + "topics" : [ { + "name" : "right_topic", + "replicas" : 1, + "numPartitions" : 4 + }, { + "name" : "OUTPUT", + "replicas" : 1, + "numPartitions" : 4 + }, { + "name" : "left_topic", + "replicas" : 1, + "numPartitions" : 4 + } ], + "statements" : [ "CREATE STREAM L (ID STRING KEY, l0 STRING, l1 INT) WITH (kafka_topic='left_topic', value_format='PROTOBUF_NOSR');", "CREATE STREAM R (ID STRING KEY, r0 STRING, r1 INT) WITH (kafka_topic='right_topic', value_format='PROTOBUF_NOSR');", "CREATE STREAM OUTPUT as SELECT L.l0, L.ID, L1, R1 FROM L join R WITHIN 11 SECONDS ON L.l0 = R.r0;" ], + "post" : { + "sources" : [ { + "name" : "L", + "type" : "STREAM", + "schema" : "`ID` STRING KEY, `L0` STRING, `L1` INTEGER", + "keyFormat" : { + "format" : "KAFKA" + }, + "valueFormat" : "PROTOBUF_NOSR", + "keyFeatures" : [ ], + "valueFeatures" : [ ], + "isSource" : false + }, { + "name" : "OUTPUT", + "type" : "STREAM", + "schema" : "`L0` STRING KEY, `L_ID` STRING, `L1` INTEGER, `R1` INTEGER", + "keyFormat" : { + "format" : "KAFKA" + }, + "valueFormat" : "PROTOBUF_NOSR", + "keyFeatures" : [ ], + "valueFeatures" : [ ], + "isSource" : false + }, { + "name" : "R", + "type" : "STREAM", + "schema" : "`ID` STRING KEY, `R0` STRING, `R1` INTEGER", + "keyFormat" : { + "format" : "KAFKA" + }, + "valueFormat" : "PROTOBUF_NOSR", + "keyFeatures" : [ ], + "valueFeatures" : [ ], + "isSource" : false + } ], + "topics" : { + "topics" : [ { + "name" : "_confluent-ksql-some.ksql.service.idquery_CSAS_OUTPUT_0-Join-left-repartition", + "keyFormat" : { + "format" : "KAFKA" + }, + "valueFormat" : { + "format" : "PROTOBUF_NOSR" + } + }, { + "name" : "_confluent-ksql-some.ksql.service.idquery_CSAS_OUTPUT_0-Join-right-repartition", + "keyFormat" : { + "format" : "KAFKA" + }, + "valueFormat" : { + "format" : "PROTOBUF_NOSR" + } + }, { + "name" : "_confluent-ksql-some.ksql.service.idquery_CSAS_OUTPUT_0-KSTREAM-JOINTHIS-0000000016-store-changelog", + "keyFormat" : { + "format" : "KAFKA" + }, + "valueFormat" : { + "format" : "PROTOBUF_NOSR" + } + }, { + "name" : "_confluent-ksql-some.ksql.service.idquery_CSAS_OUTPUT_0-KSTREAM-JOINOTHER-0000000017-store-changelog", + "keyFormat" : { + "format" : "KAFKA" + }, + "valueFormat" : { + "format" : "PROTOBUF_NOSR" + } + }, { + "name" : "left_topic", + "keyFormat" : { + "format" : "KAFKA" + }, + "valueFormat" : { + "format" : "PROTOBUF_NOSR" + }, + "partitions" : 4 + }, { + "name" : "OUTPUT", + "keyFormat" : { + "format" : "KAFKA" + }, + "valueFormat" : { + "format" : "PROTOBUF_NOSR" + }, + "partitions" : 4 + }, { + "name" : "right_topic", + "keyFormat" : { + "format" : "KAFKA" + }, + "valueFormat" : { + "format" : "PROTOBUF_NOSR" + }, + "partitions" : 4 + } ] + } + } + } +} \ No newline at end of file diff --git a/ksqldb-functional-tests/src/test/resources/historical_plans/joins_-_on_STRING_column_-_KAFKA_-_PROTOBUF_NOSR/7.3.0_1651071431375/topology b/ksqldb-functional-tests/src/test/resources/historical_plans/joins_-_on_STRING_column_-_KAFKA_-_PROTOBUF_NOSR/7.3.0_1651071431375/topology new file mode 100644 index 000000000000..e6d9bd120b1e --- /dev/null +++ b/ksqldb-functional-tests/src/test/resources/historical_plans/joins_-_on_STRING_column_-_KAFKA_-_PROTOBUF_NOSR/7.3.0_1651071431375/topology @@ -0,0 +1,63 @@ +Topologies: + Sub-topology: 0 + Source: KSTREAM-SOURCE-0000000000 (topics: [left_topic]) + --> KSTREAM-TRANSFORMVALUES-0000000001 + Processor: KSTREAM-TRANSFORMVALUES-0000000001 (stores: []) + --> LeftSourceKeyed-SelectKey + <-- KSTREAM-SOURCE-0000000000 + Processor: LeftSourceKeyed-SelectKey (stores: []) + --> PrependAliasLeft + <-- KSTREAM-TRANSFORMVALUES-0000000001 + Processor: PrependAliasLeft (stores: []) + --> Join-left-repartition-filter + <-- LeftSourceKeyed-SelectKey + Processor: Join-left-repartition-filter (stores: []) + --> Join-left-repartition-sink + <-- PrependAliasLeft + Sink: Join-left-repartition-sink (topic: Join-left-repartition) + <-- Join-left-repartition-filter + + Sub-topology: 1 + Source: KSTREAM-SOURCE-0000000004 (topics: [right_topic]) + --> KSTREAM-TRANSFORMVALUES-0000000005 + Processor: KSTREAM-TRANSFORMVALUES-0000000005 (stores: []) + --> RightSourceKeyed-SelectKey + <-- KSTREAM-SOURCE-0000000004 + Processor: RightSourceKeyed-SelectKey (stores: []) + --> PrependAliasRight + <-- KSTREAM-TRANSFORMVALUES-0000000005 + Processor: PrependAliasRight (stores: []) + --> Join-right-repartition-filter + <-- RightSourceKeyed-SelectKey + Processor: Join-right-repartition-filter (stores: []) + --> Join-right-repartition-sink + <-- PrependAliasRight + Sink: Join-right-repartition-sink (topic: Join-right-repartition) + <-- Join-right-repartition-filter + + Sub-topology: 2 + Source: Join-left-repartition-source (topics: [Join-left-repartition]) + --> Join-this-windowed + Source: Join-right-repartition-source (topics: [Join-right-repartition]) + --> Join-other-windowed + Processor: Join-other-windowed (stores: [KSTREAM-JOINOTHER-0000000017-store]) + --> Join-other-join + <-- Join-right-repartition-source + Processor: Join-this-windowed (stores: [KSTREAM-JOINTHIS-0000000016-store]) + --> Join-this-join + <-- Join-left-repartition-source + Processor: Join-other-join (stores: [KSTREAM-JOINTHIS-0000000016-store]) + --> Join-merge + <-- Join-other-windowed + Processor: Join-this-join (stores: [KSTREAM-JOINOTHER-0000000017-store]) + --> Join-merge + <-- Join-this-windowed + Processor: Join-merge (stores: []) + --> Project + <-- Join-this-join, Join-other-join + Processor: Project (stores: []) + --> KSTREAM-SINK-0000000020 + <-- Join-merge + Sink: KSTREAM-SINK-0000000020 (topic: OUTPUT) + <-- Project + diff --git a/ksqldb-functional-tests/src/test/resources/historical_plans/joins_-_stream_stream_inner_join_-_PROTOBUF_NOSR/7.3.0_1651071304832/plan.json b/ksqldb-functional-tests/src/test/resources/historical_plans/joins_-_stream_stream_inner_join_-_PROTOBUF_NOSR/7.3.0_1651071304832/plan.json new file mode 100644 index 000000000000..07f198bb5277 --- /dev/null +++ b/ksqldb-functional-tests/src/test/resources/historical_plans/joins_-_stream_stream_inner_join_-_PROTOBUF_NOSR/7.3.0_1651071304832/plan.json @@ -0,0 +1,296 @@ +{ + "plan" : [ { + "@type" : "ksqlPlanV1", + "statementText" : "CREATE STREAM TEST (ID BIGINT KEY, NAME STRING, VALUE BIGINT) WITH (KAFKA_TOPIC='left_topic', KEY_FORMAT='KAFKA', VALUE_FORMAT='PROTOBUF_NOSR');", + "ddlCommand" : { + "@type" : "createStreamV1", + "sourceName" : "TEST", + "schema" : "`ID` BIGINT KEY, `NAME` STRING, `VALUE` BIGINT", + "timestampColumn" : null, + "topicName" : "left_topic", + "formats" : { + "keyFormat" : { + "format" : "KAFKA", + "properties" : { } + }, + "valueFormat" : { + "format" : "PROTOBUF_NOSR", + "properties" : { } + } + }, + "windowInfo" : null, + "orReplace" : false, + "isSource" : false + }, + "queryPlan" : null + }, { + "@type" : "ksqlPlanV1", + "statementText" : "CREATE STREAM TEST_STREAM (ID BIGINT KEY, F1 STRING, F2 BIGINT) WITH (KAFKA_TOPIC='right_topic', KEY_FORMAT='KAFKA', VALUE_FORMAT='PROTOBUF_NOSR');", + "ddlCommand" : { + "@type" : "createStreamV1", + "sourceName" : "TEST_STREAM", + "schema" : "`ID` BIGINT KEY, `F1` STRING, `F2` BIGINT", + "timestampColumn" : null, + "topicName" : "right_topic", + "formats" : { + "keyFormat" : { + "format" : "KAFKA", + "properties" : { } + }, + "valueFormat" : { + "format" : "PROTOBUF_NOSR", + "properties" : { } + } + }, + "windowInfo" : null, + "orReplace" : false, + "isSource" : false + }, + "queryPlan" : null + }, { + "@type" : "ksqlPlanV1", + "statementText" : "CREATE STREAM INNER_JOIN AS SELECT\n T.ID ID,\n T.NAME NAME,\n T.VALUE VALUE,\n TT.F1 F1,\n TT.F2 F2\nFROM TEST T\nINNER JOIN TEST_STREAM TT WITHIN 11 SECONDS ON ((T.ID = TT.ID))\nEMIT CHANGES", + "ddlCommand" : { + "@type" : "createStreamV1", + "sourceName" : "INNER_JOIN", + "schema" : "`ID` BIGINT KEY, `NAME` STRING, `VALUE` BIGINT, `F1` STRING, `F2` BIGINT", + "timestampColumn" : null, + "topicName" : "INNER_JOIN", + "formats" : { + "keyFormat" : { + "format" : "KAFKA", + "properties" : { } + }, + "valueFormat" : { + "format" : "PROTOBUF_NOSR", + "properties" : { } + } + }, + "windowInfo" : null, + "orReplace" : false, + "isSource" : false + }, + "queryPlan" : { + "sources" : [ "TEST", "TEST_STREAM" ], + "sink" : "INNER_JOIN", + "physicalPlan" : { + "@type" : "streamSinkV1", + "properties" : { + "queryContext" : "INNER_JOIN" + }, + "source" : { + "@type" : "streamSelectV1", + "properties" : { + "queryContext" : "Project" + }, + "source" : { + "@type" : "streamStreamJoinV1", + "properties" : { + "queryContext" : "Join" + }, + "joinType" : "INNER", + "leftInternalFormats" : { + "keyFormat" : { + "format" : "KAFKA", + "properties" : { } + }, + "valueFormat" : { + "format" : "PROTOBUF_NOSR", + "properties" : { } + } + }, + "rightInternalFormats" : { + "keyFormat" : { + "format" : "KAFKA", + "properties" : { } + }, + "valueFormat" : { + "format" : "PROTOBUF_NOSR", + "properties" : { } + } + }, + "leftSource" : { + "@type" : "streamSelectV1", + "properties" : { + "queryContext" : "PrependAliasLeft" + }, + "source" : { + "@type" : "streamSourceV1", + "properties" : { + "queryContext" : "KafkaTopic_Left/Source" + }, + "topicName" : "left_topic", + "formats" : { + "keyFormat" : { + "format" : "KAFKA", + "properties" : { } + }, + "valueFormat" : { + "format" : "PROTOBUF_NOSR", + "properties" : { } + } + }, + "timestampColumn" : null, + "sourceSchema" : "`ID` BIGINT KEY, `NAME` STRING, `VALUE` BIGINT", + "pseudoColumnVersion" : 1 + }, + "keyColumnNames" : [ "T_ID" ], + "selectedKeys" : null, + "selectExpressions" : [ "NAME AS T_NAME", "VALUE AS T_VALUE", "ROWTIME AS T_ROWTIME", "ROWPARTITION AS T_ROWPARTITION", "ROWOFFSET AS T_ROWOFFSET", "ID AS T_ID" ] + }, + "rightSource" : { + "@type" : "streamSelectV1", + "properties" : { + "queryContext" : "PrependAliasRight" + }, + "source" : { + "@type" : "streamSourceV1", + "properties" : { + "queryContext" : "KafkaTopic_Right/Source" + }, + "topicName" : "right_topic", + "formats" : { + "keyFormat" : { + "format" : "KAFKA", + "properties" : { } + }, + "valueFormat" : { + "format" : "PROTOBUF_NOSR", + "properties" : { } + } + }, + "timestampColumn" : null, + "sourceSchema" : "`ID` BIGINT KEY, `F1` STRING, `F2` BIGINT", + "pseudoColumnVersion" : 1 + }, + "keyColumnNames" : [ "TT_ID" ], + "selectedKeys" : null, + "selectExpressions" : [ "F1 AS TT_F1", "F2 AS TT_F2", "ROWTIME AS TT_ROWTIME", "ROWPARTITION AS TT_ROWPARTITION", "ROWOFFSET AS TT_ROWOFFSET", "ID AS TT_ID" ] + }, + "beforeMillis" : 11.000000000, + "afterMillis" : 11.000000000, + "graceMillis" : null, + "keyColName" : "T_ID" + }, + "keyColumnNames" : [ "ID" ], + "selectedKeys" : null, + "selectExpressions" : [ "T_NAME AS NAME", "T_VALUE AS VALUE", "TT_F1 AS F1", "TT_F2 AS F2" ] + }, + "formats" : { + "keyFormat" : { + "format" : "KAFKA", + "properties" : { } + }, + "valueFormat" : { + "format" : "PROTOBUF_NOSR", + "properties" : { } + } + }, + "topicName" : "INNER_JOIN", + "timestampColumn" : null + }, + "queryId" : "CSAS_INNER_JOIN_0", + "runtimeId" : null + } + } ], + "configs" : { + "ksql.extension.dir" : "ext", + "ksql.streams.cache.max.bytes.buffering" : "0", + "metric.reporters" : "", + "ksql.query.status.running.threshold.seconds" : "300", + "ksql.connect.basic.auth.credentials.reload" : "false", + "ksql.output.topic.name.prefix" : "", + "ksql.query.pull.stream.enabled" : "true", + "ksql.query.push.v2.interpreter.enabled" : "true", + "ksql.queryanonymizer.logs_enabled" : "true", + "ksql.variable.substitution.enable" : "true", + "ksql.streams.shutdown.timeout.ms" : "300000", + "ksql.query.pull.max.allowed.offset.lag" : "9223372036854775807", + "ksql.query.pull.max.qps" : "2147483647", + "ksql.access.validator.enable" : "auto", + "ksql.streams.bootstrap.servers" : "localhost:0", + "ksql.query.pull.metrics.enabled" : "true", + "ksql.metrics.extension" : null, + "ksql.query.push.v2.alos.enabled" : "true", + "ksql.query.push.v2.max.hourly.bandwidth.megabytes" : "2147483647", + "ksql.query.pull.range.scan.enabled" : "true", + "ksql.transient.query.cleanup.service.initial.delay.seconds" : "600", + "ksql.hidden.topics" : "_confluent.*,__confluent.*,_schemas,__consumer_offsets,__transaction_state,connect-configs,connect-offsets,connect-status,connect-statuses", + "ksql.lambdas.enabled" : "true", + "ksql.source.table.materialization.enabled" : "true", + "ksql.query.pull.max.hourly.bandwidth.megabytes" : "2147483647", + "ksql.sink.window.change.log.additional.retention" : "1000000", + "ksql.query.persistent.active.limit" : "2147483647", + "ksql.persistence.wrap.single.values" : null, + "ksql.query.transient.max.bytes.buffering.total" : "-1", + "ksql.connect.basic.auth.credentials.source" : "NONE", + "ksql.schema.registry.url" : "schema_registry.url:0", + "ksql.properties.overrides.denylist" : "", + "ksql.service.id" : "some.ksql.service.id", + "ksql.query.push.v2.max.catchup.consumers" : "5", + "ksql.query.push.v2.enabled" : "false", + "ksql.transient.query.cleanup.service.enable" : "true", + "ksql.query.push.v2.metrics.enabled" : "true", + "ksql.rowpartition.rowoffset.enabled" : "true", + "ksql.streams.commit.interval.ms" : "2000", + "ksql.query.pull.table.scan.enabled" : "true", + "ksql.streams.auto.commit.interval.ms" : "0", + "ksql.streams.topology.optimization" : "all", + "ksql.endpoint.migrate.query" : "true", + "ksql.query.push.v2.registry.installed" : "false", + "ksql.streams.num.stream.threads" : "4", + "ksql.metrics.tags.custom" : "", + "ksql.query.push.v2.catchup.consumer.msg.window" : "50", + "ksql.runtime.feature.shared.enabled" : "false", + "ksql.udf.collect.metrics" : "false", + "ksql.new.query.planner.enabled" : "false", + "ksql.connect.request.headers.plugin" : null, + "ksql.security.extension.class" : null, + "ksql.transient.prefix" : "transient_", + "ksql.headers.columns.enabled" : "true", + "ksql.streams.default.deserialization.exception.handler" : "io.confluent.ksql.errors.LogMetricAndContinueExceptionHandler", + "ksql.connect.request.timeout.ms" : "5000", + "ksql.query.pull.enable.standby.reads" : "false", + "ksql.persistence.default.format.key" : "KAFKA", + "ksql.query.persistent.max.bytes.buffering.total" : "-1", + "ksql.query.error.max.queue.size" : "10", + "ksql.query.cleanup.shutdown.timeout.ms" : "30000", + "ksql.internal.topic.min.insync.replicas" : "1", + "ksql.internal.topic.replicas" : "1", + "ksql.insert.into.values.enabled" : "true", + "ksql.queryanonymizer.cluster_namespace" : null, + "ksql.create.or.replace.enabled" : "true", + "ksql.shared.runtimes.count" : "2", + "ksql.cast.strings.preserve.nulls" : "true", + "ksql.authorization.cache.max.entries" : "10000", + "ksql.pull.queries.enable" : "true", + "ksql.transient.query.cleanup.service.period.seconds" : "600", + "ksql.suppress.enabled" : "false", + "ksql.readonly.topics" : "_confluent.*,__confluent.*,_schemas,__consumer_offsets,__transaction_state,connect-configs,connect-offsets,connect-status,connect-statuses", + "ksql.connect.basic.auth.credentials.file" : "", + "ksql.authorization.cache.expiry.time.secs" : "30", + "ksql.query.retry.backoff.initial.ms" : "15000", + "ksql.query.pull.max.concurrent.requests" : "2147483647", + "ksql.streams.auto.offset.reset" : "earliest", + "ksql.connect.url" : "http://localhost:8083", + "ksql.query.push.v2.new.latest.delay.ms" : "5000", + "ksql.query.push.v2.latest.reset.age.ms" : "30000", + "ksql.streams.default.production.exception.handler" : "io.confluent.ksql.errors.ProductionExceptionHandlerUtil$LogAndFailProductionExceptionHandler", + "ksql.query.pull.interpreter.enabled" : "true", + "ksql.query.pull.limit.clause.enabled" : "true", + "ksql.query.pull.router.thread.pool.size" : "50", + "ksql.query.push.v2.continuation.tokens.enabled" : "false", + "ksql.query.retry.backoff.max.ms" : "900000", + "ksql.timestamp.throw.on.invalid" : "false", + "ksql.persistence.default.format.value" : null, + "ksql.udfs.enabled" : "true", + "ksql.udf.enable.security.manager" : "true", + "ksql.connect.worker.config" : "", + "ksql.nested.error.set.null" : "true", + "ksql.query.pull.thread.pool.size" : "50", + "ksql.persistent.prefix" : "query_", + "ksql.metastore.backup.location" : "", + "ksql.error.classifier.regex" : "", + "ksql.suppress.buffer.size.bytes" : "-1" + } +} \ No newline at end of file diff --git a/ksqldb-functional-tests/src/test/resources/historical_plans/joins_-_stream_stream_inner_join_-_PROTOBUF_NOSR/7.3.0_1651071304832/spec.json b/ksqldb-functional-tests/src/test/resources/historical_plans/joins_-_stream_stream_inner_join_-_PROTOBUF_NOSR/7.3.0_1651071304832/spec.json new file mode 100644 index 000000000000..03a4e28aa01c --- /dev/null +++ b/ksqldb-functional-tests/src/test/resources/historical_plans/joins_-_stream_stream_inner_join_-_PROTOBUF_NOSR/7.3.0_1651071304832/spec.json @@ -0,0 +1,248 @@ +{ + "version" : "7.3.0", + "timestamp" : 1651071304832, + "path" : "query-validation-tests/joins.json", + "schemas" : { + "CSAS_INNER_JOIN_0.INNER_JOIN" : { + "schema" : "`ID` BIGINT KEY, `NAME` STRING, `VALUE` BIGINT, `F1` STRING, `F2` BIGINT", + "keyFormat" : { + "format" : "KAFKA" + }, + "valueFormat" : { + "format" : "PROTOBUF_NOSR" + } + }, + "CSAS_INNER_JOIN_0.Join.Left" : { + "schema" : "`T_ID` BIGINT KEY, `T_NAME` STRING, `T_VALUE` BIGINT, `T_ROWTIME` BIGINT, `T_ROWPARTITION` INTEGER, `T_ROWOFFSET` BIGINT, `T_ID` BIGINT", + "valueFormat" : { + "format" : "PROTOBUF_NOSR" + } + }, + "CSAS_INNER_JOIN_0.KafkaTopic_Right.Source" : { + "schema" : "`ID` BIGINT KEY, `F1` STRING, `F2` BIGINT", + "keyFormat" : { + "format" : "KAFKA" + }, + "valueFormat" : { + "format" : "PROTOBUF_NOSR" + } + }, + "CSAS_INNER_JOIN_0.Join" : { + "schema" : "`T_ID` BIGINT KEY, `T_NAME` STRING, `T_VALUE` BIGINT, `T_ROWTIME` BIGINT, `T_ROWPARTITION` INTEGER, `T_ROWOFFSET` BIGINT, `T_ID` BIGINT", + "keyFormat" : { + "format" : "KAFKA" + } + }, + "CSAS_INNER_JOIN_0.KafkaTopic_Left.Source" : { + "schema" : "`ID` BIGINT KEY, `NAME` STRING, `VALUE` BIGINT", + "keyFormat" : { + "format" : "KAFKA" + }, + "valueFormat" : { + "format" : "PROTOBUF_NOSR" + } + }, + "CSAS_INNER_JOIN_0.Join.Right" : { + "schema" : "`TT_ID` BIGINT KEY, `TT_F1` STRING, `TT_F2` BIGINT, `TT_ROWTIME` BIGINT, `TT_ROWPARTITION` INTEGER, `TT_ROWOFFSET` BIGINT, `TT_ID` BIGINT", + "valueFormat" : { + "format" : "PROTOBUF_NOSR" + } + } + }, + "testCase" : { + "name" : "stream stream inner join - PROTOBUF_NOSR", + "inputs" : [ { + "topic" : "left_topic", + "key" : 0, + "value" : { + "NAME" : "zero", + "VALUE" : 0 + }, + "timestamp" : 0 + }, { + "topic" : "right_topic", + "key" : 0, + "value" : { + "F1" : "blah", + "F2" : 50 + }, + "timestamp" : 10000 + }, { + "topic" : "left_topic", + "key" : 10, + "value" : { + "NAME" : "100", + "VALUE" : 5 + }, + "timestamp" : 11000 + }, { + "topic" : "left_topic", + "key" : 0, + "value" : { + "NAME" : "foo", + "VALUE" : 100 + }, + "timestamp" : 13000 + }, { + "topic" : "right_topic", + "key" : 0, + "value" : { + "F1" : "a", + "F2" : 10 + }, + "timestamp" : 15000 + }, { + "topic" : "right_topic", + "key" : 100, + "value" : { + "F1" : "newblah", + "F2" : 150 + }, + "timestamp" : 16000 + }, { + "topic" : "left_topic", + "key" : 90, + "value" : { + "NAME" : "ninety", + "VALUE" : 90 + }, + "timestamp" : 17000 + }, { + "topic" : "left_topic", + "key" : 0, + "value" : { + "NAME" : "bar", + "VALUE" : 99 + }, + "timestamp" : 30000 + } ], + "outputs" : [ { + "topic" : "INNER_JOIN", + "key" : 0, + "value" : { + "NAME" : "zero", + "VALUE" : 0, + "F1" : "blah", + "F2" : 50 + }, + "timestamp" : 10000 + }, { + "topic" : "INNER_JOIN", + "key" : 0, + "value" : { + "NAME" : "foo", + "VALUE" : 100, + "F1" : "blah", + "F2" : 50 + }, + "timestamp" : 13000 + }, { + "topic" : "INNER_JOIN", + "key" : 0, + "value" : { + "NAME" : "foo", + "VALUE" : 100, + "F1" : "a", + "F2" : 10 + }, + "timestamp" : 15000 + } ], + "topics" : [ { + "name" : "right_topic", + "replicas" : 1, + "numPartitions" : 4 + }, { + "name" : "INNER_JOIN", + "replicas" : 1, + "numPartitions" : 4 + }, { + "name" : "left_topic", + "replicas" : 1, + "numPartitions" : 4 + } ], + "statements" : [ "CREATE STREAM TEST (ID BIGINT KEY, NAME varchar, VALUE bigint) WITH (kafka_topic='left_topic', value_format='PROTOBUF_NOSR');", "CREATE STREAM TEST_STREAM (ID BIGINT KEY, F1 varchar, F2 bigint) WITH (kafka_topic='right_topic', value_format='PROTOBUF_NOSR');", "CREATE STREAM INNER_JOIN as SELECT t.id as ID, name, value, f1, f2 FROM test t join TEST_STREAM tt WITHIN 11 SECONDS ON t.id = tt.id;" ], + "post" : { + "sources" : [ { + "name" : "INNER_JOIN", + "type" : "STREAM", + "schema" : "`ID` BIGINT KEY, `NAME` STRING, `VALUE` BIGINT, `F1` STRING, `F2` BIGINT", + "keyFormat" : { + "format" : "KAFKA" + }, + "valueFormat" : "PROTOBUF_NOSR", + "keyFeatures" : [ ], + "valueFeatures" : [ ], + "isSource" : false + }, { + "name" : "TEST", + "type" : "STREAM", + "schema" : "`ID` BIGINT KEY, `NAME` STRING, `VALUE` BIGINT", + "keyFormat" : { + "format" : "KAFKA" + }, + "valueFormat" : "PROTOBUF_NOSR", + "keyFeatures" : [ ], + "valueFeatures" : [ ], + "isSource" : false + }, { + "name" : "TEST_STREAM", + "type" : "STREAM", + "schema" : "`ID` BIGINT KEY, `F1` STRING, `F2` BIGINT", + "keyFormat" : { + "format" : "KAFKA" + }, + "valueFormat" : "PROTOBUF_NOSR", + "keyFeatures" : [ ], + "valueFeatures" : [ ], + "isSource" : false + } ], + "topics" : { + "topics" : [ { + "name" : "_confluent-ksql-some.ksql.service.idquery_CSAS_INNER_JOIN_0-KSTREAM-JOINTHIS-0000000008-store-changelog", + "keyFormat" : { + "format" : "KAFKA" + }, + "valueFormat" : { + "format" : "PROTOBUF_NOSR" + } + }, { + "name" : "INNER_JOIN", + "keyFormat" : { + "format" : "KAFKA" + }, + "valueFormat" : { + "format" : "PROTOBUF_NOSR" + }, + "partitions" : 4 + }, { + "name" : "_confluent-ksql-some.ksql.service.idquery_CSAS_INNER_JOIN_0-KSTREAM-JOINOTHER-0000000009-store-changelog", + "keyFormat" : { + "format" : "KAFKA" + }, + "valueFormat" : { + "format" : "PROTOBUF_NOSR" + } + }, { + "name" : "left_topic", + "keyFormat" : { + "format" : "KAFKA" + }, + "valueFormat" : { + "format" : "PROTOBUF_NOSR" + }, + "partitions" : 4 + }, { + "name" : "right_topic", + "keyFormat" : { + "format" : "KAFKA" + }, + "valueFormat" : { + "format" : "PROTOBUF_NOSR" + }, + "partitions" : 4 + } ], + "blackList" : ".*-repartition" + } + } + } +} \ No newline at end of file diff --git a/ksqldb-functional-tests/src/test/resources/historical_plans/joins_-_stream_stream_inner_join_-_PROTOBUF_NOSR/7.3.0_1651071304832/topology b/ksqldb-functional-tests/src/test/resources/historical_plans/joins_-_stream_stream_inner_join_-_PROTOBUF_NOSR/7.3.0_1651071304832/topology new file mode 100644 index 000000000000..942a8e59aba7 --- /dev/null +++ b/ksqldb-functional-tests/src/test/resources/historical_plans/joins_-_stream_stream_inner_join_-_PROTOBUF_NOSR/7.3.0_1651071304832/topology @@ -0,0 +1,39 @@ +Topologies: + Sub-topology: 0 + Source: KSTREAM-SOURCE-0000000000 (topics: [left_topic]) + --> KSTREAM-TRANSFORMVALUES-0000000001 + Source: KSTREAM-SOURCE-0000000003 (topics: [right_topic]) + --> KSTREAM-TRANSFORMVALUES-0000000004 + Processor: KSTREAM-TRANSFORMVALUES-0000000001 (stores: []) + --> PrependAliasLeft + <-- KSTREAM-SOURCE-0000000000 + Processor: KSTREAM-TRANSFORMVALUES-0000000004 (stores: []) + --> PrependAliasRight + <-- KSTREAM-SOURCE-0000000003 + Processor: PrependAliasLeft (stores: []) + --> Join-this-windowed + <-- KSTREAM-TRANSFORMVALUES-0000000001 + Processor: PrependAliasRight (stores: []) + --> Join-other-windowed + <-- KSTREAM-TRANSFORMVALUES-0000000004 + Processor: Join-other-windowed (stores: [KSTREAM-JOINOTHER-0000000009-store]) + --> Join-other-join + <-- PrependAliasRight + Processor: Join-this-windowed (stores: [KSTREAM-JOINTHIS-0000000008-store]) + --> Join-this-join + <-- PrependAliasLeft + Processor: Join-other-join (stores: [KSTREAM-JOINTHIS-0000000008-store]) + --> Join-merge + <-- Join-other-windowed + Processor: Join-this-join (stores: [KSTREAM-JOINOTHER-0000000009-store]) + --> Join-merge + <-- Join-this-windowed + Processor: Join-merge (stores: []) + --> Project + <-- Join-this-join, Join-other-join + Processor: Project (stores: []) + --> KSTREAM-SINK-0000000012 + <-- Join-merge + Sink: KSTREAM-SINK-0000000012 (topic: INNER_JOIN) + <-- Project + diff --git a/ksqldb-functional-tests/src/test/resources/historical_plans/joins_-_stream_stream_inner_join_all_fields_-_PROTOBUF_NOSR/7.3.0_1651071319346/plan.json b/ksqldb-functional-tests/src/test/resources/historical_plans/joins_-_stream_stream_inner_join_all_fields_-_PROTOBUF_NOSR/7.3.0_1651071319346/plan.json new file mode 100644 index 000000000000..b043b1cdcf77 --- /dev/null +++ b/ksqldb-functional-tests/src/test/resources/historical_plans/joins_-_stream_stream_inner_join_all_fields_-_PROTOBUF_NOSR/7.3.0_1651071319346/plan.json @@ -0,0 +1,296 @@ +{ + "plan" : [ { + "@type" : "ksqlPlanV1", + "statementText" : "CREATE STREAM TEST (ID BIGINT KEY, NAME STRING) WITH (KAFKA_TOPIC='left_topic', KEY_FORMAT='KAFKA', VALUE_FORMAT='PROTOBUF_NOSR');", + "ddlCommand" : { + "@type" : "createStreamV1", + "sourceName" : "TEST", + "schema" : "`ID` BIGINT KEY, `NAME` STRING", + "timestampColumn" : null, + "topicName" : "left_topic", + "formats" : { + "keyFormat" : { + "format" : "KAFKA", + "properties" : { } + }, + "valueFormat" : { + "format" : "PROTOBUF_NOSR", + "properties" : { } + } + }, + "windowInfo" : null, + "orReplace" : false, + "isSource" : false + }, + "queryPlan" : null + }, { + "@type" : "ksqlPlanV1", + "statementText" : "CREATE STREAM TEST_STREAM (ID BIGINT KEY, F1 STRING) WITH (KAFKA_TOPIC='right_topic', KEY_FORMAT='KAFKA', VALUE_FORMAT='PROTOBUF_NOSR');", + "ddlCommand" : { + "@type" : "createStreamV1", + "sourceName" : "TEST_STREAM", + "schema" : "`ID` BIGINT KEY, `F1` STRING", + "timestampColumn" : null, + "topicName" : "right_topic", + "formats" : { + "keyFormat" : { + "format" : "KAFKA", + "properties" : { } + }, + "valueFormat" : { + "format" : "PROTOBUF_NOSR", + "properties" : { } + } + }, + "windowInfo" : null, + "orReplace" : false, + "isSource" : false + }, + "queryPlan" : null + }, { + "@type" : "ksqlPlanV1", + "statementText" : "CREATE STREAM INNER_JOIN AS SELECT *\nFROM TEST TT\nINNER JOIN TEST_STREAM T WITHIN 11 SECONDS ON ((T.ID = TT.ID))\nEMIT CHANGES", + "ddlCommand" : { + "@type" : "createStreamV1", + "sourceName" : "INNER_JOIN", + "schema" : "`TT_ID` BIGINT KEY, `TT_NAME` STRING, `T_ID` BIGINT, `T_F1` STRING", + "timestampColumn" : null, + "topicName" : "INNER_JOIN", + "formats" : { + "keyFormat" : { + "format" : "KAFKA", + "properties" : { } + }, + "valueFormat" : { + "format" : "PROTOBUF_NOSR", + "properties" : { } + } + }, + "windowInfo" : null, + "orReplace" : false, + "isSource" : false + }, + "queryPlan" : { + "sources" : [ "TEST", "TEST_STREAM" ], + "sink" : "INNER_JOIN", + "physicalPlan" : { + "@type" : "streamSinkV1", + "properties" : { + "queryContext" : "INNER_JOIN" + }, + "source" : { + "@type" : "streamSelectV1", + "properties" : { + "queryContext" : "Project" + }, + "source" : { + "@type" : "streamStreamJoinV1", + "properties" : { + "queryContext" : "Join" + }, + "joinType" : "INNER", + "leftInternalFormats" : { + "keyFormat" : { + "format" : "KAFKA", + "properties" : { } + }, + "valueFormat" : { + "format" : "PROTOBUF_NOSR", + "properties" : { } + } + }, + "rightInternalFormats" : { + "keyFormat" : { + "format" : "KAFKA", + "properties" : { } + }, + "valueFormat" : { + "format" : "PROTOBUF_NOSR", + "properties" : { } + } + }, + "leftSource" : { + "@type" : "streamSelectV1", + "properties" : { + "queryContext" : "PrependAliasLeft" + }, + "source" : { + "@type" : "streamSourceV1", + "properties" : { + "queryContext" : "KafkaTopic_Left/Source" + }, + "topicName" : "left_topic", + "formats" : { + "keyFormat" : { + "format" : "KAFKA", + "properties" : { } + }, + "valueFormat" : { + "format" : "PROTOBUF_NOSR", + "properties" : { } + } + }, + "timestampColumn" : null, + "sourceSchema" : "`ID` BIGINT KEY, `NAME` STRING", + "pseudoColumnVersion" : 1 + }, + "keyColumnNames" : [ "TT_ID" ], + "selectedKeys" : null, + "selectExpressions" : [ "NAME AS TT_NAME", "ROWTIME AS TT_ROWTIME", "ROWPARTITION AS TT_ROWPARTITION", "ROWOFFSET AS TT_ROWOFFSET", "ID AS TT_ID" ] + }, + "rightSource" : { + "@type" : "streamSelectV1", + "properties" : { + "queryContext" : "PrependAliasRight" + }, + "source" : { + "@type" : "streamSourceV1", + "properties" : { + "queryContext" : "KafkaTopic_Right/Source" + }, + "topicName" : "right_topic", + "formats" : { + "keyFormat" : { + "format" : "KAFKA", + "properties" : { } + }, + "valueFormat" : { + "format" : "PROTOBUF_NOSR", + "properties" : { } + } + }, + "timestampColumn" : null, + "sourceSchema" : "`ID` BIGINT KEY, `F1` STRING", + "pseudoColumnVersion" : 1 + }, + "keyColumnNames" : [ "T_ID" ], + "selectedKeys" : null, + "selectExpressions" : [ "F1 AS T_F1", "ROWTIME AS T_ROWTIME", "ROWPARTITION AS T_ROWPARTITION", "ROWOFFSET AS T_ROWOFFSET", "ID AS T_ID" ] + }, + "beforeMillis" : 11.000000000, + "afterMillis" : 11.000000000, + "graceMillis" : null, + "keyColName" : "TT_ID" + }, + "keyColumnNames" : [ "TT_ID" ], + "selectedKeys" : null, + "selectExpressions" : [ "TT_NAME AS TT_NAME", "T_ID AS T_ID", "T_F1 AS T_F1" ] + }, + "formats" : { + "keyFormat" : { + "format" : "KAFKA", + "properties" : { } + }, + "valueFormat" : { + "format" : "PROTOBUF_NOSR", + "properties" : { } + } + }, + "topicName" : "INNER_JOIN", + "timestampColumn" : null + }, + "queryId" : "CSAS_INNER_JOIN_0", + "runtimeId" : null + } + } ], + "configs" : { + "ksql.extension.dir" : "ext", + "ksql.streams.cache.max.bytes.buffering" : "0", + "metric.reporters" : "", + "ksql.query.status.running.threshold.seconds" : "300", + "ksql.connect.basic.auth.credentials.reload" : "false", + "ksql.output.topic.name.prefix" : "", + "ksql.query.pull.stream.enabled" : "true", + "ksql.query.push.v2.interpreter.enabled" : "true", + "ksql.queryanonymizer.logs_enabled" : "true", + "ksql.variable.substitution.enable" : "true", + "ksql.streams.shutdown.timeout.ms" : "300000", + "ksql.query.pull.max.allowed.offset.lag" : "9223372036854775807", + "ksql.query.pull.max.qps" : "2147483647", + "ksql.access.validator.enable" : "auto", + "ksql.streams.bootstrap.servers" : "localhost:0", + "ksql.query.pull.metrics.enabled" : "true", + "ksql.metrics.extension" : null, + "ksql.query.push.v2.alos.enabled" : "true", + "ksql.query.push.v2.max.hourly.bandwidth.megabytes" : "2147483647", + "ksql.query.pull.range.scan.enabled" : "true", + "ksql.transient.query.cleanup.service.initial.delay.seconds" : "600", + "ksql.hidden.topics" : "_confluent.*,__confluent.*,_schemas,__consumer_offsets,__transaction_state,connect-configs,connect-offsets,connect-status,connect-statuses", + "ksql.lambdas.enabled" : "true", + "ksql.source.table.materialization.enabled" : "true", + "ksql.query.pull.max.hourly.bandwidth.megabytes" : "2147483647", + "ksql.sink.window.change.log.additional.retention" : "1000000", + "ksql.query.persistent.active.limit" : "2147483647", + "ksql.persistence.wrap.single.values" : null, + "ksql.query.transient.max.bytes.buffering.total" : "-1", + "ksql.connect.basic.auth.credentials.source" : "NONE", + "ksql.schema.registry.url" : "schema_registry.url:0", + "ksql.properties.overrides.denylist" : "", + "ksql.service.id" : "some.ksql.service.id", + "ksql.query.push.v2.max.catchup.consumers" : "5", + "ksql.query.push.v2.enabled" : "false", + "ksql.transient.query.cleanup.service.enable" : "true", + "ksql.query.push.v2.metrics.enabled" : "true", + "ksql.rowpartition.rowoffset.enabled" : "true", + "ksql.streams.commit.interval.ms" : "2000", + "ksql.query.pull.table.scan.enabled" : "true", + "ksql.streams.auto.commit.interval.ms" : "0", + "ksql.streams.topology.optimization" : "all", + "ksql.endpoint.migrate.query" : "true", + "ksql.query.push.v2.registry.installed" : "false", + "ksql.streams.num.stream.threads" : "4", + "ksql.metrics.tags.custom" : "", + "ksql.query.push.v2.catchup.consumer.msg.window" : "50", + "ksql.runtime.feature.shared.enabled" : "false", + "ksql.udf.collect.metrics" : "false", + "ksql.new.query.planner.enabled" : "false", + "ksql.connect.request.headers.plugin" : null, + "ksql.security.extension.class" : null, + "ksql.transient.prefix" : "transient_", + "ksql.headers.columns.enabled" : "true", + "ksql.streams.default.deserialization.exception.handler" : "io.confluent.ksql.errors.LogMetricAndContinueExceptionHandler", + "ksql.connect.request.timeout.ms" : "5000", + "ksql.query.pull.enable.standby.reads" : "false", + "ksql.persistence.default.format.key" : "KAFKA", + "ksql.query.persistent.max.bytes.buffering.total" : "-1", + "ksql.query.error.max.queue.size" : "10", + "ksql.query.cleanup.shutdown.timeout.ms" : "30000", + "ksql.internal.topic.min.insync.replicas" : "1", + "ksql.internal.topic.replicas" : "1", + "ksql.insert.into.values.enabled" : "true", + "ksql.queryanonymizer.cluster_namespace" : null, + "ksql.create.or.replace.enabled" : "true", + "ksql.shared.runtimes.count" : "2", + "ksql.cast.strings.preserve.nulls" : "true", + "ksql.authorization.cache.max.entries" : "10000", + "ksql.pull.queries.enable" : "true", + "ksql.transient.query.cleanup.service.period.seconds" : "600", + "ksql.suppress.enabled" : "false", + "ksql.readonly.topics" : "_confluent.*,__confluent.*,_schemas,__consumer_offsets,__transaction_state,connect-configs,connect-offsets,connect-status,connect-statuses", + "ksql.connect.basic.auth.credentials.file" : "", + "ksql.authorization.cache.expiry.time.secs" : "30", + "ksql.query.retry.backoff.initial.ms" : "15000", + "ksql.query.pull.max.concurrent.requests" : "2147483647", + "ksql.streams.auto.offset.reset" : "earliest", + "ksql.connect.url" : "http://localhost:8083", + "ksql.query.push.v2.new.latest.delay.ms" : "5000", + "ksql.query.push.v2.latest.reset.age.ms" : "30000", + "ksql.streams.default.production.exception.handler" : "io.confluent.ksql.errors.ProductionExceptionHandlerUtil$LogAndFailProductionExceptionHandler", + "ksql.query.pull.interpreter.enabled" : "true", + "ksql.query.pull.limit.clause.enabled" : "true", + "ksql.query.pull.router.thread.pool.size" : "50", + "ksql.query.push.v2.continuation.tokens.enabled" : "false", + "ksql.query.retry.backoff.max.ms" : "900000", + "ksql.timestamp.throw.on.invalid" : "false", + "ksql.persistence.default.format.value" : null, + "ksql.udfs.enabled" : "true", + "ksql.udf.enable.security.manager" : "true", + "ksql.connect.worker.config" : "", + "ksql.nested.error.set.null" : "true", + "ksql.query.pull.thread.pool.size" : "50", + "ksql.persistent.prefix" : "query_", + "ksql.metastore.backup.location" : "", + "ksql.error.classifier.regex" : "", + "ksql.suppress.buffer.size.bytes" : "-1" + } +} \ No newline at end of file diff --git a/ksqldb-functional-tests/src/test/resources/historical_plans/joins_-_stream_stream_inner_join_all_fields_-_PROTOBUF_NOSR/7.3.0_1651071319346/spec.json b/ksqldb-functional-tests/src/test/resources/historical_plans/joins_-_stream_stream_inner_join_all_fields_-_PROTOBUF_NOSR/7.3.0_1651071319346/spec.json new file mode 100644 index 000000000000..09426c7aa9d0 --- /dev/null +++ b/ksqldb-functional-tests/src/test/resources/historical_plans/joins_-_stream_stream_inner_join_all_fields_-_PROTOBUF_NOSR/7.3.0_1651071319346/spec.json @@ -0,0 +1,236 @@ +{ + "version" : "7.3.0", + "timestamp" : 1651071319346, + "path" : "query-validation-tests/joins.json", + "schemas" : { + "CSAS_INNER_JOIN_0.INNER_JOIN" : { + "schema" : "`TT_ID` BIGINT KEY, `TT_NAME` STRING, `T_ID` BIGINT, `T_F1` STRING", + "keyFormat" : { + "format" : "KAFKA" + }, + "valueFormat" : { + "format" : "PROTOBUF_NOSR" + } + }, + "CSAS_INNER_JOIN_0.Join.Left" : { + "schema" : "`TT_ID` BIGINT KEY, `TT_NAME` STRING, `TT_ROWTIME` BIGINT, `TT_ROWPARTITION` INTEGER, `TT_ROWOFFSET` BIGINT, `TT_ID` BIGINT", + "valueFormat" : { + "format" : "PROTOBUF_NOSR" + } + }, + "CSAS_INNER_JOIN_0.KafkaTopic_Right.Source" : { + "schema" : "`ID` BIGINT KEY, `F1` STRING", + "keyFormat" : { + "format" : "KAFKA" + }, + "valueFormat" : { + "format" : "PROTOBUF_NOSR" + } + }, + "CSAS_INNER_JOIN_0.Join" : { + "schema" : "`TT_ID` BIGINT KEY, `TT_NAME` STRING, `TT_ROWTIME` BIGINT, `TT_ROWPARTITION` INTEGER, `TT_ROWOFFSET` BIGINT, `TT_ID` BIGINT", + "keyFormat" : { + "format" : "KAFKA" + } + }, + "CSAS_INNER_JOIN_0.KafkaTopic_Left.Source" : { + "schema" : "`ID` BIGINT KEY, `NAME` STRING", + "keyFormat" : { + "format" : "KAFKA" + }, + "valueFormat" : { + "format" : "PROTOBUF_NOSR" + } + }, + "CSAS_INNER_JOIN_0.Join.Right" : { + "schema" : "`T_ID` BIGINT KEY, `T_F1` STRING, `T_ROWTIME` BIGINT, `T_ROWPARTITION` INTEGER, `T_ROWOFFSET` BIGINT, `T_ID` BIGINT", + "valueFormat" : { + "format" : "PROTOBUF_NOSR" + } + } + }, + "testCase" : { + "name" : "stream stream inner join all fields - PROTOBUF_NOSR", + "inputs" : [ { + "topic" : "left_topic", + "key" : 0, + "value" : { + "NAME" : "zero" + }, + "timestamp" : 0 + }, { + "topic" : "right_topic", + "key" : 0, + "value" : { + "F1" : "blah" + }, + "timestamp" : 10000 + }, { + "topic" : "left_topic", + "key" : 10, + "value" : { + "NAME" : "100" + }, + "timestamp" : 11000 + }, { + "topic" : "left_topic", + "key" : 0, + "value" : { + "NAME" : "foo" + }, + "timestamp" : 13000 + }, { + "topic" : "right_topic", + "key" : 0, + "value" : { + "F1" : "a" + }, + "timestamp" : 15000 + }, { + "topic" : "right_topic", + "key" : 100, + "value" : { + "F1" : "newblah" + }, + "timestamp" : 16000 + }, { + "topic" : "left_topic", + "key" : 90, + "value" : { + "NAME" : "ninety" + }, + "timestamp" : 17000 + }, { + "topic" : "left_topic", + "key" : 0, + "value" : { + "NAME" : "bar" + }, + "timestamp" : 30000 + } ], + "outputs" : [ { + "topic" : "INNER_JOIN", + "key" : 0, + "value" : { + "T_ID" : 0, + "T_F1" : "blah", + "TT_NAME" : "zero" + }, + "timestamp" : 10000 + }, { + "topic" : "INNER_JOIN", + "key" : 0, + "value" : { + "T_ID" : 0, + "T_F1" : "blah", + "TT_NAME" : "foo" + }, + "timestamp" : 13000 + }, { + "topic" : "INNER_JOIN", + "key" : 0, + "value" : { + "T_ID" : 0, + "T_F1" : "a", + "TT_NAME" : "foo" + }, + "timestamp" : 15000 + } ], + "topics" : [ { + "name" : "right_topic", + "replicas" : 1, + "numPartitions" : 4 + }, { + "name" : "INNER_JOIN", + "replicas" : 1, + "numPartitions" : 4 + }, { + "name" : "left_topic", + "replicas" : 1, + "numPartitions" : 4 + } ], + "statements" : [ "CREATE STREAM TEST (ID BIGINT KEY, NAME varchar) WITH (kafka_topic='left_topic', value_format='PROTOBUF_NOSR');", "CREATE STREAM TEST_STREAM (ID BIGINT KEY, F1 varchar) WITH (kafka_topic='right_topic', value_format='PROTOBUF_NOSR');", "CREATE STREAM INNER_JOIN as SELECT * FROM test tt inner join TEST_STREAM t WITHIN 11 SECONDS ON t.id = tt.id;" ], + "post" : { + "sources" : [ { + "name" : "INNER_JOIN", + "type" : "STREAM", + "schema" : "`TT_ID` BIGINT KEY, `TT_NAME` STRING, `T_ID` BIGINT, `T_F1` STRING", + "keyFormat" : { + "format" : "KAFKA" + }, + "valueFormat" : "PROTOBUF_NOSR", + "keyFeatures" : [ ], + "valueFeatures" : [ ], + "isSource" : false + }, { + "name" : "TEST", + "type" : "STREAM", + "schema" : "`ID` BIGINT KEY, `NAME` STRING", + "keyFormat" : { + "format" : "KAFKA" + }, + "valueFormat" : "PROTOBUF_NOSR", + "keyFeatures" : [ ], + "valueFeatures" : [ ], + "isSource" : false + }, { + "name" : "TEST_STREAM", + "type" : "STREAM", + "schema" : "`ID` BIGINT KEY, `F1` STRING", + "keyFormat" : { + "format" : "KAFKA" + }, + "valueFormat" : "PROTOBUF_NOSR", + "keyFeatures" : [ ], + "valueFeatures" : [ ], + "isSource" : false + } ], + "topics" : { + "topics" : [ { + "name" : "_confluent-ksql-some.ksql.service.idquery_CSAS_INNER_JOIN_0-KSTREAM-JOINTHIS-0000000008-store-changelog", + "keyFormat" : { + "format" : "KAFKA" + }, + "valueFormat" : { + "format" : "PROTOBUF_NOSR" + } + }, { + "name" : "INNER_JOIN", + "keyFormat" : { + "format" : "KAFKA" + }, + "valueFormat" : { + "format" : "PROTOBUF_NOSR" + }, + "partitions" : 4 + }, { + "name" : "_confluent-ksql-some.ksql.service.idquery_CSAS_INNER_JOIN_0-KSTREAM-JOINOTHER-0000000009-store-changelog", + "keyFormat" : { + "format" : "KAFKA" + }, + "valueFormat" : { + "format" : "PROTOBUF_NOSR" + } + }, { + "name" : "left_topic", + "keyFormat" : { + "format" : "KAFKA" + }, + "valueFormat" : { + "format" : "PROTOBUF_NOSR" + }, + "partitions" : 4 + }, { + "name" : "right_topic", + "keyFormat" : { + "format" : "KAFKA" + }, + "valueFormat" : { + "format" : "PROTOBUF_NOSR" + }, + "partitions" : 4 + } ] + } + } + } +} \ No newline at end of file diff --git a/ksqldb-functional-tests/src/test/resources/historical_plans/joins_-_stream_stream_inner_join_all_fields_-_PROTOBUF_NOSR/7.3.0_1651071319346/topology b/ksqldb-functional-tests/src/test/resources/historical_plans/joins_-_stream_stream_inner_join_all_fields_-_PROTOBUF_NOSR/7.3.0_1651071319346/topology new file mode 100644 index 000000000000..942a8e59aba7 --- /dev/null +++ b/ksqldb-functional-tests/src/test/resources/historical_plans/joins_-_stream_stream_inner_join_all_fields_-_PROTOBUF_NOSR/7.3.0_1651071319346/topology @@ -0,0 +1,39 @@ +Topologies: + Sub-topology: 0 + Source: KSTREAM-SOURCE-0000000000 (topics: [left_topic]) + --> KSTREAM-TRANSFORMVALUES-0000000001 + Source: KSTREAM-SOURCE-0000000003 (topics: [right_topic]) + --> KSTREAM-TRANSFORMVALUES-0000000004 + Processor: KSTREAM-TRANSFORMVALUES-0000000001 (stores: []) + --> PrependAliasLeft + <-- KSTREAM-SOURCE-0000000000 + Processor: KSTREAM-TRANSFORMVALUES-0000000004 (stores: []) + --> PrependAliasRight + <-- KSTREAM-SOURCE-0000000003 + Processor: PrependAliasLeft (stores: []) + --> Join-this-windowed + <-- KSTREAM-TRANSFORMVALUES-0000000001 + Processor: PrependAliasRight (stores: []) + --> Join-other-windowed + <-- KSTREAM-TRANSFORMVALUES-0000000004 + Processor: Join-other-windowed (stores: [KSTREAM-JOINOTHER-0000000009-store]) + --> Join-other-join + <-- PrependAliasRight + Processor: Join-this-windowed (stores: [KSTREAM-JOINTHIS-0000000008-store]) + --> Join-this-join + <-- PrependAliasLeft + Processor: Join-other-join (stores: [KSTREAM-JOINTHIS-0000000008-store]) + --> Join-merge + <-- Join-other-windowed + Processor: Join-this-join (stores: [KSTREAM-JOINOTHER-0000000009-store]) + --> Join-merge + <-- Join-this-windowed + Processor: Join-merge (stores: []) + --> Project + <-- Join-this-join, Join-other-join + Processor: Project (stores: []) + --> KSTREAM-SINK-0000000012 + <-- Join-merge + Sink: KSTREAM-SINK-0000000012 (topic: INNER_JOIN) + <-- Project + diff --git a/ksqldb-functional-tests/src/test/resources/historical_plans/joins_-_stream_stream_inner_join_all_left_fields_some_right_-_PROTOBUF_NOSR/7.3.0_1651071309315/plan.json b/ksqldb-functional-tests/src/test/resources/historical_plans/joins_-_stream_stream_inner_join_all_left_fields_some_right_-_PROTOBUF_NOSR/7.3.0_1651071309315/plan.json new file mode 100644 index 000000000000..ff8bf3681cf8 --- /dev/null +++ b/ksqldb-functional-tests/src/test/resources/historical_plans/joins_-_stream_stream_inner_join_all_left_fields_some_right_-_PROTOBUF_NOSR/7.3.0_1651071309315/plan.json @@ -0,0 +1,296 @@ +{ + "plan" : [ { + "@type" : "ksqlPlanV1", + "statementText" : "CREATE STREAM TEST (ID BIGINT KEY, NAME STRING, VALUE BIGINT) WITH (KAFKA_TOPIC='left_topic', KEY_FORMAT='KAFKA', VALUE_FORMAT='PROTOBUF_NOSR');", + "ddlCommand" : { + "@type" : "createStreamV1", + "sourceName" : "TEST", + "schema" : "`ID` BIGINT KEY, `NAME` STRING, `VALUE` BIGINT", + "timestampColumn" : null, + "topicName" : "left_topic", + "formats" : { + "keyFormat" : { + "format" : "KAFKA", + "properties" : { } + }, + "valueFormat" : { + "format" : "PROTOBUF_NOSR", + "properties" : { } + } + }, + "windowInfo" : null, + "orReplace" : false, + "isSource" : false + }, + "queryPlan" : null + }, { + "@type" : "ksqlPlanV1", + "statementText" : "CREATE STREAM TEST_STREAM (ID BIGINT KEY, F1 STRING, F2 BIGINT) WITH (KAFKA_TOPIC='right_topic', KEY_FORMAT='KAFKA', VALUE_FORMAT='PROTOBUF_NOSR');", + "ddlCommand" : { + "@type" : "createStreamV1", + "sourceName" : "TEST_STREAM", + "schema" : "`ID` BIGINT KEY, `F1` STRING, `F2` BIGINT", + "timestampColumn" : null, + "topicName" : "right_topic", + "formats" : { + "keyFormat" : { + "format" : "KAFKA", + "properties" : { } + }, + "valueFormat" : { + "format" : "PROTOBUF_NOSR", + "properties" : { } + } + }, + "windowInfo" : null, + "orReplace" : false, + "isSource" : false + }, + "queryPlan" : null + }, { + "@type" : "ksqlPlanV1", + "statementText" : "CREATE STREAM INNER_JOIN AS SELECT\n T.*,\n TT.F1 F1\nFROM TEST T\nINNER JOIN TEST_STREAM TT WITHIN 11 SECONDS ON ((T.ID = TT.ID))\nEMIT CHANGES", + "ddlCommand" : { + "@type" : "createStreamV1", + "sourceName" : "INNER_JOIN", + "schema" : "`T_ID` BIGINT KEY, `T_NAME` STRING, `T_VALUE` BIGINT, `F1` STRING", + "timestampColumn" : null, + "topicName" : "INNER_JOIN", + "formats" : { + "keyFormat" : { + "format" : "KAFKA", + "properties" : { } + }, + "valueFormat" : { + "format" : "PROTOBUF_NOSR", + "properties" : { } + } + }, + "windowInfo" : null, + "orReplace" : false, + "isSource" : false + }, + "queryPlan" : { + "sources" : [ "TEST", "TEST_STREAM" ], + "sink" : "INNER_JOIN", + "physicalPlan" : { + "@type" : "streamSinkV1", + "properties" : { + "queryContext" : "INNER_JOIN" + }, + "source" : { + "@type" : "streamSelectV1", + "properties" : { + "queryContext" : "Project" + }, + "source" : { + "@type" : "streamStreamJoinV1", + "properties" : { + "queryContext" : "Join" + }, + "joinType" : "INNER", + "leftInternalFormats" : { + "keyFormat" : { + "format" : "KAFKA", + "properties" : { } + }, + "valueFormat" : { + "format" : "PROTOBUF_NOSR", + "properties" : { } + } + }, + "rightInternalFormats" : { + "keyFormat" : { + "format" : "KAFKA", + "properties" : { } + }, + "valueFormat" : { + "format" : "PROTOBUF_NOSR", + "properties" : { } + } + }, + "leftSource" : { + "@type" : "streamSelectV1", + "properties" : { + "queryContext" : "PrependAliasLeft" + }, + "source" : { + "@type" : "streamSourceV1", + "properties" : { + "queryContext" : "KafkaTopic_Left/Source" + }, + "topicName" : "left_topic", + "formats" : { + "keyFormat" : { + "format" : "KAFKA", + "properties" : { } + }, + "valueFormat" : { + "format" : "PROTOBUF_NOSR", + "properties" : { } + } + }, + "timestampColumn" : null, + "sourceSchema" : "`ID` BIGINT KEY, `NAME` STRING, `VALUE` BIGINT", + "pseudoColumnVersion" : 1 + }, + "keyColumnNames" : [ "T_ID" ], + "selectedKeys" : null, + "selectExpressions" : [ "NAME AS T_NAME", "VALUE AS T_VALUE", "ROWTIME AS T_ROWTIME", "ROWPARTITION AS T_ROWPARTITION", "ROWOFFSET AS T_ROWOFFSET", "ID AS T_ID" ] + }, + "rightSource" : { + "@type" : "streamSelectV1", + "properties" : { + "queryContext" : "PrependAliasRight" + }, + "source" : { + "@type" : "streamSourceV1", + "properties" : { + "queryContext" : "KafkaTopic_Right/Source" + }, + "topicName" : "right_topic", + "formats" : { + "keyFormat" : { + "format" : "KAFKA", + "properties" : { } + }, + "valueFormat" : { + "format" : "PROTOBUF_NOSR", + "properties" : { } + } + }, + "timestampColumn" : null, + "sourceSchema" : "`ID` BIGINT KEY, `F1` STRING, `F2` BIGINT", + "pseudoColumnVersion" : 1 + }, + "keyColumnNames" : [ "TT_ID" ], + "selectedKeys" : null, + "selectExpressions" : [ "F1 AS TT_F1", "F2 AS TT_F2", "ROWTIME AS TT_ROWTIME", "ROWPARTITION AS TT_ROWPARTITION", "ROWOFFSET AS TT_ROWOFFSET", "ID AS TT_ID" ] + }, + "beforeMillis" : 11.000000000, + "afterMillis" : 11.000000000, + "graceMillis" : null, + "keyColName" : "T_ID" + }, + "keyColumnNames" : [ "T_ID" ], + "selectedKeys" : null, + "selectExpressions" : [ "T_NAME AS T_NAME", "T_VALUE AS T_VALUE", "TT_F1 AS F1" ] + }, + "formats" : { + "keyFormat" : { + "format" : "KAFKA", + "properties" : { } + }, + "valueFormat" : { + "format" : "PROTOBUF_NOSR", + "properties" : { } + } + }, + "topicName" : "INNER_JOIN", + "timestampColumn" : null + }, + "queryId" : "CSAS_INNER_JOIN_0", + "runtimeId" : null + } + } ], + "configs" : { + "ksql.extension.dir" : "ext", + "ksql.streams.cache.max.bytes.buffering" : "0", + "metric.reporters" : "", + "ksql.query.status.running.threshold.seconds" : "300", + "ksql.connect.basic.auth.credentials.reload" : "false", + "ksql.output.topic.name.prefix" : "", + "ksql.query.pull.stream.enabled" : "true", + "ksql.query.push.v2.interpreter.enabled" : "true", + "ksql.queryanonymizer.logs_enabled" : "true", + "ksql.variable.substitution.enable" : "true", + "ksql.streams.shutdown.timeout.ms" : "300000", + "ksql.query.pull.max.allowed.offset.lag" : "9223372036854775807", + "ksql.query.pull.max.qps" : "2147483647", + "ksql.access.validator.enable" : "auto", + "ksql.streams.bootstrap.servers" : "localhost:0", + "ksql.query.pull.metrics.enabled" : "true", + "ksql.metrics.extension" : null, + "ksql.query.push.v2.alos.enabled" : "true", + "ksql.query.push.v2.max.hourly.bandwidth.megabytes" : "2147483647", + "ksql.query.pull.range.scan.enabled" : "true", + "ksql.transient.query.cleanup.service.initial.delay.seconds" : "600", + "ksql.hidden.topics" : "_confluent.*,__confluent.*,_schemas,__consumer_offsets,__transaction_state,connect-configs,connect-offsets,connect-status,connect-statuses", + "ksql.lambdas.enabled" : "true", + "ksql.source.table.materialization.enabled" : "true", + "ksql.query.pull.max.hourly.bandwidth.megabytes" : "2147483647", + "ksql.sink.window.change.log.additional.retention" : "1000000", + "ksql.query.persistent.active.limit" : "2147483647", + "ksql.persistence.wrap.single.values" : null, + "ksql.query.transient.max.bytes.buffering.total" : "-1", + "ksql.connect.basic.auth.credentials.source" : "NONE", + "ksql.schema.registry.url" : "schema_registry.url:0", + "ksql.properties.overrides.denylist" : "", + "ksql.service.id" : "some.ksql.service.id", + "ksql.query.push.v2.max.catchup.consumers" : "5", + "ksql.query.push.v2.enabled" : "false", + "ksql.transient.query.cleanup.service.enable" : "true", + "ksql.query.push.v2.metrics.enabled" : "true", + "ksql.rowpartition.rowoffset.enabled" : "true", + "ksql.streams.commit.interval.ms" : "2000", + "ksql.query.pull.table.scan.enabled" : "true", + "ksql.streams.auto.commit.interval.ms" : "0", + "ksql.streams.topology.optimization" : "all", + "ksql.endpoint.migrate.query" : "true", + "ksql.query.push.v2.registry.installed" : "false", + "ksql.streams.num.stream.threads" : "4", + "ksql.metrics.tags.custom" : "", + "ksql.query.push.v2.catchup.consumer.msg.window" : "50", + "ksql.runtime.feature.shared.enabled" : "false", + "ksql.udf.collect.metrics" : "false", + "ksql.new.query.planner.enabled" : "false", + "ksql.connect.request.headers.plugin" : null, + "ksql.security.extension.class" : null, + "ksql.transient.prefix" : "transient_", + "ksql.headers.columns.enabled" : "true", + "ksql.streams.default.deserialization.exception.handler" : "io.confluent.ksql.errors.LogMetricAndContinueExceptionHandler", + "ksql.connect.request.timeout.ms" : "5000", + "ksql.query.pull.enable.standby.reads" : "false", + "ksql.persistence.default.format.key" : "KAFKA", + "ksql.query.persistent.max.bytes.buffering.total" : "-1", + "ksql.query.error.max.queue.size" : "10", + "ksql.query.cleanup.shutdown.timeout.ms" : "30000", + "ksql.internal.topic.min.insync.replicas" : "1", + "ksql.internal.topic.replicas" : "1", + "ksql.insert.into.values.enabled" : "true", + "ksql.queryanonymizer.cluster_namespace" : null, + "ksql.create.or.replace.enabled" : "true", + "ksql.shared.runtimes.count" : "2", + "ksql.cast.strings.preserve.nulls" : "true", + "ksql.authorization.cache.max.entries" : "10000", + "ksql.pull.queries.enable" : "true", + "ksql.transient.query.cleanup.service.period.seconds" : "600", + "ksql.suppress.enabled" : "false", + "ksql.readonly.topics" : "_confluent.*,__confluent.*,_schemas,__consumer_offsets,__transaction_state,connect-configs,connect-offsets,connect-status,connect-statuses", + "ksql.connect.basic.auth.credentials.file" : "", + "ksql.authorization.cache.expiry.time.secs" : "30", + "ksql.query.retry.backoff.initial.ms" : "15000", + "ksql.query.pull.max.concurrent.requests" : "2147483647", + "ksql.streams.auto.offset.reset" : "earliest", + "ksql.connect.url" : "http://localhost:8083", + "ksql.query.push.v2.new.latest.delay.ms" : "5000", + "ksql.query.push.v2.latest.reset.age.ms" : "30000", + "ksql.streams.default.production.exception.handler" : "io.confluent.ksql.errors.ProductionExceptionHandlerUtil$LogAndFailProductionExceptionHandler", + "ksql.query.pull.interpreter.enabled" : "true", + "ksql.query.pull.limit.clause.enabled" : "true", + "ksql.query.pull.router.thread.pool.size" : "50", + "ksql.query.push.v2.continuation.tokens.enabled" : "false", + "ksql.query.retry.backoff.max.ms" : "900000", + "ksql.timestamp.throw.on.invalid" : "false", + "ksql.persistence.default.format.value" : null, + "ksql.udfs.enabled" : "true", + "ksql.udf.enable.security.manager" : "true", + "ksql.connect.worker.config" : "", + "ksql.nested.error.set.null" : "true", + "ksql.query.pull.thread.pool.size" : "50", + "ksql.persistent.prefix" : "query_", + "ksql.metastore.backup.location" : "", + "ksql.error.classifier.regex" : "", + "ksql.suppress.buffer.size.bytes" : "-1" + } +} \ No newline at end of file diff --git a/ksqldb-functional-tests/src/test/resources/historical_plans/joins_-_stream_stream_inner_join_all_left_fields_some_right_-_PROTOBUF_NOSR/7.3.0_1651071309315/spec.json b/ksqldb-functional-tests/src/test/resources/historical_plans/joins_-_stream_stream_inner_join_all_left_fields_some_right_-_PROTOBUF_NOSR/7.3.0_1651071309315/spec.json new file mode 100644 index 000000000000..d5a4278a9a37 --- /dev/null +++ b/ksqldb-functional-tests/src/test/resources/historical_plans/joins_-_stream_stream_inner_join_all_left_fields_some_right_-_PROTOBUF_NOSR/7.3.0_1651071309315/spec.json @@ -0,0 +1,244 @@ +{ + "version" : "7.3.0", + "timestamp" : 1651071309315, + "path" : "query-validation-tests/joins.json", + "schemas" : { + "CSAS_INNER_JOIN_0.INNER_JOIN" : { + "schema" : "`T_ID` BIGINT KEY, `T_NAME` STRING, `T_VALUE` BIGINT, `F1` STRING", + "keyFormat" : { + "format" : "KAFKA" + }, + "valueFormat" : { + "format" : "PROTOBUF_NOSR" + } + }, + "CSAS_INNER_JOIN_0.Join.Left" : { + "schema" : "`T_ID` BIGINT KEY, `T_NAME` STRING, `T_VALUE` BIGINT, `T_ROWTIME` BIGINT, `T_ROWPARTITION` INTEGER, `T_ROWOFFSET` BIGINT, `T_ID` BIGINT", + "valueFormat" : { + "format" : "PROTOBUF_NOSR" + } + }, + "CSAS_INNER_JOIN_0.KafkaTopic_Right.Source" : { + "schema" : "`ID` BIGINT KEY, `F1` STRING, `F2` BIGINT", + "keyFormat" : { + "format" : "KAFKA" + }, + "valueFormat" : { + "format" : "PROTOBUF_NOSR" + } + }, + "CSAS_INNER_JOIN_0.Join" : { + "schema" : "`T_ID` BIGINT KEY, `T_NAME` STRING, `T_VALUE` BIGINT, `T_ROWTIME` BIGINT, `T_ROWPARTITION` INTEGER, `T_ROWOFFSET` BIGINT, `T_ID` BIGINT", + "keyFormat" : { + "format" : "KAFKA" + } + }, + "CSAS_INNER_JOIN_0.KafkaTopic_Left.Source" : { + "schema" : "`ID` BIGINT KEY, `NAME` STRING, `VALUE` BIGINT", + "keyFormat" : { + "format" : "KAFKA" + }, + "valueFormat" : { + "format" : "PROTOBUF_NOSR" + } + }, + "CSAS_INNER_JOIN_0.Join.Right" : { + "schema" : "`TT_ID` BIGINT KEY, `TT_F1` STRING, `TT_F2` BIGINT, `TT_ROWTIME` BIGINT, `TT_ROWPARTITION` INTEGER, `TT_ROWOFFSET` BIGINT, `TT_ID` BIGINT", + "valueFormat" : { + "format" : "PROTOBUF_NOSR" + } + } + }, + "testCase" : { + "name" : "stream stream inner join all left fields some right - PROTOBUF_NOSR", + "inputs" : [ { + "topic" : "left_topic", + "key" : 0, + "value" : { + "NAME" : "zero", + "VALUE" : 0 + }, + "timestamp" : 0 + }, { + "topic" : "right_topic", + "key" : 0, + "value" : { + "F1" : "blah", + "F2" : 50 + }, + "timestamp" : 10000 + }, { + "topic" : "left_topic", + "key" : 10, + "value" : { + "NAME" : "100", + "VALUE" : 5 + }, + "timestamp" : 11000 + }, { + "topic" : "left_topic", + "key" : 0, + "value" : { + "NAME" : "foo", + "VALUE" : 100 + }, + "timestamp" : 13000 + }, { + "topic" : "right_topic", + "key" : 0, + "value" : { + "F1" : "a", + "F2" : 10 + }, + "timestamp" : 15000 + }, { + "topic" : "right_topic", + "key" : 100, + "value" : { + "F1" : "newblah", + "F2" : 150 + }, + "timestamp" : 16000 + }, { + "topic" : "left_topic", + "key" : 90, + "value" : { + "NAME" : "ninety", + "VALUE" : 90 + }, + "timestamp" : 17000 + }, { + "topic" : "left_topic", + "key" : 0, + "value" : { + "NAME" : "bar", + "VALUE" : 99 + }, + "timestamp" : 30000 + } ], + "outputs" : [ { + "topic" : "INNER_JOIN", + "key" : 0, + "value" : { + "T_NAME" : "zero", + "T_VALUE" : 0, + "F1" : "blah" + }, + "timestamp" : 10000 + }, { + "topic" : "INNER_JOIN", + "key" : 0, + "value" : { + "T_NAME" : "foo", + "T_VALUE" : 100, + "F1" : "blah" + }, + "timestamp" : 13000 + }, { + "topic" : "INNER_JOIN", + "key" : 0, + "value" : { + "T_NAME" : "foo", + "T_VALUE" : 100, + "F1" : "a" + }, + "timestamp" : 15000 + } ], + "topics" : [ { + "name" : "right_topic", + "replicas" : 1, + "numPartitions" : 4 + }, { + "name" : "INNER_JOIN", + "replicas" : 1, + "numPartitions" : 4 + }, { + "name" : "left_topic", + "replicas" : 1, + "numPartitions" : 4 + } ], + "statements" : [ "CREATE STREAM TEST (ID BIGINT KEY, NAME varchar, VALUE bigint) WITH (kafka_topic='left_topic', value_format='PROTOBUF_NOSR');", "CREATE STREAM TEST_STREAM (ID BIGINT KEY, F1 varchar, F2 bigint) WITH (kafka_topic='right_topic', value_format='PROTOBUF_NOSR');", "CREATE STREAM INNER_JOIN as SELECT t.*, tt.f1 FROM test t inner join TEST_STREAM tt WITHIN 11 SECONDS ON t.id = tt.id;" ], + "post" : { + "sources" : [ { + "name" : "INNER_JOIN", + "type" : "STREAM", + "schema" : "`T_ID` BIGINT KEY, `T_NAME` STRING, `T_VALUE` BIGINT, `F1` STRING", + "keyFormat" : { + "format" : "KAFKA" + }, + "valueFormat" : "PROTOBUF_NOSR", + "keyFeatures" : [ ], + "valueFeatures" : [ ], + "isSource" : false + }, { + "name" : "TEST", + "type" : "STREAM", + "schema" : "`ID` BIGINT KEY, `NAME` STRING, `VALUE` BIGINT", + "keyFormat" : { + "format" : "KAFKA" + }, + "valueFormat" : "PROTOBUF_NOSR", + "keyFeatures" : [ ], + "valueFeatures" : [ ], + "isSource" : false + }, { + "name" : "TEST_STREAM", + "type" : "STREAM", + "schema" : "`ID` BIGINT KEY, `F1` STRING, `F2` BIGINT", + "keyFormat" : { + "format" : "KAFKA" + }, + "valueFormat" : "PROTOBUF_NOSR", + "keyFeatures" : [ ], + "valueFeatures" : [ ], + "isSource" : false + } ], + "topics" : { + "topics" : [ { + "name" : "_confluent-ksql-some.ksql.service.idquery_CSAS_INNER_JOIN_0-KSTREAM-JOINTHIS-0000000008-store-changelog", + "keyFormat" : { + "format" : "KAFKA" + }, + "valueFormat" : { + "format" : "PROTOBUF_NOSR" + } + }, { + "name" : "INNER_JOIN", + "keyFormat" : { + "format" : "KAFKA" + }, + "valueFormat" : { + "format" : "PROTOBUF_NOSR" + }, + "partitions" : 4 + }, { + "name" : "_confluent-ksql-some.ksql.service.idquery_CSAS_INNER_JOIN_0-KSTREAM-JOINOTHER-0000000009-store-changelog", + "keyFormat" : { + "format" : "KAFKA" + }, + "valueFormat" : { + "format" : "PROTOBUF_NOSR" + } + }, { + "name" : "left_topic", + "keyFormat" : { + "format" : "KAFKA" + }, + "valueFormat" : { + "format" : "PROTOBUF_NOSR" + }, + "partitions" : 4 + }, { + "name" : "right_topic", + "keyFormat" : { + "format" : "KAFKA" + }, + "valueFormat" : { + "format" : "PROTOBUF_NOSR" + }, + "partitions" : 4 + } ] + } + } + } +} \ No newline at end of file diff --git a/ksqldb-functional-tests/src/test/resources/historical_plans/joins_-_stream_stream_inner_join_all_left_fields_some_right_-_PROTOBUF_NOSR/7.3.0_1651071309315/topology b/ksqldb-functional-tests/src/test/resources/historical_plans/joins_-_stream_stream_inner_join_all_left_fields_some_right_-_PROTOBUF_NOSR/7.3.0_1651071309315/topology new file mode 100644 index 000000000000..942a8e59aba7 --- /dev/null +++ b/ksqldb-functional-tests/src/test/resources/historical_plans/joins_-_stream_stream_inner_join_all_left_fields_some_right_-_PROTOBUF_NOSR/7.3.0_1651071309315/topology @@ -0,0 +1,39 @@ +Topologies: + Sub-topology: 0 + Source: KSTREAM-SOURCE-0000000000 (topics: [left_topic]) + --> KSTREAM-TRANSFORMVALUES-0000000001 + Source: KSTREAM-SOURCE-0000000003 (topics: [right_topic]) + --> KSTREAM-TRANSFORMVALUES-0000000004 + Processor: KSTREAM-TRANSFORMVALUES-0000000001 (stores: []) + --> PrependAliasLeft + <-- KSTREAM-SOURCE-0000000000 + Processor: KSTREAM-TRANSFORMVALUES-0000000004 (stores: []) + --> PrependAliasRight + <-- KSTREAM-SOURCE-0000000003 + Processor: PrependAliasLeft (stores: []) + --> Join-this-windowed + <-- KSTREAM-TRANSFORMVALUES-0000000001 + Processor: PrependAliasRight (stores: []) + --> Join-other-windowed + <-- KSTREAM-TRANSFORMVALUES-0000000004 + Processor: Join-other-windowed (stores: [KSTREAM-JOINOTHER-0000000009-store]) + --> Join-other-join + <-- PrependAliasRight + Processor: Join-this-windowed (stores: [KSTREAM-JOINTHIS-0000000008-store]) + --> Join-this-join + <-- PrependAliasLeft + Processor: Join-other-join (stores: [KSTREAM-JOINTHIS-0000000008-store]) + --> Join-merge + <-- Join-other-windowed + Processor: Join-this-join (stores: [KSTREAM-JOINOTHER-0000000009-store]) + --> Join-merge + <-- Join-this-windowed + Processor: Join-merge (stores: []) + --> Project + <-- Join-this-join, Join-other-join + Processor: Project (stores: []) + --> KSTREAM-SINK-0000000012 + <-- Join-merge + Sink: KSTREAM-SINK-0000000012 (topic: INNER_JOIN) + <-- Project + diff --git a/ksqldb-functional-tests/src/test/resources/historical_plans/joins_-_stream_stream_inner_join_all_right_fields_some_left_-_PROTOBUF_NOSR/7.3.0_1651071313869/plan.json b/ksqldb-functional-tests/src/test/resources/historical_plans/joins_-_stream_stream_inner_join_all_right_fields_some_left_-_PROTOBUF_NOSR/7.3.0_1651071313869/plan.json new file mode 100644 index 000000000000..b37eefcdd6fc --- /dev/null +++ b/ksqldb-functional-tests/src/test/resources/historical_plans/joins_-_stream_stream_inner_join_all_right_fields_some_left_-_PROTOBUF_NOSR/7.3.0_1651071313869/plan.json @@ -0,0 +1,296 @@ +{ + "plan" : [ { + "@type" : "ksqlPlanV1", + "statementText" : "CREATE STREAM TEST (ID BIGINT KEY, NAME STRING, VALUE BIGINT) WITH (KAFKA_TOPIC='left_topic', KEY_FORMAT='KAFKA', VALUE_FORMAT='PROTOBUF_NOSR');", + "ddlCommand" : { + "@type" : "createStreamV1", + "sourceName" : "TEST", + "schema" : "`ID` BIGINT KEY, `NAME` STRING, `VALUE` BIGINT", + "timestampColumn" : null, + "topicName" : "left_topic", + "formats" : { + "keyFormat" : { + "format" : "KAFKA", + "properties" : { } + }, + "valueFormat" : { + "format" : "PROTOBUF_NOSR", + "properties" : { } + } + }, + "windowInfo" : null, + "orReplace" : false, + "isSource" : false + }, + "queryPlan" : null + }, { + "@type" : "ksqlPlanV1", + "statementText" : "CREATE STREAM TEST_STREAM (ID BIGINT KEY, F1 STRING, F2 BIGINT) WITH (KAFKA_TOPIC='right_topic', KEY_FORMAT='KAFKA', VALUE_FORMAT='PROTOBUF_NOSR');", + "ddlCommand" : { + "@type" : "createStreamV1", + "sourceName" : "TEST_STREAM", + "schema" : "`ID` BIGINT KEY, `F1` STRING, `F2` BIGINT", + "timestampColumn" : null, + "topicName" : "right_topic", + "formats" : { + "keyFormat" : { + "format" : "KAFKA", + "properties" : { } + }, + "valueFormat" : { + "format" : "PROTOBUF_NOSR", + "properties" : { } + } + }, + "windowInfo" : null, + "orReplace" : false, + "isSource" : false + }, + "queryPlan" : null + }, { + "@type" : "ksqlPlanV1", + "statementText" : "CREATE STREAM INNER_JOIN AS SELECT\n T.*,\n TT.NAME NAME,\n TT.ID TT_ID\nFROM TEST TT\nINNER JOIN TEST_STREAM T WITHIN 11 SECONDS ON ((T.ID = TT.ID))\nEMIT CHANGES", + "ddlCommand" : { + "@type" : "createStreamV1", + "sourceName" : "INNER_JOIN", + "schema" : "`TT_ID` BIGINT KEY, `T_ID` BIGINT, `T_F1` STRING, `T_F2` BIGINT, `NAME` STRING", + "timestampColumn" : null, + "topicName" : "INNER_JOIN", + "formats" : { + "keyFormat" : { + "format" : "KAFKA", + "properties" : { } + }, + "valueFormat" : { + "format" : "PROTOBUF_NOSR", + "properties" : { } + } + }, + "windowInfo" : null, + "orReplace" : false, + "isSource" : false + }, + "queryPlan" : { + "sources" : [ "TEST", "TEST_STREAM" ], + "sink" : "INNER_JOIN", + "physicalPlan" : { + "@type" : "streamSinkV1", + "properties" : { + "queryContext" : "INNER_JOIN" + }, + "source" : { + "@type" : "streamSelectV1", + "properties" : { + "queryContext" : "Project" + }, + "source" : { + "@type" : "streamStreamJoinV1", + "properties" : { + "queryContext" : "Join" + }, + "joinType" : "INNER", + "leftInternalFormats" : { + "keyFormat" : { + "format" : "KAFKA", + "properties" : { } + }, + "valueFormat" : { + "format" : "PROTOBUF_NOSR", + "properties" : { } + } + }, + "rightInternalFormats" : { + "keyFormat" : { + "format" : "KAFKA", + "properties" : { } + }, + "valueFormat" : { + "format" : "PROTOBUF_NOSR", + "properties" : { } + } + }, + "leftSource" : { + "@type" : "streamSelectV1", + "properties" : { + "queryContext" : "PrependAliasLeft" + }, + "source" : { + "@type" : "streamSourceV1", + "properties" : { + "queryContext" : "KafkaTopic_Left/Source" + }, + "topicName" : "left_topic", + "formats" : { + "keyFormat" : { + "format" : "KAFKA", + "properties" : { } + }, + "valueFormat" : { + "format" : "PROTOBUF_NOSR", + "properties" : { } + } + }, + "timestampColumn" : null, + "sourceSchema" : "`ID` BIGINT KEY, `NAME` STRING, `VALUE` BIGINT", + "pseudoColumnVersion" : 1 + }, + "keyColumnNames" : [ "TT_ID" ], + "selectedKeys" : null, + "selectExpressions" : [ "NAME AS TT_NAME", "VALUE AS TT_VALUE", "ROWTIME AS TT_ROWTIME", "ROWPARTITION AS TT_ROWPARTITION", "ROWOFFSET AS TT_ROWOFFSET", "ID AS TT_ID" ] + }, + "rightSource" : { + "@type" : "streamSelectV1", + "properties" : { + "queryContext" : "PrependAliasRight" + }, + "source" : { + "@type" : "streamSourceV1", + "properties" : { + "queryContext" : "KafkaTopic_Right/Source" + }, + "topicName" : "right_topic", + "formats" : { + "keyFormat" : { + "format" : "KAFKA", + "properties" : { } + }, + "valueFormat" : { + "format" : "PROTOBUF_NOSR", + "properties" : { } + } + }, + "timestampColumn" : null, + "sourceSchema" : "`ID` BIGINT KEY, `F1` STRING, `F2` BIGINT", + "pseudoColumnVersion" : 1 + }, + "keyColumnNames" : [ "T_ID" ], + "selectedKeys" : null, + "selectExpressions" : [ "F1 AS T_F1", "F2 AS T_F2", "ROWTIME AS T_ROWTIME", "ROWPARTITION AS T_ROWPARTITION", "ROWOFFSET AS T_ROWOFFSET", "ID AS T_ID" ] + }, + "beforeMillis" : 11.000000000, + "afterMillis" : 11.000000000, + "graceMillis" : null, + "keyColName" : "TT_ID" + }, + "keyColumnNames" : [ "TT_ID" ], + "selectedKeys" : null, + "selectExpressions" : [ "T_ID AS T_ID", "T_F1 AS T_F1", "T_F2 AS T_F2", "TT_NAME AS NAME" ] + }, + "formats" : { + "keyFormat" : { + "format" : "KAFKA", + "properties" : { } + }, + "valueFormat" : { + "format" : "PROTOBUF_NOSR", + "properties" : { } + } + }, + "topicName" : "INNER_JOIN", + "timestampColumn" : null + }, + "queryId" : "CSAS_INNER_JOIN_0", + "runtimeId" : null + } + } ], + "configs" : { + "ksql.extension.dir" : "ext", + "ksql.streams.cache.max.bytes.buffering" : "0", + "metric.reporters" : "", + "ksql.query.status.running.threshold.seconds" : "300", + "ksql.connect.basic.auth.credentials.reload" : "false", + "ksql.output.topic.name.prefix" : "", + "ksql.query.pull.stream.enabled" : "true", + "ksql.query.push.v2.interpreter.enabled" : "true", + "ksql.queryanonymizer.logs_enabled" : "true", + "ksql.variable.substitution.enable" : "true", + "ksql.streams.shutdown.timeout.ms" : "300000", + "ksql.query.pull.max.allowed.offset.lag" : "9223372036854775807", + "ksql.query.pull.max.qps" : "2147483647", + "ksql.access.validator.enable" : "auto", + "ksql.streams.bootstrap.servers" : "localhost:0", + "ksql.query.pull.metrics.enabled" : "true", + "ksql.metrics.extension" : null, + "ksql.query.push.v2.alos.enabled" : "true", + "ksql.query.push.v2.max.hourly.bandwidth.megabytes" : "2147483647", + "ksql.query.pull.range.scan.enabled" : "true", + "ksql.transient.query.cleanup.service.initial.delay.seconds" : "600", + "ksql.hidden.topics" : "_confluent.*,__confluent.*,_schemas,__consumer_offsets,__transaction_state,connect-configs,connect-offsets,connect-status,connect-statuses", + "ksql.lambdas.enabled" : "true", + "ksql.source.table.materialization.enabled" : "true", + "ksql.query.pull.max.hourly.bandwidth.megabytes" : "2147483647", + "ksql.sink.window.change.log.additional.retention" : "1000000", + "ksql.query.persistent.active.limit" : "2147483647", + "ksql.persistence.wrap.single.values" : null, + "ksql.query.transient.max.bytes.buffering.total" : "-1", + "ksql.connect.basic.auth.credentials.source" : "NONE", + "ksql.schema.registry.url" : "schema_registry.url:0", + "ksql.properties.overrides.denylist" : "", + "ksql.service.id" : "some.ksql.service.id", + "ksql.query.push.v2.max.catchup.consumers" : "5", + "ksql.query.push.v2.enabled" : "false", + "ksql.transient.query.cleanup.service.enable" : "true", + "ksql.query.push.v2.metrics.enabled" : "true", + "ksql.rowpartition.rowoffset.enabled" : "true", + "ksql.streams.commit.interval.ms" : "2000", + "ksql.query.pull.table.scan.enabled" : "true", + "ksql.streams.auto.commit.interval.ms" : "0", + "ksql.streams.topology.optimization" : "all", + "ksql.endpoint.migrate.query" : "true", + "ksql.query.push.v2.registry.installed" : "false", + "ksql.streams.num.stream.threads" : "4", + "ksql.metrics.tags.custom" : "", + "ksql.query.push.v2.catchup.consumer.msg.window" : "50", + "ksql.runtime.feature.shared.enabled" : "false", + "ksql.udf.collect.metrics" : "false", + "ksql.new.query.planner.enabled" : "false", + "ksql.connect.request.headers.plugin" : null, + "ksql.security.extension.class" : null, + "ksql.transient.prefix" : "transient_", + "ksql.headers.columns.enabled" : "true", + "ksql.streams.default.deserialization.exception.handler" : "io.confluent.ksql.errors.LogMetricAndContinueExceptionHandler", + "ksql.connect.request.timeout.ms" : "5000", + "ksql.query.pull.enable.standby.reads" : "false", + "ksql.persistence.default.format.key" : "KAFKA", + "ksql.query.persistent.max.bytes.buffering.total" : "-1", + "ksql.query.error.max.queue.size" : "10", + "ksql.query.cleanup.shutdown.timeout.ms" : "30000", + "ksql.internal.topic.min.insync.replicas" : "1", + "ksql.internal.topic.replicas" : "1", + "ksql.insert.into.values.enabled" : "true", + "ksql.queryanonymizer.cluster_namespace" : null, + "ksql.create.or.replace.enabled" : "true", + "ksql.shared.runtimes.count" : "2", + "ksql.cast.strings.preserve.nulls" : "true", + "ksql.authorization.cache.max.entries" : "10000", + "ksql.pull.queries.enable" : "true", + "ksql.transient.query.cleanup.service.period.seconds" : "600", + "ksql.suppress.enabled" : "false", + "ksql.readonly.topics" : "_confluent.*,__confluent.*,_schemas,__consumer_offsets,__transaction_state,connect-configs,connect-offsets,connect-status,connect-statuses", + "ksql.connect.basic.auth.credentials.file" : "", + "ksql.authorization.cache.expiry.time.secs" : "30", + "ksql.query.retry.backoff.initial.ms" : "15000", + "ksql.query.pull.max.concurrent.requests" : "2147483647", + "ksql.streams.auto.offset.reset" : "earliest", + "ksql.connect.url" : "http://localhost:8083", + "ksql.query.push.v2.new.latest.delay.ms" : "5000", + "ksql.query.push.v2.latest.reset.age.ms" : "30000", + "ksql.streams.default.production.exception.handler" : "io.confluent.ksql.errors.ProductionExceptionHandlerUtil$LogAndFailProductionExceptionHandler", + "ksql.query.pull.interpreter.enabled" : "true", + "ksql.query.pull.limit.clause.enabled" : "true", + "ksql.query.pull.router.thread.pool.size" : "50", + "ksql.query.push.v2.continuation.tokens.enabled" : "false", + "ksql.query.retry.backoff.max.ms" : "900000", + "ksql.timestamp.throw.on.invalid" : "false", + "ksql.persistence.default.format.value" : null, + "ksql.udfs.enabled" : "true", + "ksql.udf.enable.security.manager" : "true", + "ksql.connect.worker.config" : "", + "ksql.nested.error.set.null" : "true", + "ksql.query.pull.thread.pool.size" : "50", + "ksql.persistent.prefix" : "query_", + "ksql.metastore.backup.location" : "", + "ksql.error.classifier.regex" : "", + "ksql.suppress.buffer.size.bytes" : "-1" + } +} \ No newline at end of file diff --git a/ksqldb-functional-tests/src/test/resources/historical_plans/joins_-_stream_stream_inner_join_all_right_fields_some_left_-_PROTOBUF_NOSR/7.3.0_1651071313869/spec.json b/ksqldb-functional-tests/src/test/resources/historical_plans/joins_-_stream_stream_inner_join_all_right_fields_some_left_-_PROTOBUF_NOSR/7.3.0_1651071313869/spec.json new file mode 100644 index 000000000000..4d28a02e9916 --- /dev/null +++ b/ksqldb-functional-tests/src/test/resources/historical_plans/joins_-_stream_stream_inner_join_all_right_fields_some_left_-_PROTOBUF_NOSR/7.3.0_1651071313869/spec.json @@ -0,0 +1,247 @@ +{ + "version" : "7.3.0", + "timestamp" : 1651071313869, + "path" : "query-validation-tests/joins.json", + "schemas" : { + "CSAS_INNER_JOIN_0.INNER_JOIN" : { + "schema" : "`TT_ID` BIGINT KEY, `T_ID` BIGINT, `T_F1` STRING, `T_F2` BIGINT, `NAME` STRING", + "keyFormat" : { + "format" : "KAFKA" + }, + "valueFormat" : { + "format" : "PROTOBUF_NOSR" + } + }, + "CSAS_INNER_JOIN_0.Join.Left" : { + "schema" : "`TT_ID` BIGINT KEY, `TT_NAME` STRING, `TT_VALUE` BIGINT, `TT_ROWTIME` BIGINT, `TT_ROWPARTITION` INTEGER, `TT_ROWOFFSET` BIGINT, `TT_ID` BIGINT", + "valueFormat" : { + "format" : "PROTOBUF_NOSR" + } + }, + "CSAS_INNER_JOIN_0.KafkaTopic_Right.Source" : { + "schema" : "`ID` BIGINT KEY, `F1` STRING, `F2` BIGINT", + "keyFormat" : { + "format" : "KAFKA" + }, + "valueFormat" : { + "format" : "PROTOBUF_NOSR" + } + }, + "CSAS_INNER_JOIN_0.Join" : { + "schema" : "`TT_ID` BIGINT KEY, `TT_NAME` STRING, `TT_VALUE` BIGINT, `TT_ROWTIME` BIGINT, `TT_ROWPARTITION` INTEGER, `TT_ROWOFFSET` BIGINT, `TT_ID` BIGINT", + "keyFormat" : { + "format" : "KAFKA" + } + }, + "CSAS_INNER_JOIN_0.KafkaTopic_Left.Source" : { + "schema" : "`ID` BIGINT KEY, `NAME` STRING, `VALUE` BIGINT", + "keyFormat" : { + "format" : "KAFKA" + }, + "valueFormat" : { + "format" : "PROTOBUF_NOSR" + } + }, + "CSAS_INNER_JOIN_0.Join.Right" : { + "schema" : "`T_ID` BIGINT KEY, `T_F1` STRING, `T_F2` BIGINT, `T_ROWTIME` BIGINT, `T_ROWPARTITION` INTEGER, `T_ROWOFFSET` BIGINT, `T_ID` BIGINT", + "valueFormat" : { + "format" : "PROTOBUF_NOSR" + } + } + }, + "testCase" : { + "name" : "stream stream inner join all right fields some left - PROTOBUF_NOSR", + "inputs" : [ { + "topic" : "left_topic", + "key" : 0, + "value" : { + "NAME" : "zero", + "VALUE" : 0 + }, + "timestamp" : 0 + }, { + "topic" : "right_topic", + "key" : 0, + "value" : { + "F1" : "blah", + "F2" : 50 + }, + "timestamp" : 10000 + }, { + "topic" : "left_topic", + "key" : 10, + "value" : { + "NAME" : "100", + "VALUE" : 5 + }, + "timestamp" : 11000 + }, { + "topic" : "left_topic", + "key" : 0, + "value" : { + "NAME" : "foo", + "VALUE" : 100 + }, + "timestamp" : 13000 + }, { + "topic" : "right_topic", + "key" : 0, + "value" : { + "F1" : "a", + "F2" : 10 + }, + "timestamp" : 15000 + }, { + "topic" : "right_topic", + "key" : 100, + "value" : { + "F1" : "newblah", + "F2" : 150 + }, + "timestamp" : 16000 + }, { + "topic" : "left_topic", + "key" : 90, + "value" : { + "NAME" : "ninety", + "VALUE" : 90 + }, + "timestamp" : 17000 + }, { + "topic" : "left_topic", + "key" : 0, + "value" : { + "NAME" : "bar", + "VALUE" : 99 + }, + "timestamp" : 30000 + } ], + "outputs" : [ { + "topic" : "INNER_JOIN", + "key" : 0, + "value" : { + "T_ID" : 0, + "T_F1" : "blah", + "T_F2" : 50, + "NAME" : "zero" + }, + "timestamp" : 10000 + }, { + "topic" : "INNER_JOIN", + "key" : 0, + "value" : { + "T_ID" : 0, + "T_F1" : "blah", + "T_F2" : 50, + "NAME" : "foo" + }, + "timestamp" : 13000 + }, { + "topic" : "INNER_JOIN", + "key" : 0, + "value" : { + "T_ID" : 0, + "T_F1" : "a", + "T_F2" : 10, + "NAME" : "foo" + }, + "timestamp" : 15000 + } ], + "topics" : [ { + "name" : "right_topic", + "replicas" : 1, + "numPartitions" : 4 + }, { + "name" : "INNER_JOIN", + "replicas" : 1, + "numPartitions" : 4 + }, { + "name" : "left_topic", + "replicas" : 1, + "numPartitions" : 4 + } ], + "statements" : [ "CREATE STREAM TEST (ID BIGINT KEY, NAME varchar, VALUE bigint) WITH (kafka_topic='left_topic', value_format='PROTOBUF_NOSR');", "CREATE STREAM TEST_STREAM (ID BIGINT KEY, F1 varchar, F2 bigint) WITH (kafka_topic='right_topic', value_format='PROTOBUF_NOSR');", "CREATE STREAM INNER_JOIN as SELECT t.*, tt.name, tt.id FROM test tt inner join TEST_STREAM t WITHIN 11 SECONDS ON t.id = tt.id;" ], + "post" : { + "sources" : [ { + "name" : "INNER_JOIN", + "type" : "STREAM", + "schema" : "`TT_ID` BIGINT KEY, `T_ID` BIGINT, `T_F1` STRING, `T_F2` BIGINT, `NAME` STRING", + "keyFormat" : { + "format" : "KAFKA" + }, + "valueFormat" : "PROTOBUF_NOSR", + "keyFeatures" : [ ], + "valueFeatures" : [ ], + "isSource" : false + }, { + "name" : "TEST", + "type" : "STREAM", + "schema" : "`ID` BIGINT KEY, `NAME` STRING, `VALUE` BIGINT", + "keyFormat" : { + "format" : "KAFKA" + }, + "valueFormat" : "PROTOBUF_NOSR", + "keyFeatures" : [ ], + "valueFeatures" : [ ], + "isSource" : false + }, { + "name" : "TEST_STREAM", + "type" : "STREAM", + "schema" : "`ID` BIGINT KEY, `F1` STRING, `F2` BIGINT", + "keyFormat" : { + "format" : "KAFKA" + }, + "valueFormat" : "PROTOBUF_NOSR", + "keyFeatures" : [ ], + "valueFeatures" : [ ], + "isSource" : false + } ], + "topics" : { + "topics" : [ { + "name" : "_confluent-ksql-some.ksql.service.idquery_CSAS_INNER_JOIN_0-KSTREAM-JOINTHIS-0000000008-store-changelog", + "keyFormat" : { + "format" : "KAFKA" + }, + "valueFormat" : { + "format" : "PROTOBUF_NOSR" + } + }, { + "name" : "INNER_JOIN", + "keyFormat" : { + "format" : "KAFKA" + }, + "valueFormat" : { + "format" : "PROTOBUF_NOSR" + }, + "partitions" : 4 + }, { + "name" : "_confluent-ksql-some.ksql.service.idquery_CSAS_INNER_JOIN_0-KSTREAM-JOINOTHER-0000000009-store-changelog", + "keyFormat" : { + "format" : "KAFKA" + }, + "valueFormat" : { + "format" : "PROTOBUF_NOSR" + } + }, { + "name" : "left_topic", + "keyFormat" : { + "format" : "KAFKA" + }, + "valueFormat" : { + "format" : "PROTOBUF_NOSR" + }, + "partitions" : 4 + }, { + "name" : "right_topic", + "keyFormat" : { + "format" : "KAFKA" + }, + "valueFormat" : { + "format" : "PROTOBUF_NOSR" + }, + "partitions" : 4 + } ] + } + } + } +} \ No newline at end of file diff --git a/ksqldb-functional-tests/src/test/resources/historical_plans/joins_-_stream_stream_inner_join_all_right_fields_some_left_-_PROTOBUF_NOSR/7.3.0_1651071313869/topology b/ksqldb-functional-tests/src/test/resources/historical_plans/joins_-_stream_stream_inner_join_all_right_fields_some_left_-_PROTOBUF_NOSR/7.3.0_1651071313869/topology new file mode 100644 index 000000000000..942a8e59aba7 --- /dev/null +++ b/ksqldb-functional-tests/src/test/resources/historical_plans/joins_-_stream_stream_inner_join_all_right_fields_some_left_-_PROTOBUF_NOSR/7.3.0_1651071313869/topology @@ -0,0 +1,39 @@ +Topologies: + Sub-topology: 0 + Source: KSTREAM-SOURCE-0000000000 (topics: [left_topic]) + --> KSTREAM-TRANSFORMVALUES-0000000001 + Source: KSTREAM-SOURCE-0000000003 (topics: [right_topic]) + --> KSTREAM-TRANSFORMVALUES-0000000004 + Processor: KSTREAM-TRANSFORMVALUES-0000000001 (stores: []) + --> PrependAliasLeft + <-- KSTREAM-SOURCE-0000000000 + Processor: KSTREAM-TRANSFORMVALUES-0000000004 (stores: []) + --> PrependAliasRight + <-- KSTREAM-SOURCE-0000000003 + Processor: PrependAliasLeft (stores: []) + --> Join-this-windowed + <-- KSTREAM-TRANSFORMVALUES-0000000001 + Processor: PrependAliasRight (stores: []) + --> Join-other-windowed + <-- KSTREAM-TRANSFORMVALUES-0000000004 + Processor: Join-other-windowed (stores: [KSTREAM-JOINOTHER-0000000009-store]) + --> Join-other-join + <-- PrependAliasRight + Processor: Join-this-windowed (stores: [KSTREAM-JOINTHIS-0000000008-store]) + --> Join-this-join + <-- PrependAliasLeft + Processor: Join-other-join (stores: [KSTREAM-JOINTHIS-0000000008-store]) + --> Join-merge + <-- Join-other-windowed + Processor: Join-this-join (stores: [KSTREAM-JOINOTHER-0000000009-store]) + --> Join-merge + <-- Join-this-windowed + Processor: Join-merge (stores: []) + --> Project + <-- Join-this-join, Join-other-join + Processor: Project (stores: []) + --> KSTREAM-SINK-0000000012 + <-- Join-merge + Sink: KSTREAM-SINK-0000000012 (topic: INNER_JOIN) + <-- Project + diff --git a/ksqldb-functional-tests/src/test/resources/historical_plans/joins_-_stream_stream_inner_join_with_different_before_and_after_windows_-_PROTOBUF_NOSR/7.3.0_1651071323747/plan.json b/ksqldb-functional-tests/src/test/resources/historical_plans/joins_-_stream_stream_inner_join_with_different_before_and_after_windows_-_PROTOBUF_NOSR/7.3.0_1651071323747/plan.json new file mode 100644 index 000000000000..dadc2912dad9 --- /dev/null +++ b/ksqldb-functional-tests/src/test/resources/historical_plans/joins_-_stream_stream_inner_join_with_different_before_and_after_windows_-_PROTOBUF_NOSR/7.3.0_1651071323747/plan.json @@ -0,0 +1,296 @@ +{ + "plan" : [ { + "@type" : "ksqlPlanV1", + "statementText" : "CREATE STREAM TEST (ID BIGINT KEY, NAME STRING, VALUE BIGINT) WITH (KAFKA_TOPIC='left_topic', KEY_FORMAT='KAFKA', VALUE_FORMAT='PROTOBUF_NOSR');", + "ddlCommand" : { + "@type" : "createStreamV1", + "sourceName" : "TEST", + "schema" : "`ID` BIGINT KEY, `NAME` STRING, `VALUE` BIGINT", + "timestampColumn" : null, + "topicName" : "left_topic", + "formats" : { + "keyFormat" : { + "format" : "KAFKA", + "properties" : { } + }, + "valueFormat" : { + "format" : "PROTOBUF_NOSR", + "properties" : { } + } + }, + "windowInfo" : null, + "orReplace" : false, + "isSource" : false + }, + "queryPlan" : null + }, { + "@type" : "ksqlPlanV1", + "statementText" : "CREATE STREAM TEST_STREAM (ID BIGINT KEY, F1 STRING, F2 BIGINT) WITH (KAFKA_TOPIC='right_topic', KEY_FORMAT='KAFKA', VALUE_FORMAT='PROTOBUF_NOSR');", + "ddlCommand" : { + "@type" : "createStreamV1", + "sourceName" : "TEST_STREAM", + "schema" : "`ID` BIGINT KEY, `F1` STRING, `F2` BIGINT", + "timestampColumn" : null, + "topicName" : "right_topic", + "formats" : { + "keyFormat" : { + "format" : "KAFKA", + "properties" : { } + }, + "valueFormat" : { + "format" : "PROTOBUF_NOSR", + "properties" : { } + } + }, + "windowInfo" : null, + "orReplace" : false, + "isSource" : false + }, + "queryPlan" : null + }, { + "@type" : "ksqlPlanV1", + "statementText" : "CREATE STREAM INNER_JOIN AS SELECT\n T.ID T_ID,\n T.NAME NAME,\n T.VALUE VALUE,\n TT.F1 F1,\n TT.F2 F2\nFROM TEST T\nINNER JOIN TEST_STREAM TT WITHIN (11 SECONDS, 10 SECONDS) ON ((T.ID = TT.ID))\nEMIT CHANGES", + "ddlCommand" : { + "@type" : "createStreamV1", + "sourceName" : "INNER_JOIN", + "schema" : "`T_ID` BIGINT KEY, `NAME` STRING, `VALUE` BIGINT, `F1` STRING, `F2` BIGINT", + "timestampColumn" : null, + "topicName" : "INNER_JOIN", + "formats" : { + "keyFormat" : { + "format" : "KAFKA", + "properties" : { } + }, + "valueFormat" : { + "format" : "PROTOBUF_NOSR", + "properties" : { } + } + }, + "windowInfo" : null, + "orReplace" : false, + "isSource" : false + }, + "queryPlan" : { + "sources" : [ "TEST", "TEST_STREAM" ], + "sink" : "INNER_JOIN", + "physicalPlan" : { + "@type" : "streamSinkV1", + "properties" : { + "queryContext" : "INNER_JOIN" + }, + "source" : { + "@type" : "streamSelectV1", + "properties" : { + "queryContext" : "Project" + }, + "source" : { + "@type" : "streamStreamJoinV1", + "properties" : { + "queryContext" : "Join" + }, + "joinType" : "INNER", + "leftInternalFormats" : { + "keyFormat" : { + "format" : "KAFKA", + "properties" : { } + }, + "valueFormat" : { + "format" : "PROTOBUF_NOSR", + "properties" : { } + } + }, + "rightInternalFormats" : { + "keyFormat" : { + "format" : "KAFKA", + "properties" : { } + }, + "valueFormat" : { + "format" : "PROTOBUF_NOSR", + "properties" : { } + } + }, + "leftSource" : { + "@type" : "streamSelectV1", + "properties" : { + "queryContext" : "PrependAliasLeft" + }, + "source" : { + "@type" : "streamSourceV1", + "properties" : { + "queryContext" : "KafkaTopic_Left/Source" + }, + "topicName" : "left_topic", + "formats" : { + "keyFormat" : { + "format" : "KAFKA", + "properties" : { } + }, + "valueFormat" : { + "format" : "PROTOBUF_NOSR", + "properties" : { } + } + }, + "timestampColumn" : null, + "sourceSchema" : "`ID` BIGINT KEY, `NAME` STRING, `VALUE` BIGINT", + "pseudoColumnVersion" : 1 + }, + "keyColumnNames" : [ "T_ID" ], + "selectedKeys" : null, + "selectExpressions" : [ "NAME AS T_NAME", "VALUE AS T_VALUE", "ROWTIME AS T_ROWTIME", "ROWPARTITION AS T_ROWPARTITION", "ROWOFFSET AS T_ROWOFFSET", "ID AS T_ID" ] + }, + "rightSource" : { + "@type" : "streamSelectV1", + "properties" : { + "queryContext" : "PrependAliasRight" + }, + "source" : { + "@type" : "streamSourceV1", + "properties" : { + "queryContext" : "KafkaTopic_Right/Source" + }, + "topicName" : "right_topic", + "formats" : { + "keyFormat" : { + "format" : "KAFKA", + "properties" : { } + }, + "valueFormat" : { + "format" : "PROTOBUF_NOSR", + "properties" : { } + } + }, + "timestampColumn" : null, + "sourceSchema" : "`ID` BIGINT KEY, `F1` STRING, `F2` BIGINT", + "pseudoColumnVersion" : 1 + }, + "keyColumnNames" : [ "TT_ID" ], + "selectedKeys" : null, + "selectExpressions" : [ "F1 AS TT_F1", "F2 AS TT_F2", "ROWTIME AS TT_ROWTIME", "ROWPARTITION AS TT_ROWPARTITION", "ROWOFFSET AS TT_ROWOFFSET", "ID AS TT_ID" ] + }, + "beforeMillis" : 11.000000000, + "afterMillis" : 10.000000000, + "graceMillis" : null, + "keyColName" : "T_ID" + }, + "keyColumnNames" : [ "T_ID" ], + "selectedKeys" : null, + "selectExpressions" : [ "T_NAME AS NAME", "T_VALUE AS VALUE", "TT_F1 AS F1", "TT_F2 AS F2" ] + }, + "formats" : { + "keyFormat" : { + "format" : "KAFKA", + "properties" : { } + }, + "valueFormat" : { + "format" : "PROTOBUF_NOSR", + "properties" : { } + } + }, + "topicName" : "INNER_JOIN", + "timestampColumn" : null + }, + "queryId" : "CSAS_INNER_JOIN_0", + "runtimeId" : null + } + } ], + "configs" : { + "ksql.extension.dir" : "ext", + "ksql.streams.cache.max.bytes.buffering" : "0", + "metric.reporters" : "", + "ksql.query.status.running.threshold.seconds" : "300", + "ksql.connect.basic.auth.credentials.reload" : "false", + "ksql.output.topic.name.prefix" : "", + "ksql.query.pull.stream.enabled" : "true", + "ksql.query.push.v2.interpreter.enabled" : "true", + "ksql.queryanonymizer.logs_enabled" : "true", + "ksql.variable.substitution.enable" : "true", + "ksql.streams.shutdown.timeout.ms" : "300000", + "ksql.query.pull.max.allowed.offset.lag" : "9223372036854775807", + "ksql.query.pull.max.qps" : "2147483647", + "ksql.access.validator.enable" : "auto", + "ksql.streams.bootstrap.servers" : "localhost:0", + "ksql.query.pull.metrics.enabled" : "true", + "ksql.metrics.extension" : null, + "ksql.query.push.v2.alos.enabled" : "true", + "ksql.query.push.v2.max.hourly.bandwidth.megabytes" : "2147483647", + "ksql.query.pull.range.scan.enabled" : "true", + "ksql.transient.query.cleanup.service.initial.delay.seconds" : "600", + "ksql.hidden.topics" : "_confluent.*,__confluent.*,_schemas,__consumer_offsets,__transaction_state,connect-configs,connect-offsets,connect-status,connect-statuses", + "ksql.lambdas.enabled" : "true", + "ksql.source.table.materialization.enabled" : "true", + "ksql.query.pull.max.hourly.bandwidth.megabytes" : "2147483647", + "ksql.sink.window.change.log.additional.retention" : "1000000", + "ksql.query.persistent.active.limit" : "2147483647", + "ksql.persistence.wrap.single.values" : null, + "ksql.query.transient.max.bytes.buffering.total" : "-1", + "ksql.connect.basic.auth.credentials.source" : "NONE", + "ksql.schema.registry.url" : "schema_registry.url:0", + "ksql.properties.overrides.denylist" : "", + "ksql.service.id" : "some.ksql.service.id", + "ksql.query.push.v2.max.catchup.consumers" : "5", + "ksql.query.push.v2.enabled" : "false", + "ksql.transient.query.cleanup.service.enable" : "true", + "ksql.query.push.v2.metrics.enabled" : "true", + "ksql.rowpartition.rowoffset.enabled" : "true", + "ksql.streams.commit.interval.ms" : "2000", + "ksql.query.pull.table.scan.enabled" : "true", + "ksql.streams.auto.commit.interval.ms" : "0", + "ksql.streams.topology.optimization" : "all", + "ksql.endpoint.migrate.query" : "true", + "ksql.query.push.v2.registry.installed" : "false", + "ksql.streams.num.stream.threads" : "4", + "ksql.metrics.tags.custom" : "", + "ksql.query.push.v2.catchup.consumer.msg.window" : "50", + "ksql.runtime.feature.shared.enabled" : "false", + "ksql.udf.collect.metrics" : "false", + "ksql.new.query.planner.enabled" : "false", + "ksql.connect.request.headers.plugin" : null, + "ksql.security.extension.class" : null, + "ksql.transient.prefix" : "transient_", + "ksql.headers.columns.enabled" : "true", + "ksql.streams.default.deserialization.exception.handler" : "io.confluent.ksql.errors.LogMetricAndContinueExceptionHandler", + "ksql.connect.request.timeout.ms" : "5000", + "ksql.query.pull.enable.standby.reads" : "false", + "ksql.persistence.default.format.key" : "KAFKA", + "ksql.query.persistent.max.bytes.buffering.total" : "-1", + "ksql.query.error.max.queue.size" : "10", + "ksql.query.cleanup.shutdown.timeout.ms" : "30000", + "ksql.internal.topic.min.insync.replicas" : "1", + "ksql.internal.topic.replicas" : "1", + "ksql.insert.into.values.enabled" : "true", + "ksql.queryanonymizer.cluster_namespace" : null, + "ksql.create.or.replace.enabled" : "true", + "ksql.shared.runtimes.count" : "2", + "ksql.cast.strings.preserve.nulls" : "true", + "ksql.authorization.cache.max.entries" : "10000", + "ksql.pull.queries.enable" : "true", + "ksql.transient.query.cleanup.service.period.seconds" : "600", + "ksql.suppress.enabled" : "false", + "ksql.readonly.topics" : "_confluent.*,__confluent.*,_schemas,__consumer_offsets,__transaction_state,connect-configs,connect-offsets,connect-status,connect-statuses", + "ksql.connect.basic.auth.credentials.file" : "", + "ksql.authorization.cache.expiry.time.secs" : "30", + "ksql.query.retry.backoff.initial.ms" : "15000", + "ksql.query.pull.max.concurrent.requests" : "2147483647", + "ksql.streams.auto.offset.reset" : "earliest", + "ksql.connect.url" : "http://localhost:8083", + "ksql.query.push.v2.new.latest.delay.ms" : "5000", + "ksql.query.push.v2.latest.reset.age.ms" : "30000", + "ksql.streams.default.production.exception.handler" : "io.confluent.ksql.errors.ProductionExceptionHandlerUtil$LogAndFailProductionExceptionHandler", + "ksql.query.pull.interpreter.enabled" : "true", + "ksql.query.pull.limit.clause.enabled" : "true", + "ksql.query.pull.router.thread.pool.size" : "50", + "ksql.query.push.v2.continuation.tokens.enabled" : "false", + "ksql.query.retry.backoff.max.ms" : "900000", + "ksql.timestamp.throw.on.invalid" : "false", + "ksql.persistence.default.format.value" : null, + "ksql.udfs.enabled" : "true", + "ksql.udf.enable.security.manager" : "true", + "ksql.connect.worker.config" : "", + "ksql.nested.error.set.null" : "true", + "ksql.query.pull.thread.pool.size" : "50", + "ksql.persistent.prefix" : "query_", + "ksql.metastore.backup.location" : "", + "ksql.error.classifier.regex" : "", + "ksql.suppress.buffer.size.bytes" : "-1" + } +} \ No newline at end of file diff --git a/ksqldb-functional-tests/src/test/resources/historical_plans/joins_-_stream_stream_inner_join_with_different_before_and_after_windows_-_PROTOBUF_NOSR/7.3.0_1651071323747/spec.json b/ksqldb-functional-tests/src/test/resources/historical_plans/joins_-_stream_stream_inner_join_with_different_before_and_after_windows_-_PROTOBUF_NOSR/7.3.0_1651071323747/spec.json new file mode 100644 index 000000000000..2e3bdff38885 --- /dev/null +++ b/ksqldb-functional-tests/src/test/resources/historical_plans/joins_-_stream_stream_inner_join_with_different_before_and_after_windows_-_PROTOBUF_NOSR/7.3.0_1651071323747/spec.json @@ -0,0 +1,237 @@ +{ + "version" : "7.3.0", + "timestamp" : 1651071323747, + "path" : "query-validation-tests/joins.json", + "schemas" : { + "CSAS_INNER_JOIN_0.INNER_JOIN" : { + "schema" : "`T_ID` BIGINT KEY, `NAME` STRING, `VALUE` BIGINT, `F1` STRING, `F2` BIGINT", + "keyFormat" : { + "format" : "KAFKA" + }, + "valueFormat" : { + "format" : "PROTOBUF_NOSR" + } + }, + "CSAS_INNER_JOIN_0.Join.Left" : { + "schema" : "`T_ID` BIGINT KEY, `T_NAME` STRING, `T_VALUE` BIGINT, `T_ROWTIME` BIGINT, `T_ROWPARTITION` INTEGER, `T_ROWOFFSET` BIGINT, `T_ID` BIGINT", + "valueFormat" : { + "format" : "PROTOBUF_NOSR" + } + }, + "CSAS_INNER_JOIN_0.KafkaTopic_Right.Source" : { + "schema" : "`ID` BIGINT KEY, `F1` STRING, `F2` BIGINT", + "keyFormat" : { + "format" : "KAFKA" + }, + "valueFormat" : { + "format" : "PROTOBUF_NOSR" + } + }, + "CSAS_INNER_JOIN_0.Join" : { + "schema" : "`T_ID` BIGINT KEY, `T_NAME` STRING, `T_VALUE` BIGINT, `T_ROWTIME` BIGINT, `T_ROWPARTITION` INTEGER, `T_ROWOFFSET` BIGINT, `T_ID` BIGINT", + "keyFormat" : { + "format" : "KAFKA" + } + }, + "CSAS_INNER_JOIN_0.KafkaTopic_Left.Source" : { + "schema" : "`ID` BIGINT KEY, `NAME` STRING, `VALUE` BIGINT", + "keyFormat" : { + "format" : "KAFKA" + }, + "valueFormat" : { + "format" : "PROTOBUF_NOSR" + } + }, + "CSAS_INNER_JOIN_0.Join.Right" : { + "schema" : "`TT_ID` BIGINT KEY, `TT_F1` STRING, `TT_F2` BIGINT, `TT_ROWTIME` BIGINT, `TT_ROWPARTITION` INTEGER, `TT_ROWOFFSET` BIGINT, `TT_ID` BIGINT", + "valueFormat" : { + "format" : "PROTOBUF_NOSR" + } + } + }, + "testCase" : { + "name" : "stream stream inner join with different before and after windows - PROTOBUF_NOSR", + "inputs" : [ { + "topic" : "left_topic", + "key" : 0, + "value" : { + "NAME" : "zero", + "VALUE" : 0 + }, + "timestamp" : 0 + }, { + "topic" : "right_topic", + "key" : 0, + "value" : { + "F1" : "blah", + "F2" : 50 + }, + "timestamp" : 11000 + }, { + "topic" : "left_topic", + "key" : 10, + "value" : { + "NAME" : "100", + "VALUE" : 5 + }, + "timestamp" : 12000 + }, { + "topic" : "left_topic", + "key" : 0, + "value" : { + "NAME" : "foo", + "VALUE" : 100 + }, + "timestamp" : 13000 + }, { + "topic" : "right_topic", + "key" : 0, + "value" : { + "F1" : "a", + "F2" : 10 + }, + "timestamp" : 15000 + }, { + "topic" : "right_topic", + "key" : 100, + "value" : { + "F1" : "newblah", + "F2" : 150 + }, + "timestamp" : 16000 + }, { + "topic" : "left_topic", + "key" : 90, + "value" : { + "NAME" : "ninety", + "VALUE" : 90 + }, + "timestamp" : 17000 + }, { + "topic" : "left_topic", + "key" : 0, + "value" : { + "NAME" : "bar", + "VALUE" : 99 + }, + "timestamp" : 30000 + } ], + "outputs" : [ { + "topic" : "INNER_JOIN", + "key" : 0, + "value" : { + "NAME" : "foo", + "VALUE" : 100, + "F1" : "blah", + "F2" : 50 + }, + "timestamp" : 13000 + }, { + "topic" : "INNER_JOIN", + "key" : 0, + "value" : { + "NAME" : "foo", + "VALUE" : 100, + "F1" : "a", + "F2" : 10 + }, + "timestamp" : 15000 + } ], + "topics" : [ { + "name" : "right_topic", + "replicas" : 1, + "numPartitions" : 4 + }, { + "name" : "INNER_JOIN", + "replicas" : 1, + "numPartitions" : 4 + }, { + "name" : "left_topic", + "replicas" : 1, + "numPartitions" : 4 + } ], + "statements" : [ "CREATE STREAM TEST (ID BIGINT KEY, NAME varchar, VALUE bigint) WITH (kafka_topic='left_topic', value_format='PROTOBUF_NOSR');", "CREATE STREAM TEST_STREAM (ID BIGINT KEY, F1 varchar, F2 bigint) WITH (kafka_topic='right_topic', value_format='PROTOBUF_NOSR');", "CREATE STREAM INNER_JOIN as SELECT t.id, name, value, f1, f2 FROM test t join TEST_STREAM tt WITHIN (11 seconds, 10 seconds) on t.id = tt.id;" ], + "post" : { + "sources" : [ { + "name" : "INNER_JOIN", + "type" : "STREAM", + "schema" : "`T_ID` BIGINT KEY, `NAME` STRING, `VALUE` BIGINT, `F1` STRING, `F2` BIGINT", + "keyFormat" : { + "format" : "KAFKA" + }, + "valueFormat" : "PROTOBUF_NOSR", + "keyFeatures" : [ ], + "valueFeatures" : [ ], + "isSource" : false + }, { + "name" : "TEST", + "type" : "STREAM", + "schema" : "`ID` BIGINT KEY, `NAME` STRING, `VALUE` BIGINT", + "keyFormat" : { + "format" : "KAFKA" + }, + "valueFormat" : "PROTOBUF_NOSR", + "keyFeatures" : [ ], + "valueFeatures" : [ ], + "isSource" : false + }, { + "name" : "TEST_STREAM", + "type" : "STREAM", + "schema" : "`ID` BIGINT KEY, `F1` STRING, `F2` BIGINT", + "keyFormat" : { + "format" : "KAFKA" + }, + "valueFormat" : "PROTOBUF_NOSR", + "keyFeatures" : [ ], + "valueFeatures" : [ ], + "isSource" : false + } ], + "topics" : { + "topics" : [ { + "name" : "_confluent-ksql-some.ksql.service.idquery_CSAS_INNER_JOIN_0-KSTREAM-JOINTHIS-0000000008-store-changelog", + "keyFormat" : { + "format" : "KAFKA" + }, + "valueFormat" : { + "format" : "PROTOBUF_NOSR" + } + }, { + "name" : "INNER_JOIN", + "keyFormat" : { + "format" : "KAFKA" + }, + "valueFormat" : { + "format" : "PROTOBUF_NOSR" + }, + "partitions" : 4 + }, { + "name" : "_confluent-ksql-some.ksql.service.idquery_CSAS_INNER_JOIN_0-KSTREAM-JOINOTHER-0000000009-store-changelog", + "keyFormat" : { + "format" : "KAFKA" + }, + "valueFormat" : { + "format" : "PROTOBUF_NOSR" + } + }, { + "name" : "left_topic", + "keyFormat" : { + "format" : "KAFKA" + }, + "valueFormat" : { + "format" : "PROTOBUF_NOSR" + }, + "partitions" : 4 + }, { + "name" : "right_topic", + "keyFormat" : { + "format" : "KAFKA" + }, + "valueFormat" : { + "format" : "PROTOBUF_NOSR" + }, + "partitions" : 4 + } ] + } + } + } +} \ No newline at end of file diff --git a/ksqldb-functional-tests/src/test/resources/historical_plans/joins_-_stream_stream_inner_join_with_different_before_and_after_windows_-_PROTOBUF_NOSR/7.3.0_1651071323747/topology b/ksqldb-functional-tests/src/test/resources/historical_plans/joins_-_stream_stream_inner_join_with_different_before_and_after_windows_-_PROTOBUF_NOSR/7.3.0_1651071323747/topology new file mode 100644 index 000000000000..942a8e59aba7 --- /dev/null +++ b/ksqldb-functional-tests/src/test/resources/historical_plans/joins_-_stream_stream_inner_join_with_different_before_and_after_windows_-_PROTOBUF_NOSR/7.3.0_1651071323747/topology @@ -0,0 +1,39 @@ +Topologies: + Sub-topology: 0 + Source: KSTREAM-SOURCE-0000000000 (topics: [left_topic]) + --> KSTREAM-TRANSFORMVALUES-0000000001 + Source: KSTREAM-SOURCE-0000000003 (topics: [right_topic]) + --> KSTREAM-TRANSFORMVALUES-0000000004 + Processor: KSTREAM-TRANSFORMVALUES-0000000001 (stores: []) + --> PrependAliasLeft + <-- KSTREAM-SOURCE-0000000000 + Processor: KSTREAM-TRANSFORMVALUES-0000000004 (stores: []) + --> PrependAliasRight + <-- KSTREAM-SOURCE-0000000003 + Processor: PrependAliasLeft (stores: []) + --> Join-this-windowed + <-- KSTREAM-TRANSFORMVALUES-0000000001 + Processor: PrependAliasRight (stores: []) + --> Join-other-windowed + <-- KSTREAM-TRANSFORMVALUES-0000000004 + Processor: Join-other-windowed (stores: [KSTREAM-JOINOTHER-0000000009-store]) + --> Join-other-join + <-- PrependAliasRight + Processor: Join-this-windowed (stores: [KSTREAM-JOINTHIS-0000000008-store]) + --> Join-this-join + <-- PrependAliasLeft + Processor: Join-other-join (stores: [KSTREAM-JOINTHIS-0000000008-store]) + --> Join-merge + <-- Join-other-windowed + Processor: Join-this-join (stores: [KSTREAM-JOINOTHER-0000000009-store]) + --> Join-merge + <-- Join-this-windowed + Processor: Join-merge (stores: []) + --> Project + <-- Join-this-join, Join-other-join + Processor: Project (stores: []) + --> KSTREAM-SINK-0000000012 + <-- Join-merge + Sink: KSTREAM-SINK-0000000012 (topic: INNER_JOIN) + <-- Project + diff --git a/ksqldb-functional-tests/src/test/resources/historical_plans/joins_-_stream_stream_inner_join_with_out_of_order_and_custom_grace_period_-_PROTOBUF_NOSR/7.3.0_1651071333941/plan.json b/ksqldb-functional-tests/src/test/resources/historical_plans/joins_-_stream_stream_inner_join_with_out_of_order_and_custom_grace_period_-_PROTOBUF_NOSR/7.3.0_1651071333941/plan.json new file mode 100644 index 000000000000..18743f559a87 --- /dev/null +++ b/ksqldb-functional-tests/src/test/resources/historical_plans/joins_-_stream_stream_inner_join_with_out_of_order_and_custom_grace_period_-_PROTOBUF_NOSR/7.3.0_1651071333941/plan.json @@ -0,0 +1,296 @@ +{ + "plan" : [ { + "@type" : "ksqlPlanV1", + "statementText" : "CREATE STREAM LEFT_STREAM (ID BIGINT KEY, L1 STRING) WITH (KAFKA_TOPIC='left_topic', KEY_FORMAT='KAFKA', VALUE_FORMAT='PROTOBUF_NOSR');", + "ddlCommand" : { + "@type" : "createStreamV1", + "sourceName" : "LEFT_STREAM", + "schema" : "`ID` BIGINT KEY, `L1` STRING", + "timestampColumn" : null, + "topicName" : "left_topic", + "formats" : { + "keyFormat" : { + "format" : "KAFKA", + "properties" : { } + }, + "valueFormat" : { + "format" : "PROTOBUF_NOSR", + "properties" : { } + } + }, + "windowInfo" : null, + "orReplace" : false, + "isSource" : false + }, + "queryPlan" : null + }, { + "@type" : "ksqlPlanV1", + "statementText" : "CREATE STREAM RIGHT_STREAM (ID BIGINT KEY, L2 STRING) WITH (KAFKA_TOPIC='right_topic', KEY_FORMAT='KAFKA', VALUE_FORMAT='PROTOBUF_NOSR');", + "ddlCommand" : { + "@type" : "createStreamV1", + "sourceName" : "RIGHT_STREAM", + "schema" : "`ID` BIGINT KEY, `L2` STRING", + "timestampColumn" : null, + "topicName" : "right_topic", + "formats" : { + "keyFormat" : { + "format" : "KAFKA", + "properties" : { } + }, + "valueFormat" : { + "format" : "PROTOBUF_NOSR", + "properties" : { } + } + }, + "windowInfo" : null, + "orReplace" : false, + "isSource" : false + }, + "queryPlan" : null + }, { + "@type" : "ksqlPlanV1", + "statementText" : "CREATE STREAM INNER_JOIN AS SELECT\n T.ID T_ID,\n T.L1 L1,\n TT.L2 L2\nFROM LEFT_STREAM T\nINNER JOIN RIGHT_STREAM TT WITHIN 1 MINUTES GRACE PERIOD 1 MINUTES ON ((T.ID = TT.ID))\nEMIT CHANGES", + "ddlCommand" : { + "@type" : "createStreamV1", + "sourceName" : "INNER_JOIN", + "schema" : "`T_ID` BIGINT KEY, `L1` STRING, `L2` STRING", + "timestampColumn" : null, + "topicName" : "INNER_JOIN", + "formats" : { + "keyFormat" : { + "format" : "KAFKA", + "properties" : { } + }, + "valueFormat" : { + "format" : "PROTOBUF_NOSR", + "properties" : { } + } + }, + "windowInfo" : null, + "orReplace" : false, + "isSource" : false + }, + "queryPlan" : { + "sources" : [ "LEFT_STREAM", "RIGHT_STREAM" ], + "sink" : "INNER_JOIN", + "physicalPlan" : { + "@type" : "streamSinkV1", + "properties" : { + "queryContext" : "INNER_JOIN" + }, + "source" : { + "@type" : "streamSelectV1", + "properties" : { + "queryContext" : "Project" + }, + "source" : { + "@type" : "streamStreamJoinV1", + "properties" : { + "queryContext" : "Join" + }, + "joinType" : "INNER", + "leftInternalFormats" : { + "keyFormat" : { + "format" : "KAFKA", + "properties" : { } + }, + "valueFormat" : { + "format" : "PROTOBUF_NOSR", + "properties" : { } + } + }, + "rightInternalFormats" : { + "keyFormat" : { + "format" : "KAFKA", + "properties" : { } + }, + "valueFormat" : { + "format" : "PROTOBUF_NOSR", + "properties" : { } + } + }, + "leftSource" : { + "@type" : "streamSelectV1", + "properties" : { + "queryContext" : "PrependAliasLeft" + }, + "source" : { + "@type" : "streamSourceV1", + "properties" : { + "queryContext" : "KafkaTopic_Left/Source" + }, + "topicName" : "left_topic", + "formats" : { + "keyFormat" : { + "format" : "KAFKA", + "properties" : { } + }, + "valueFormat" : { + "format" : "PROTOBUF_NOSR", + "properties" : { } + } + }, + "timestampColumn" : null, + "sourceSchema" : "`ID` BIGINT KEY, `L1` STRING", + "pseudoColumnVersion" : 1 + }, + "keyColumnNames" : [ "T_ID" ], + "selectedKeys" : null, + "selectExpressions" : [ "L1 AS T_L1", "ROWTIME AS T_ROWTIME", "ROWPARTITION AS T_ROWPARTITION", "ROWOFFSET AS T_ROWOFFSET", "ID AS T_ID" ] + }, + "rightSource" : { + "@type" : "streamSelectV1", + "properties" : { + "queryContext" : "PrependAliasRight" + }, + "source" : { + "@type" : "streamSourceV1", + "properties" : { + "queryContext" : "KafkaTopic_Right/Source" + }, + "topicName" : "right_topic", + "formats" : { + "keyFormat" : { + "format" : "KAFKA", + "properties" : { } + }, + "valueFormat" : { + "format" : "PROTOBUF_NOSR", + "properties" : { } + } + }, + "timestampColumn" : null, + "sourceSchema" : "`ID` BIGINT KEY, `L2` STRING", + "pseudoColumnVersion" : 1 + }, + "keyColumnNames" : [ "TT_ID" ], + "selectedKeys" : null, + "selectExpressions" : [ "L2 AS TT_L2", "ROWTIME AS TT_ROWTIME", "ROWPARTITION AS TT_ROWPARTITION", "ROWOFFSET AS TT_ROWOFFSET", "ID AS TT_ID" ] + }, + "beforeMillis" : 60.000000000, + "afterMillis" : 60.000000000, + "graceMillis" : 60.000000000, + "keyColName" : "T_ID" + }, + "keyColumnNames" : [ "T_ID" ], + "selectedKeys" : null, + "selectExpressions" : [ "T_L1 AS L1", "TT_L2 AS L2" ] + }, + "formats" : { + "keyFormat" : { + "format" : "KAFKA", + "properties" : { } + }, + "valueFormat" : { + "format" : "PROTOBUF_NOSR", + "properties" : { } + } + }, + "topicName" : "INNER_JOIN", + "timestampColumn" : null + }, + "queryId" : "CSAS_INNER_JOIN_0", + "runtimeId" : null + } + } ], + "configs" : { + "ksql.extension.dir" : "ext", + "ksql.streams.cache.max.bytes.buffering" : "0", + "metric.reporters" : "", + "ksql.query.status.running.threshold.seconds" : "300", + "ksql.connect.basic.auth.credentials.reload" : "false", + "ksql.output.topic.name.prefix" : "", + "ksql.query.pull.stream.enabled" : "true", + "ksql.query.push.v2.interpreter.enabled" : "true", + "ksql.queryanonymizer.logs_enabled" : "true", + "ksql.variable.substitution.enable" : "true", + "ksql.streams.shutdown.timeout.ms" : "300000", + "ksql.query.pull.max.allowed.offset.lag" : "9223372036854775807", + "ksql.query.pull.max.qps" : "2147483647", + "ksql.access.validator.enable" : "auto", + "ksql.streams.bootstrap.servers" : "localhost:0", + "ksql.query.pull.metrics.enabled" : "true", + "ksql.metrics.extension" : null, + "ksql.query.push.v2.alos.enabled" : "true", + "ksql.query.push.v2.max.hourly.bandwidth.megabytes" : "2147483647", + "ksql.query.pull.range.scan.enabled" : "true", + "ksql.transient.query.cleanup.service.initial.delay.seconds" : "600", + "ksql.hidden.topics" : "_confluent.*,__confluent.*,_schemas,__consumer_offsets,__transaction_state,connect-configs,connect-offsets,connect-status,connect-statuses", + "ksql.lambdas.enabled" : "true", + "ksql.source.table.materialization.enabled" : "true", + "ksql.query.pull.max.hourly.bandwidth.megabytes" : "2147483647", + "ksql.sink.window.change.log.additional.retention" : "1000000", + "ksql.query.persistent.active.limit" : "2147483647", + "ksql.persistence.wrap.single.values" : null, + "ksql.query.transient.max.bytes.buffering.total" : "-1", + "ksql.connect.basic.auth.credentials.source" : "NONE", + "ksql.schema.registry.url" : "schema_registry.url:0", + "ksql.properties.overrides.denylist" : "", + "ksql.service.id" : "some.ksql.service.id", + "ksql.query.push.v2.max.catchup.consumers" : "5", + "ksql.query.push.v2.enabled" : "false", + "ksql.transient.query.cleanup.service.enable" : "true", + "ksql.query.push.v2.metrics.enabled" : "true", + "ksql.rowpartition.rowoffset.enabled" : "true", + "ksql.streams.commit.interval.ms" : "2000", + "ksql.query.pull.table.scan.enabled" : "true", + "ksql.streams.auto.commit.interval.ms" : "0", + "ksql.streams.topology.optimization" : "all", + "ksql.endpoint.migrate.query" : "true", + "ksql.query.push.v2.registry.installed" : "false", + "ksql.streams.num.stream.threads" : "4", + "ksql.metrics.tags.custom" : "", + "ksql.query.push.v2.catchup.consumer.msg.window" : "50", + "ksql.runtime.feature.shared.enabled" : "false", + "ksql.udf.collect.metrics" : "false", + "ksql.new.query.planner.enabled" : "false", + "ksql.connect.request.headers.plugin" : null, + "ksql.security.extension.class" : null, + "ksql.transient.prefix" : "transient_", + "ksql.headers.columns.enabled" : "true", + "ksql.streams.default.deserialization.exception.handler" : "io.confluent.ksql.errors.LogMetricAndContinueExceptionHandler", + "ksql.connect.request.timeout.ms" : "5000", + "ksql.query.pull.enable.standby.reads" : "false", + "ksql.persistence.default.format.key" : "KAFKA", + "ksql.query.persistent.max.bytes.buffering.total" : "-1", + "ksql.query.error.max.queue.size" : "10", + "ksql.query.cleanup.shutdown.timeout.ms" : "30000", + "ksql.internal.topic.min.insync.replicas" : "1", + "ksql.internal.topic.replicas" : "1", + "ksql.insert.into.values.enabled" : "true", + "ksql.queryanonymizer.cluster_namespace" : null, + "ksql.create.or.replace.enabled" : "true", + "ksql.shared.runtimes.count" : "2", + "ksql.cast.strings.preserve.nulls" : "true", + "ksql.authorization.cache.max.entries" : "10000", + "ksql.pull.queries.enable" : "true", + "ksql.transient.query.cleanup.service.period.seconds" : "600", + "ksql.suppress.enabled" : "false", + "ksql.readonly.topics" : "_confluent.*,__confluent.*,_schemas,__consumer_offsets,__transaction_state,connect-configs,connect-offsets,connect-status,connect-statuses", + "ksql.connect.basic.auth.credentials.file" : "", + "ksql.authorization.cache.expiry.time.secs" : "30", + "ksql.query.retry.backoff.initial.ms" : "15000", + "ksql.query.pull.max.concurrent.requests" : "2147483647", + "ksql.streams.auto.offset.reset" : "earliest", + "ksql.connect.url" : "http://localhost:8083", + "ksql.query.push.v2.new.latest.delay.ms" : "5000", + "ksql.query.push.v2.latest.reset.age.ms" : "30000", + "ksql.streams.default.production.exception.handler" : "io.confluent.ksql.errors.ProductionExceptionHandlerUtil$LogAndFailProductionExceptionHandler", + "ksql.query.pull.interpreter.enabled" : "true", + "ksql.query.pull.limit.clause.enabled" : "true", + "ksql.query.pull.router.thread.pool.size" : "50", + "ksql.query.push.v2.continuation.tokens.enabled" : "false", + "ksql.query.retry.backoff.max.ms" : "900000", + "ksql.timestamp.throw.on.invalid" : "false", + "ksql.persistence.default.format.value" : null, + "ksql.udfs.enabled" : "true", + "ksql.udf.enable.security.manager" : "true", + "ksql.connect.worker.config" : "", + "ksql.nested.error.set.null" : "true", + "ksql.query.pull.thread.pool.size" : "50", + "ksql.persistent.prefix" : "query_", + "ksql.metastore.backup.location" : "", + "ksql.error.classifier.regex" : "", + "ksql.suppress.buffer.size.bytes" : "-1" + } +} \ No newline at end of file diff --git a/ksqldb-functional-tests/src/test/resources/historical_plans/joins_-_stream_stream_inner_join_with_out_of_order_and_custom_grace_period_-_PROTOBUF_NOSR/7.3.0_1651071333941/spec.json b/ksqldb-functional-tests/src/test/resources/historical_plans/joins_-_stream_stream_inner_join_with_out_of_order_and_custom_grace_period_-_PROTOBUF_NOSR/7.3.0_1651071333941/spec.json new file mode 100644 index 000000000000..38a9a764ac72 --- /dev/null +++ b/ksqldb-functional-tests/src/test/resources/historical_plans/joins_-_stream_stream_inner_join_with_out_of_order_and_custom_grace_period_-_PROTOBUF_NOSR/7.3.0_1651071333941/spec.json @@ -0,0 +1,218 @@ +{ + "version" : "7.3.0", + "timestamp" : 1651071333941, + "path" : "query-validation-tests/joins.json", + "schemas" : { + "CSAS_INNER_JOIN_0.INNER_JOIN" : { + "schema" : "`T_ID` BIGINT KEY, `L1` STRING, `L2` STRING", + "keyFormat" : { + "format" : "KAFKA" + }, + "valueFormat" : { + "format" : "PROTOBUF_NOSR" + } + }, + "CSAS_INNER_JOIN_0.Join.Left" : { + "schema" : "`T_ID` BIGINT KEY, `T_L1` STRING, `T_ROWTIME` BIGINT, `T_ROWPARTITION` INTEGER, `T_ROWOFFSET` BIGINT, `T_ID` BIGINT", + "valueFormat" : { + "format" : "PROTOBUF_NOSR" + } + }, + "CSAS_INNER_JOIN_0.KafkaTopic_Right.Source" : { + "schema" : "`ID` BIGINT KEY, `L2` STRING", + "keyFormat" : { + "format" : "KAFKA" + }, + "valueFormat" : { + "format" : "PROTOBUF_NOSR" + } + }, + "CSAS_INNER_JOIN_0.Join" : { + "schema" : "`T_ID` BIGINT KEY, `T_L1` STRING, `T_ROWTIME` BIGINT, `T_ROWPARTITION` INTEGER, `T_ROWOFFSET` BIGINT, `T_ID` BIGINT", + "keyFormat" : { + "format" : "KAFKA" + } + }, + "CSAS_INNER_JOIN_0.KafkaTopic_Left.Source" : { + "schema" : "`ID` BIGINT KEY, `L1` STRING", + "keyFormat" : { + "format" : "KAFKA" + }, + "valueFormat" : { + "format" : "PROTOBUF_NOSR" + } + }, + "CSAS_INNER_JOIN_0.Join.Right" : { + "schema" : "`TT_ID` BIGINT KEY, `TT_L2` STRING, `TT_ROWTIME` BIGINT, `TT_ROWPARTITION` INTEGER, `TT_ROWOFFSET` BIGINT, `TT_ID` BIGINT", + "valueFormat" : { + "format" : "PROTOBUF_NOSR" + } + } + }, + "testCase" : { + "name" : "stream stream inner join with out of order and custom grace period - PROTOBUF_NOSR", + "inputs" : [ { + "topic" : "left_topic", + "key" : 0, + "value" : { + "L1" : "A" + }, + "timestamp" : 0 + }, { + "topic" : "right_topic", + "key" : 0, + "value" : { + "L2" : "a" + }, + "timestamp" : 60000 + }, { + "topic" : "left_topic", + "key" : 1, + "value" : { + "L1" : "B" + }, + "timestamp" : 330000 + }, { + "topic" : "left_topic", + "key" : 2, + "value" : { + "L1" : "C" + }, + "timestamp" : 90000 + }, { + "topic" : "right_topic", + "key" : 2, + "value" : { + "L2" : "c" + }, + "timestamp" : 90000 + }, { + "topic" : "left_topic", + "key" : 3, + "value" : { + "L1" : "D" + }, + "timestamp" : 60000 + }, { + "topic" : "right_topic", + "key" : 3, + "value" : { + "L2" : "d" + }, + "timestamp" : 60000 + } ], + "outputs" : [ { + "topic" : "INNER_JOIN", + "key" : 0, + "value" : { + "L1" : "A", + "L2" : "a" + }, + "timestamp" : 60000 + }, { + "topic" : "INNER_JOIN", + "key" : 2, + "value" : { + "L1" : "C", + "L2" : "c" + }, + "timestamp" : 90000 + } ], + "topics" : [ { + "name" : "right_topic", + "replicas" : 1, + "numPartitions" : 4 + }, { + "name" : "INNER_JOIN", + "replicas" : 1, + "numPartitions" : 4 + }, { + "name" : "left_topic", + "replicas" : 1, + "numPartitions" : 4 + } ], + "statements" : [ "CREATE STREAM LEFT_STREAM (id BIGINT KEY, l1 VARCHAR) WITH (kafka_topic='left_topic', value_format='PROTOBUF_NOSR');", "CREATE STREAM RIGHT_STREAM (id BIGINT KEY, l2 VARCHAR) WITH (kafka_topic='right_topic', value_format='PROTOBUF_NOSR');", "CREATE STREAM INNER_JOIN as SELECT t.id, l1, l2 FROM LEFT_STREAM t join RIGHT_STREAM tt WITHIN 1 minute GRACE PERIOD 1 minute on t.id = tt.id;" ], + "post" : { + "sources" : [ { + "name" : "INNER_JOIN", + "type" : "STREAM", + "schema" : "`T_ID` BIGINT KEY, `L1` STRING, `L2` STRING", + "keyFormat" : { + "format" : "KAFKA" + }, + "valueFormat" : "PROTOBUF_NOSR", + "keyFeatures" : [ ], + "valueFeatures" : [ ], + "isSource" : false + }, { + "name" : "LEFT_STREAM", + "type" : "STREAM", + "schema" : "`ID` BIGINT KEY, `L1` STRING", + "keyFormat" : { + "format" : "KAFKA" + }, + "valueFormat" : "PROTOBUF_NOSR", + "keyFeatures" : [ ], + "valueFeatures" : [ ], + "isSource" : false + }, { + "name" : "RIGHT_STREAM", + "type" : "STREAM", + "schema" : "`ID` BIGINT KEY, `L2` STRING", + "keyFormat" : { + "format" : "KAFKA" + }, + "valueFormat" : "PROTOBUF_NOSR", + "keyFeatures" : [ ], + "valueFeatures" : [ ], + "isSource" : false + } ], + "topics" : { + "topics" : [ { + "name" : "_confluent-ksql-some.ksql.service.idquery_CSAS_INNER_JOIN_0-KSTREAM-JOINTHIS-0000000008-store-changelog", + "keyFormat" : { + "format" : "KAFKA" + }, + "valueFormat" : { + "format" : "PROTOBUF_NOSR" + } + }, { + "name" : "INNER_JOIN", + "keyFormat" : { + "format" : "KAFKA" + }, + "valueFormat" : { + "format" : "PROTOBUF_NOSR" + }, + "partitions" : 4 + }, { + "name" : "_confluent-ksql-some.ksql.service.idquery_CSAS_INNER_JOIN_0-KSTREAM-JOINOTHER-0000000009-store-changelog", + "keyFormat" : { + "format" : "KAFKA" + }, + "valueFormat" : { + "format" : "PROTOBUF_NOSR" + } + }, { + "name" : "left_topic", + "keyFormat" : { + "format" : "KAFKA" + }, + "valueFormat" : { + "format" : "PROTOBUF_NOSR" + }, + "partitions" : 4 + }, { + "name" : "right_topic", + "keyFormat" : { + "format" : "KAFKA" + }, + "valueFormat" : { + "format" : "PROTOBUF_NOSR" + }, + "partitions" : 4 + } ] + } + } + } +} \ No newline at end of file diff --git a/ksqldb-functional-tests/src/test/resources/historical_plans/joins_-_stream_stream_inner_join_with_out_of_order_and_custom_grace_period_-_PROTOBUF_NOSR/7.3.0_1651071333941/topology b/ksqldb-functional-tests/src/test/resources/historical_plans/joins_-_stream_stream_inner_join_with_out_of_order_and_custom_grace_period_-_PROTOBUF_NOSR/7.3.0_1651071333941/topology new file mode 100644 index 000000000000..942a8e59aba7 --- /dev/null +++ b/ksqldb-functional-tests/src/test/resources/historical_plans/joins_-_stream_stream_inner_join_with_out_of_order_and_custom_grace_period_-_PROTOBUF_NOSR/7.3.0_1651071333941/topology @@ -0,0 +1,39 @@ +Topologies: + Sub-topology: 0 + Source: KSTREAM-SOURCE-0000000000 (topics: [left_topic]) + --> KSTREAM-TRANSFORMVALUES-0000000001 + Source: KSTREAM-SOURCE-0000000003 (topics: [right_topic]) + --> KSTREAM-TRANSFORMVALUES-0000000004 + Processor: KSTREAM-TRANSFORMVALUES-0000000001 (stores: []) + --> PrependAliasLeft + <-- KSTREAM-SOURCE-0000000000 + Processor: KSTREAM-TRANSFORMVALUES-0000000004 (stores: []) + --> PrependAliasRight + <-- KSTREAM-SOURCE-0000000003 + Processor: PrependAliasLeft (stores: []) + --> Join-this-windowed + <-- KSTREAM-TRANSFORMVALUES-0000000001 + Processor: PrependAliasRight (stores: []) + --> Join-other-windowed + <-- KSTREAM-TRANSFORMVALUES-0000000004 + Processor: Join-other-windowed (stores: [KSTREAM-JOINOTHER-0000000009-store]) + --> Join-other-join + <-- PrependAliasRight + Processor: Join-this-windowed (stores: [KSTREAM-JOINTHIS-0000000008-store]) + --> Join-this-join + <-- PrependAliasLeft + Processor: Join-other-join (stores: [KSTREAM-JOINTHIS-0000000008-store]) + --> Join-merge + <-- Join-other-windowed + Processor: Join-this-join (stores: [KSTREAM-JOINOTHER-0000000009-store]) + --> Join-merge + <-- Join-this-windowed + Processor: Join-merge (stores: []) + --> Project + <-- Join-this-join, Join-other-join + Processor: Project (stores: []) + --> KSTREAM-SINK-0000000012 + <-- Join-merge + Sink: KSTREAM-SINK-0000000012 (topic: INNER_JOIN) + <-- Project + diff --git a/ksqldb-functional-tests/src/test/resources/historical_plans/joins_-_stream_stream_inner_join_with_out_of_order_messages_-_PROTOBUF_NOSR/7.3.0_1651071328265/plan.json b/ksqldb-functional-tests/src/test/resources/historical_plans/joins_-_stream_stream_inner_join_with_out_of_order_messages_-_PROTOBUF_NOSR/7.3.0_1651071328265/plan.json new file mode 100644 index 000000000000..69e8a8fa0db5 --- /dev/null +++ b/ksqldb-functional-tests/src/test/resources/historical_plans/joins_-_stream_stream_inner_join_with_out_of_order_messages_-_PROTOBUF_NOSR/7.3.0_1651071328265/plan.json @@ -0,0 +1,296 @@ +{ + "plan" : [ { + "@type" : "ksqlPlanV1", + "statementText" : "CREATE STREAM TEST (ID BIGINT KEY, NAME STRING, VALUE BIGINT) WITH (KAFKA_TOPIC='left_topic', KEY_FORMAT='KAFKA', VALUE_FORMAT='PROTOBUF_NOSR');", + "ddlCommand" : { + "@type" : "createStreamV1", + "sourceName" : "TEST", + "schema" : "`ID` BIGINT KEY, `NAME` STRING, `VALUE` BIGINT", + "timestampColumn" : null, + "topicName" : "left_topic", + "formats" : { + "keyFormat" : { + "format" : "KAFKA", + "properties" : { } + }, + "valueFormat" : { + "format" : "PROTOBUF_NOSR", + "properties" : { } + } + }, + "windowInfo" : null, + "orReplace" : false, + "isSource" : false + }, + "queryPlan" : null + }, { + "@type" : "ksqlPlanV1", + "statementText" : "CREATE STREAM TEST_STREAM (ID BIGINT KEY, F1 STRING, F2 BIGINT) WITH (KAFKA_TOPIC='right_topic', KEY_FORMAT='KAFKA', VALUE_FORMAT='PROTOBUF_NOSR');", + "ddlCommand" : { + "@type" : "createStreamV1", + "sourceName" : "TEST_STREAM", + "schema" : "`ID` BIGINT KEY, `F1` STRING, `F2` BIGINT", + "timestampColumn" : null, + "topicName" : "right_topic", + "formats" : { + "keyFormat" : { + "format" : "KAFKA", + "properties" : { } + }, + "valueFormat" : { + "format" : "PROTOBUF_NOSR", + "properties" : { } + } + }, + "windowInfo" : null, + "orReplace" : false, + "isSource" : false + }, + "queryPlan" : null + }, { + "@type" : "ksqlPlanV1", + "statementText" : "CREATE STREAM INNER_JOIN AS SELECT\n T.ID T_ID,\n T.NAME NAME,\n T.VALUE VALUE,\n TT.F1 F1,\n TT.F2 F2\nFROM TEST T\nINNER JOIN TEST_STREAM TT WITHIN 10 SECONDS ON ((T.ID = TT.ID))\nEMIT CHANGES", + "ddlCommand" : { + "@type" : "createStreamV1", + "sourceName" : "INNER_JOIN", + "schema" : "`T_ID` BIGINT KEY, `NAME` STRING, `VALUE` BIGINT, `F1` STRING, `F2` BIGINT", + "timestampColumn" : null, + "topicName" : "INNER_JOIN", + "formats" : { + "keyFormat" : { + "format" : "KAFKA", + "properties" : { } + }, + "valueFormat" : { + "format" : "PROTOBUF_NOSR", + "properties" : { } + } + }, + "windowInfo" : null, + "orReplace" : false, + "isSource" : false + }, + "queryPlan" : { + "sources" : [ "TEST", "TEST_STREAM" ], + "sink" : "INNER_JOIN", + "physicalPlan" : { + "@type" : "streamSinkV1", + "properties" : { + "queryContext" : "INNER_JOIN" + }, + "source" : { + "@type" : "streamSelectV1", + "properties" : { + "queryContext" : "Project" + }, + "source" : { + "@type" : "streamStreamJoinV1", + "properties" : { + "queryContext" : "Join" + }, + "joinType" : "INNER", + "leftInternalFormats" : { + "keyFormat" : { + "format" : "KAFKA", + "properties" : { } + }, + "valueFormat" : { + "format" : "PROTOBUF_NOSR", + "properties" : { } + } + }, + "rightInternalFormats" : { + "keyFormat" : { + "format" : "KAFKA", + "properties" : { } + }, + "valueFormat" : { + "format" : "PROTOBUF_NOSR", + "properties" : { } + } + }, + "leftSource" : { + "@type" : "streamSelectV1", + "properties" : { + "queryContext" : "PrependAliasLeft" + }, + "source" : { + "@type" : "streamSourceV1", + "properties" : { + "queryContext" : "KafkaTopic_Left/Source" + }, + "topicName" : "left_topic", + "formats" : { + "keyFormat" : { + "format" : "KAFKA", + "properties" : { } + }, + "valueFormat" : { + "format" : "PROTOBUF_NOSR", + "properties" : { } + } + }, + "timestampColumn" : null, + "sourceSchema" : "`ID` BIGINT KEY, `NAME` STRING, `VALUE` BIGINT", + "pseudoColumnVersion" : 1 + }, + "keyColumnNames" : [ "T_ID" ], + "selectedKeys" : null, + "selectExpressions" : [ "NAME AS T_NAME", "VALUE AS T_VALUE", "ROWTIME AS T_ROWTIME", "ROWPARTITION AS T_ROWPARTITION", "ROWOFFSET AS T_ROWOFFSET", "ID AS T_ID" ] + }, + "rightSource" : { + "@type" : "streamSelectV1", + "properties" : { + "queryContext" : "PrependAliasRight" + }, + "source" : { + "@type" : "streamSourceV1", + "properties" : { + "queryContext" : "KafkaTopic_Right/Source" + }, + "topicName" : "right_topic", + "formats" : { + "keyFormat" : { + "format" : "KAFKA", + "properties" : { } + }, + "valueFormat" : { + "format" : "PROTOBUF_NOSR", + "properties" : { } + } + }, + "timestampColumn" : null, + "sourceSchema" : "`ID` BIGINT KEY, `F1` STRING, `F2` BIGINT", + "pseudoColumnVersion" : 1 + }, + "keyColumnNames" : [ "TT_ID" ], + "selectedKeys" : null, + "selectExpressions" : [ "F1 AS TT_F1", "F2 AS TT_F2", "ROWTIME AS TT_ROWTIME", "ROWPARTITION AS TT_ROWPARTITION", "ROWOFFSET AS TT_ROWOFFSET", "ID AS TT_ID" ] + }, + "beforeMillis" : 10.000000000, + "afterMillis" : 10.000000000, + "graceMillis" : null, + "keyColName" : "T_ID" + }, + "keyColumnNames" : [ "T_ID" ], + "selectedKeys" : null, + "selectExpressions" : [ "T_NAME AS NAME", "T_VALUE AS VALUE", "TT_F1 AS F1", "TT_F2 AS F2" ] + }, + "formats" : { + "keyFormat" : { + "format" : "KAFKA", + "properties" : { } + }, + "valueFormat" : { + "format" : "PROTOBUF_NOSR", + "properties" : { } + } + }, + "topicName" : "INNER_JOIN", + "timestampColumn" : null + }, + "queryId" : "CSAS_INNER_JOIN_0", + "runtimeId" : null + } + } ], + "configs" : { + "ksql.extension.dir" : "ext", + "ksql.streams.cache.max.bytes.buffering" : "0", + "metric.reporters" : "", + "ksql.query.status.running.threshold.seconds" : "300", + "ksql.connect.basic.auth.credentials.reload" : "false", + "ksql.output.topic.name.prefix" : "", + "ksql.query.pull.stream.enabled" : "true", + "ksql.query.push.v2.interpreter.enabled" : "true", + "ksql.queryanonymizer.logs_enabled" : "true", + "ksql.variable.substitution.enable" : "true", + "ksql.streams.shutdown.timeout.ms" : "300000", + "ksql.query.pull.max.allowed.offset.lag" : "9223372036854775807", + "ksql.query.pull.max.qps" : "2147483647", + "ksql.access.validator.enable" : "auto", + "ksql.streams.bootstrap.servers" : "localhost:0", + "ksql.query.pull.metrics.enabled" : "true", + "ksql.metrics.extension" : null, + "ksql.query.push.v2.alos.enabled" : "true", + "ksql.query.push.v2.max.hourly.bandwidth.megabytes" : "2147483647", + "ksql.query.pull.range.scan.enabled" : "true", + "ksql.transient.query.cleanup.service.initial.delay.seconds" : "600", + "ksql.hidden.topics" : "_confluent.*,__confluent.*,_schemas,__consumer_offsets,__transaction_state,connect-configs,connect-offsets,connect-status,connect-statuses", + "ksql.lambdas.enabled" : "true", + "ksql.source.table.materialization.enabled" : "true", + "ksql.query.pull.max.hourly.bandwidth.megabytes" : "2147483647", + "ksql.sink.window.change.log.additional.retention" : "1000000", + "ksql.query.persistent.active.limit" : "2147483647", + "ksql.persistence.wrap.single.values" : null, + "ksql.query.transient.max.bytes.buffering.total" : "-1", + "ksql.connect.basic.auth.credentials.source" : "NONE", + "ksql.schema.registry.url" : "schema_registry.url:0", + "ksql.properties.overrides.denylist" : "", + "ksql.service.id" : "some.ksql.service.id", + "ksql.query.push.v2.max.catchup.consumers" : "5", + "ksql.query.push.v2.enabled" : "false", + "ksql.transient.query.cleanup.service.enable" : "true", + "ksql.query.push.v2.metrics.enabled" : "true", + "ksql.rowpartition.rowoffset.enabled" : "true", + "ksql.streams.commit.interval.ms" : "2000", + "ksql.query.pull.table.scan.enabled" : "true", + "ksql.streams.auto.commit.interval.ms" : "0", + "ksql.streams.topology.optimization" : "all", + "ksql.endpoint.migrate.query" : "true", + "ksql.query.push.v2.registry.installed" : "false", + "ksql.streams.num.stream.threads" : "4", + "ksql.metrics.tags.custom" : "", + "ksql.query.push.v2.catchup.consumer.msg.window" : "50", + "ksql.runtime.feature.shared.enabled" : "false", + "ksql.udf.collect.metrics" : "false", + "ksql.new.query.planner.enabled" : "false", + "ksql.connect.request.headers.plugin" : null, + "ksql.security.extension.class" : null, + "ksql.transient.prefix" : "transient_", + "ksql.headers.columns.enabled" : "true", + "ksql.streams.default.deserialization.exception.handler" : "io.confluent.ksql.errors.LogMetricAndContinueExceptionHandler", + "ksql.connect.request.timeout.ms" : "5000", + "ksql.query.pull.enable.standby.reads" : "false", + "ksql.persistence.default.format.key" : "KAFKA", + "ksql.query.persistent.max.bytes.buffering.total" : "-1", + "ksql.query.error.max.queue.size" : "10", + "ksql.query.cleanup.shutdown.timeout.ms" : "30000", + "ksql.internal.topic.min.insync.replicas" : "1", + "ksql.internal.topic.replicas" : "1", + "ksql.insert.into.values.enabled" : "true", + "ksql.queryanonymizer.cluster_namespace" : null, + "ksql.create.or.replace.enabled" : "true", + "ksql.shared.runtimes.count" : "2", + "ksql.cast.strings.preserve.nulls" : "true", + "ksql.authorization.cache.max.entries" : "10000", + "ksql.pull.queries.enable" : "true", + "ksql.transient.query.cleanup.service.period.seconds" : "600", + "ksql.suppress.enabled" : "false", + "ksql.readonly.topics" : "_confluent.*,__confluent.*,_schemas,__consumer_offsets,__transaction_state,connect-configs,connect-offsets,connect-status,connect-statuses", + "ksql.connect.basic.auth.credentials.file" : "", + "ksql.authorization.cache.expiry.time.secs" : "30", + "ksql.query.retry.backoff.initial.ms" : "15000", + "ksql.query.pull.max.concurrent.requests" : "2147483647", + "ksql.streams.auto.offset.reset" : "earliest", + "ksql.connect.url" : "http://localhost:8083", + "ksql.query.push.v2.new.latest.delay.ms" : "5000", + "ksql.query.push.v2.latest.reset.age.ms" : "30000", + "ksql.streams.default.production.exception.handler" : "io.confluent.ksql.errors.ProductionExceptionHandlerUtil$LogAndFailProductionExceptionHandler", + "ksql.query.pull.interpreter.enabled" : "true", + "ksql.query.pull.limit.clause.enabled" : "true", + "ksql.query.pull.router.thread.pool.size" : "50", + "ksql.query.push.v2.continuation.tokens.enabled" : "false", + "ksql.query.retry.backoff.max.ms" : "900000", + "ksql.timestamp.throw.on.invalid" : "false", + "ksql.persistence.default.format.value" : null, + "ksql.udfs.enabled" : "true", + "ksql.udf.enable.security.manager" : "true", + "ksql.connect.worker.config" : "", + "ksql.nested.error.set.null" : "true", + "ksql.query.pull.thread.pool.size" : "50", + "ksql.persistent.prefix" : "query_", + "ksql.metastore.backup.location" : "", + "ksql.error.classifier.regex" : "", + "ksql.suppress.buffer.size.bytes" : "-1" + } +} \ No newline at end of file diff --git a/ksqldb-functional-tests/src/test/resources/historical_plans/joins_-_stream_stream_inner_join_with_out_of_order_messages_-_PROTOBUF_NOSR/7.3.0_1651071328265/spec.json b/ksqldb-functional-tests/src/test/resources/historical_plans/joins_-_stream_stream_inner_join_with_out_of_order_messages_-_PROTOBUF_NOSR/7.3.0_1651071328265/spec.json new file mode 100644 index 000000000000..ec0c1818899c --- /dev/null +++ b/ksqldb-functional-tests/src/test/resources/historical_plans/joins_-_stream_stream_inner_join_with_out_of_order_messages_-_PROTOBUF_NOSR/7.3.0_1651071328265/spec.json @@ -0,0 +1,275 @@ +{ + "version" : "7.3.0", + "timestamp" : 1651071328265, + "path" : "query-validation-tests/joins.json", + "schemas" : { + "CSAS_INNER_JOIN_0.INNER_JOIN" : { + "schema" : "`T_ID` BIGINT KEY, `NAME` STRING, `VALUE` BIGINT, `F1` STRING, `F2` BIGINT", + "keyFormat" : { + "format" : "KAFKA" + }, + "valueFormat" : { + "format" : "PROTOBUF_NOSR" + } + }, + "CSAS_INNER_JOIN_0.Join.Left" : { + "schema" : "`T_ID` BIGINT KEY, `T_NAME` STRING, `T_VALUE` BIGINT, `T_ROWTIME` BIGINT, `T_ROWPARTITION` INTEGER, `T_ROWOFFSET` BIGINT, `T_ID` BIGINT", + "valueFormat" : { + "format" : "PROTOBUF_NOSR" + } + }, + "CSAS_INNER_JOIN_0.KafkaTopic_Right.Source" : { + "schema" : "`ID` BIGINT KEY, `F1` STRING, `F2` BIGINT", + "keyFormat" : { + "format" : "KAFKA" + }, + "valueFormat" : { + "format" : "PROTOBUF_NOSR" + } + }, + "CSAS_INNER_JOIN_0.Join" : { + "schema" : "`T_ID` BIGINT KEY, `T_NAME` STRING, `T_VALUE` BIGINT, `T_ROWTIME` BIGINT, `T_ROWPARTITION` INTEGER, `T_ROWOFFSET` BIGINT, `T_ID` BIGINT", + "keyFormat" : { + "format" : "KAFKA" + } + }, + "CSAS_INNER_JOIN_0.KafkaTopic_Left.Source" : { + "schema" : "`ID` BIGINT KEY, `NAME` STRING, `VALUE` BIGINT", + "keyFormat" : { + "format" : "KAFKA" + }, + "valueFormat" : { + "format" : "PROTOBUF_NOSR" + } + }, + "CSAS_INNER_JOIN_0.Join.Right" : { + "schema" : "`TT_ID` BIGINT KEY, `TT_F1` STRING, `TT_F2` BIGINT, `TT_ROWTIME` BIGINT, `TT_ROWPARTITION` INTEGER, `TT_ROWOFFSET` BIGINT, `TT_ID` BIGINT", + "valueFormat" : { + "format" : "PROTOBUF_NOSR" + } + } + }, + "testCase" : { + "name" : "stream stream inner join with out of order messages - PROTOBUF_NOSR", + "inputs" : [ { + "topic" : "left_topic", + "key" : 0, + "value" : { + "NAME" : "zero", + "VALUE" : 0 + }, + "timestamp" : 0 + }, { + "topic" : "right_topic", + "key" : 0, + "value" : { + "F1" : "blah", + "F2" : 50 + }, + "timestamp" : 9999 + }, { + "topic" : "left_topic", + "key" : 10, + "value" : { + "NAME" : "100", + "VALUE" : 5 + }, + "timestamp" : 11000 + }, { + "topic" : "left_topic", + "key" : 0, + "value" : { + "NAME" : "foo", + "VALUE" : 100 + }, + "timestamp" : 13000 + }, { + "topic" : "right_topic", + "key" : 0, + "value" : { + "F1" : "a", + "F2" : 10 + }, + "timestamp" : 15000 + }, { + "topic" : "right_topic", + "key" : 100, + "value" : { + "F1" : "newblah", + "F2" : 150 + }, + "timestamp" : 16000 + }, { + "topic" : "left_topic", + "key" : 90, + "value" : { + "NAME" : "ninety", + "VALUE" : 90 + }, + "timestamp" : 17000 + }, { + "topic" : "left_topic", + "key" : 0, + "value" : { + "NAME" : "bar", + "VALUE" : 99 + }, + "timestamp" : 30000 + }, { + "topic" : "left_topic", + "key" : 0, + "value" : { + "NAME" : "late-message", + "VALUE" : 10000 + }, + "timestamp" : 6000 + } ], + "outputs" : [ { + "topic" : "INNER_JOIN", + "key" : 0, + "value" : { + "NAME" : "zero", + "VALUE" : 0, + "F1" : "blah", + "F2" : 50 + }, + "timestamp" : 9999 + }, { + "topic" : "INNER_JOIN", + "key" : 0, + "value" : { + "NAME" : "foo", + "VALUE" : 100, + "F1" : "blah", + "F2" : 50 + }, + "timestamp" : 13000 + }, { + "topic" : "INNER_JOIN", + "key" : 0, + "value" : { + "NAME" : "foo", + "VALUE" : 100, + "F1" : "a", + "F2" : 10 + }, + "timestamp" : 15000 + }, { + "topic" : "INNER_JOIN", + "key" : 0, + "value" : { + "NAME" : "late-message", + "VALUE" : 10000, + "F1" : "blah", + "F2" : 50 + }, + "timestamp" : 9999 + }, { + "topic" : "INNER_JOIN", + "key" : 0, + "value" : { + "NAME" : "late-message", + "VALUE" : 10000, + "F1" : "a", + "F2" : 10 + }, + "timestamp" : 15000 + } ], + "topics" : [ { + "name" : "right_topic", + "replicas" : 1, + "numPartitions" : 4 + }, { + "name" : "INNER_JOIN", + "replicas" : 1, + "numPartitions" : 4 + }, { + "name" : "left_topic", + "replicas" : 1, + "numPartitions" : 4 + } ], + "statements" : [ "CREATE STREAM TEST (ID BIGINT KEY, NAME varchar, VALUE bigint) WITH (kafka_topic='left_topic', value_format='PROTOBUF_NOSR');", "CREATE STREAM TEST_STREAM (ID BIGINT KEY, F1 varchar, F2 bigint) WITH (kafka_topic='right_topic', value_format='PROTOBUF_NOSR');", "CREATE STREAM INNER_JOIN as SELECT t.id, name, value, f1, f2 FROM test t join TEST_STREAM tt WITHIN 10 seconds on t.id = tt.id;" ], + "post" : { + "sources" : [ { + "name" : "INNER_JOIN", + "type" : "STREAM", + "schema" : "`T_ID` BIGINT KEY, `NAME` STRING, `VALUE` BIGINT, `F1` STRING, `F2` BIGINT", + "keyFormat" : { + "format" : "KAFKA" + }, + "valueFormat" : "PROTOBUF_NOSR", + "keyFeatures" : [ ], + "valueFeatures" : [ ], + "isSource" : false + }, { + "name" : "TEST", + "type" : "STREAM", + "schema" : "`ID` BIGINT KEY, `NAME` STRING, `VALUE` BIGINT", + "keyFormat" : { + "format" : "KAFKA" + }, + "valueFormat" : "PROTOBUF_NOSR", + "keyFeatures" : [ ], + "valueFeatures" : [ ], + "isSource" : false + }, { + "name" : "TEST_STREAM", + "type" : "STREAM", + "schema" : "`ID` BIGINT KEY, `F1` STRING, `F2` BIGINT", + "keyFormat" : { + "format" : "KAFKA" + }, + "valueFormat" : "PROTOBUF_NOSR", + "keyFeatures" : [ ], + "valueFeatures" : [ ], + "isSource" : false + } ], + "topics" : { + "topics" : [ { + "name" : "_confluent-ksql-some.ksql.service.idquery_CSAS_INNER_JOIN_0-KSTREAM-JOINTHIS-0000000008-store-changelog", + "keyFormat" : { + "format" : "KAFKA" + }, + "valueFormat" : { + "format" : "PROTOBUF_NOSR" + } + }, { + "name" : "INNER_JOIN", + "keyFormat" : { + "format" : "KAFKA" + }, + "valueFormat" : { + "format" : "PROTOBUF_NOSR" + }, + "partitions" : 4 + }, { + "name" : "_confluent-ksql-some.ksql.service.idquery_CSAS_INNER_JOIN_0-KSTREAM-JOINOTHER-0000000009-store-changelog", + "keyFormat" : { + "format" : "KAFKA" + }, + "valueFormat" : { + "format" : "PROTOBUF_NOSR" + } + }, { + "name" : "left_topic", + "keyFormat" : { + "format" : "KAFKA" + }, + "valueFormat" : { + "format" : "PROTOBUF_NOSR" + }, + "partitions" : 4 + }, { + "name" : "right_topic", + "keyFormat" : { + "format" : "KAFKA" + }, + "valueFormat" : { + "format" : "PROTOBUF_NOSR" + }, + "partitions" : 4 + } ] + } + } + } +} \ No newline at end of file diff --git a/ksqldb-functional-tests/src/test/resources/historical_plans/joins_-_stream_stream_inner_join_with_out_of_order_messages_-_PROTOBUF_NOSR/7.3.0_1651071328265/topology b/ksqldb-functional-tests/src/test/resources/historical_plans/joins_-_stream_stream_inner_join_with_out_of_order_messages_-_PROTOBUF_NOSR/7.3.0_1651071328265/topology new file mode 100644 index 000000000000..942a8e59aba7 --- /dev/null +++ b/ksqldb-functional-tests/src/test/resources/historical_plans/joins_-_stream_stream_inner_join_with_out_of_order_messages_-_PROTOBUF_NOSR/7.3.0_1651071328265/topology @@ -0,0 +1,39 @@ +Topologies: + Sub-topology: 0 + Source: KSTREAM-SOURCE-0000000000 (topics: [left_topic]) + --> KSTREAM-TRANSFORMVALUES-0000000001 + Source: KSTREAM-SOURCE-0000000003 (topics: [right_topic]) + --> KSTREAM-TRANSFORMVALUES-0000000004 + Processor: KSTREAM-TRANSFORMVALUES-0000000001 (stores: []) + --> PrependAliasLeft + <-- KSTREAM-SOURCE-0000000000 + Processor: KSTREAM-TRANSFORMVALUES-0000000004 (stores: []) + --> PrependAliasRight + <-- KSTREAM-SOURCE-0000000003 + Processor: PrependAliasLeft (stores: []) + --> Join-this-windowed + <-- KSTREAM-TRANSFORMVALUES-0000000001 + Processor: PrependAliasRight (stores: []) + --> Join-other-windowed + <-- KSTREAM-TRANSFORMVALUES-0000000004 + Processor: Join-other-windowed (stores: [KSTREAM-JOINOTHER-0000000009-store]) + --> Join-other-join + <-- PrependAliasRight + Processor: Join-this-windowed (stores: [KSTREAM-JOINTHIS-0000000008-store]) + --> Join-this-join + <-- PrependAliasLeft + Processor: Join-other-join (stores: [KSTREAM-JOINTHIS-0000000008-store]) + --> Join-merge + <-- Join-other-windowed + Processor: Join-this-join (stores: [KSTREAM-JOINOTHER-0000000009-store]) + --> Join-merge + <-- Join-this-windowed + Processor: Join-merge (stores: []) + --> Project + <-- Join-this-join, Join-other-join + Processor: Project (stores: []) + --> KSTREAM-SINK-0000000012 + <-- Join-merge + Sink: KSTREAM-SINK-0000000012 (topic: INNER_JOIN) + <-- Project + diff --git a/ksqldb-functional-tests/src/test/resources/historical_plans/joins_-_stream_stream_left_join_-_PROTOBUF_-_PROTOBUF_NOSR/7.3.0_1651071276997/plan.json b/ksqldb-functional-tests/src/test/resources/historical_plans/joins_-_stream_stream_left_join_-_PROTOBUF_-_PROTOBUF_NOSR/7.3.0_1651071276997/plan.json new file mode 100644 index 000000000000..eb75879bbf15 --- /dev/null +++ b/ksqldb-functional-tests/src/test/resources/historical_plans/joins_-_stream_stream_left_join_-_PROTOBUF_-_PROTOBUF_NOSR/7.3.0_1651071276997/plan.json @@ -0,0 +1,296 @@ +{ + "plan" : [ { + "@type" : "ksqlPlanV1", + "statementText" : "CREATE STREAM TEST (ID BIGINT KEY, NAME STRING, VALUE BIGINT) WITH (KAFKA_TOPIC='left_topic', KEY_FORMAT='KAFKA', VALUE_FORMAT='PROTOBUF_NOSR');", + "ddlCommand" : { + "@type" : "createStreamV1", + "sourceName" : "TEST", + "schema" : "`ID` BIGINT KEY, `NAME` STRING, `VALUE` BIGINT", + "timestampColumn" : null, + "topicName" : "left_topic", + "formats" : { + "keyFormat" : { + "format" : "KAFKA", + "properties" : { } + }, + "valueFormat" : { + "format" : "PROTOBUF_NOSR", + "properties" : { } + } + }, + "windowInfo" : null, + "orReplace" : false, + "isSource" : false + }, + "queryPlan" : null + }, { + "@type" : "ksqlPlanV1", + "statementText" : "CREATE STREAM TEST_STREAM (ID BIGINT KEY, F1 STRING, F2 BIGINT) WITH (KAFKA_TOPIC='right_topic', KEY_FORMAT='KAFKA', VALUE_FORMAT='PROTOBUF_NOSR');", + "ddlCommand" : { + "@type" : "createStreamV1", + "sourceName" : "TEST_STREAM", + "schema" : "`ID` BIGINT KEY, `F1` STRING, `F2` BIGINT", + "timestampColumn" : null, + "topicName" : "right_topic", + "formats" : { + "keyFormat" : { + "format" : "KAFKA", + "properties" : { } + }, + "valueFormat" : { + "format" : "PROTOBUF_NOSR", + "properties" : { } + } + }, + "windowInfo" : null, + "orReplace" : false, + "isSource" : false + }, + "queryPlan" : null + }, { + "@type" : "ksqlPlanV1", + "statementText" : "CREATE STREAM OUTPUT AS SELECT\n T.ID T_ID,\n T.NAME NAME,\n T.VALUE VALUE,\n TT.F1 F1,\n TT.F2 F2\nFROM TEST T\nLEFT OUTER JOIN TEST_STREAM TT WITHIN 11 SECONDS ON ((T.ID = TT.ID))\nEMIT CHANGES", + "ddlCommand" : { + "@type" : "createStreamV1", + "sourceName" : "OUTPUT", + "schema" : "`T_ID` BIGINT KEY, `NAME` STRING, `VALUE` BIGINT, `F1` STRING, `F2` BIGINT", + "timestampColumn" : null, + "topicName" : "OUTPUT", + "formats" : { + "keyFormat" : { + "format" : "KAFKA", + "properties" : { } + }, + "valueFormat" : { + "format" : "PROTOBUF_NOSR", + "properties" : { } + } + }, + "windowInfo" : null, + "orReplace" : false, + "isSource" : false + }, + "queryPlan" : { + "sources" : [ "TEST", "TEST_STREAM" ], + "sink" : "OUTPUT", + "physicalPlan" : { + "@type" : "streamSinkV1", + "properties" : { + "queryContext" : "OUTPUT" + }, + "source" : { + "@type" : "streamSelectV1", + "properties" : { + "queryContext" : "Project" + }, + "source" : { + "@type" : "streamStreamJoinV1", + "properties" : { + "queryContext" : "Join" + }, + "joinType" : "LEFT", + "leftInternalFormats" : { + "keyFormat" : { + "format" : "KAFKA", + "properties" : { } + }, + "valueFormat" : { + "format" : "PROTOBUF_NOSR", + "properties" : { } + } + }, + "rightInternalFormats" : { + "keyFormat" : { + "format" : "KAFKA", + "properties" : { } + }, + "valueFormat" : { + "format" : "PROTOBUF_NOSR", + "properties" : { } + } + }, + "leftSource" : { + "@type" : "streamSelectV1", + "properties" : { + "queryContext" : "PrependAliasLeft" + }, + "source" : { + "@type" : "streamSourceV1", + "properties" : { + "queryContext" : "KafkaTopic_Left/Source" + }, + "topicName" : "left_topic", + "formats" : { + "keyFormat" : { + "format" : "KAFKA", + "properties" : { } + }, + "valueFormat" : { + "format" : "PROTOBUF_NOSR", + "properties" : { } + } + }, + "timestampColumn" : null, + "sourceSchema" : "`ID` BIGINT KEY, `NAME` STRING, `VALUE` BIGINT", + "pseudoColumnVersion" : 1 + }, + "keyColumnNames" : [ "T_ID" ], + "selectedKeys" : null, + "selectExpressions" : [ "NAME AS T_NAME", "VALUE AS T_VALUE", "ROWTIME AS T_ROWTIME", "ROWPARTITION AS T_ROWPARTITION", "ROWOFFSET AS T_ROWOFFSET", "ID AS T_ID" ] + }, + "rightSource" : { + "@type" : "streamSelectV1", + "properties" : { + "queryContext" : "PrependAliasRight" + }, + "source" : { + "@type" : "streamSourceV1", + "properties" : { + "queryContext" : "KafkaTopic_Right/Source" + }, + "topicName" : "right_topic", + "formats" : { + "keyFormat" : { + "format" : "KAFKA", + "properties" : { } + }, + "valueFormat" : { + "format" : "PROTOBUF_NOSR", + "properties" : { } + } + }, + "timestampColumn" : null, + "sourceSchema" : "`ID` BIGINT KEY, `F1` STRING, `F2` BIGINT", + "pseudoColumnVersion" : 1 + }, + "keyColumnNames" : [ "TT_ID" ], + "selectedKeys" : null, + "selectExpressions" : [ "F1 AS TT_F1", "F2 AS TT_F2", "ROWTIME AS TT_ROWTIME", "ROWPARTITION AS TT_ROWPARTITION", "ROWOFFSET AS TT_ROWOFFSET", "ID AS TT_ID" ] + }, + "beforeMillis" : 11.000000000, + "afterMillis" : 11.000000000, + "graceMillis" : null, + "keyColName" : "T_ID" + }, + "keyColumnNames" : [ "T_ID" ], + "selectedKeys" : null, + "selectExpressions" : [ "T_NAME AS NAME", "T_VALUE AS VALUE", "TT_F1 AS F1", "TT_F2 AS F2" ] + }, + "formats" : { + "keyFormat" : { + "format" : "KAFKA", + "properties" : { } + }, + "valueFormat" : { + "format" : "PROTOBUF_NOSR", + "properties" : { } + } + }, + "topicName" : "OUTPUT", + "timestampColumn" : null + }, + "queryId" : "CSAS_OUTPUT_0", + "runtimeId" : null + } + } ], + "configs" : { + "ksql.extension.dir" : "ext", + "ksql.streams.cache.max.bytes.buffering" : "0", + "metric.reporters" : "", + "ksql.query.status.running.threshold.seconds" : "300", + "ksql.connect.basic.auth.credentials.reload" : "false", + "ksql.output.topic.name.prefix" : "", + "ksql.query.pull.stream.enabled" : "true", + "ksql.query.push.v2.interpreter.enabled" : "true", + "ksql.queryanonymizer.logs_enabled" : "true", + "ksql.variable.substitution.enable" : "true", + "ksql.streams.shutdown.timeout.ms" : "300000", + "ksql.query.pull.max.allowed.offset.lag" : "9223372036854775807", + "ksql.query.pull.max.qps" : "2147483647", + "ksql.access.validator.enable" : "auto", + "ksql.streams.bootstrap.servers" : "localhost:0", + "ksql.query.pull.metrics.enabled" : "true", + "ksql.metrics.extension" : null, + "ksql.query.push.v2.alos.enabled" : "true", + "ksql.query.push.v2.max.hourly.bandwidth.megabytes" : "2147483647", + "ksql.query.pull.range.scan.enabled" : "true", + "ksql.transient.query.cleanup.service.initial.delay.seconds" : "600", + "ksql.hidden.topics" : "_confluent.*,__confluent.*,_schemas,__consumer_offsets,__transaction_state,connect-configs,connect-offsets,connect-status,connect-statuses", + "ksql.lambdas.enabled" : "true", + "ksql.source.table.materialization.enabled" : "true", + "ksql.query.pull.max.hourly.bandwidth.megabytes" : "2147483647", + "ksql.sink.window.change.log.additional.retention" : "1000000", + "ksql.query.persistent.active.limit" : "2147483647", + "ksql.persistence.wrap.single.values" : null, + "ksql.query.transient.max.bytes.buffering.total" : "-1", + "ksql.connect.basic.auth.credentials.source" : "NONE", + "ksql.schema.registry.url" : "schema_registry.url:0", + "ksql.properties.overrides.denylist" : "", + "ksql.service.id" : "some.ksql.service.id", + "ksql.query.push.v2.max.catchup.consumers" : "5", + "ksql.query.push.v2.enabled" : "false", + "ksql.transient.query.cleanup.service.enable" : "true", + "ksql.query.push.v2.metrics.enabled" : "true", + "ksql.rowpartition.rowoffset.enabled" : "true", + "ksql.streams.commit.interval.ms" : "2000", + "ksql.query.pull.table.scan.enabled" : "true", + "ksql.streams.auto.commit.interval.ms" : "0", + "ksql.streams.topology.optimization" : "all", + "ksql.endpoint.migrate.query" : "true", + "ksql.query.push.v2.registry.installed" : "false", + "ksql.streams.num.stream.threads" : "4", + "ksql.metrics.tags.custom" : "", + "ksql.query.push.v2.catchup.consumer.msg.window" : "50", + "ksql.runtime.feature.shared.enabled" : "false", + "ksql.udf.collect.metrics" : "false", + "ksql.new.query.planner.enabled" : "false", + "ksql.connect.request.headers.plugin" : null, + "ksql.security.extension.class" : null, + "ksql.transient.prefix" : "transient_", + "ksql.headers.columns.enabled" : "true", + "ksql.streams.default.deserialization.exception.handler" : "io.confluent.ksql.errors.LogMetricAndContinueExceptionHandler", + "ksql.connect.request.timeout.ms" : "5000", + "ksql.query.pull.enable.standby.reads" : "false", + "ksql.persistence.default.format.key" : "KAFKA", + "ksql.query.persistent.max.bytes.buffering.total" : "-1", + "ksql.query.error.max.queue.size" : "10", + "ksql.query.cleanup.shutdown.timeout.ms" : "30000", + "ksql.internal.topic.min.insync.replicas" : "1", + "ksql.internal.topic.replicas" : "1", + "ksql.insert.into.values.enabled" : "true", + "ksql.queryanonymizer.cluster_namespace" : null, + "ksql.create.or.replace.enabled" : "true", + "ksql.shared.runtimes.count" : "2", + "ksql.cast.strings.preserve.nulls" : "true", + "ksql.authorization.cache.max.entries" : "10000", + "ksql.pull.queries.enable" : "true", + "ksql.transient.query.cleanup.service.period.seconds" : "600", + "ksql.suppress.enabled" : "false", + "ksql.readonly.topics" : "_confluent.*,__confluent.*,_schemas,__consumer_offsets,__transaction_state,connect-configs,connect-offsets,connect-status,connect-statuses", + "ksql.connect.basic.auth.credentials.file" : "", + "ksql.authorization.cache.expiry.time.secs" : "30", + "ksql.query.retry.backoff.initial.ms" : "15000", + "ksql.query.pull.max.concurrent.requests" : "2147483647", + "ksql.streams.auto.offset.reset" : "earliest", + "ksql.connect.url" : "http://localhost:8083", + "ksql.query.push.v2.new.latest.delay.ms" : "5000", + "ksql.query.push.v2.latest.reset.age.ms" : "30000", + "ksql.streams.default.production.exception.handler" : "io.confluent.ksql.errors.ProductionExceptionHandlerUtil$LogAndFailProductionExceptionHandler", + "ksql.query.pull.interpreter.enabled" : "true", + "ksql.query.pull.limit.clause.enabled" : "true", + "ksql.query.pull.router.thread.pool.size" : "50", + "ksql.query.push.v2.continuation.tokens.enabled" : "false", + "ksql.query.retry.backoff.max.ms" : "900000", + "ksql.timestamp.throw.on.invalid" : "false", + "ksql.persistence.default.format.value" : null, + "ksql.udfs.enabled" : "true", + "ksql.udf.enable.security.manager" : "true", + "ksql.connect.worker.config" : "", + "ksql.nested.error.set.null" : "true", + "ksql.query.pull.thread.pool.size" : "50", + "ksql.persistent.prefix" : "query_", + "ksql.metastore.backup.location" : "", + "ksql.error.classifier.regex" : "", + "ksql.suppress.buffer.size.bytes" : "-1" + } +} \ No newline at end of file diff --git a/ksqldb-functional-tests/src/test/resources/historical_plans/joins_-_stream_stream_left_join_-_PROTOBUF_-_PROTOBUF_NOSR/7.3.0_1651071276997/spec.json b/ksqldb-functional-tests/src/test/resources/historical_plans/joins_-_stream_stream_left_join_-_PROTOBUF_-_PROTOBUF_NOSR/7.3.0_1651071276997/spec.json new file mode 100644 index 000000000000..abb602bab495 --- /dev/null +++ b/ksqldb-functional-tests/src/test/resources/historical_plans/joins_-_stream_stream_left_join_-_PROTOBUF_-_PROTOBUF_NOSR/7.3.0_1651071276997/spec.json @@ -0,0 +1,288 @@ +{ + "version" : "7.3.0", + "timestamp" : 1651071276997, + "path" : "query-validation-tests/joins.json", + "schemas" : { + "CSAS_OUTPUT_0.KafkaTopic_Right.Source" : { + "schema" : "`ID` BIGINT KEY, `F1` STRING, `F2` BIGINT", + "keyFormat" : { + "format" : "KAFKA" + }, + "valueFormat" : { + "format" : "PROTOBUF_NOSR" + } + }, + "CSAS_OUTPUT_0.Join.Left" : { + "schema" : "`T_ID` BIGINT KEY, `T_NAME` STRING, `T_VALUE` BIGINT, `T_ROWTIME` BIGINT, `T_ROWPARTITION` INTEGER, `T_ROWOFFSET` BIGINT, `T_ID` BIGINT", + "valueFormat" : { + "format" : "PROTOBUF_NOSR" + } + }, + "CSAS_OUTPUT_0.OUTPUT" : { + "schema" : "`T_ID` BIGINT KEY, `NAME` STRING, `VALUE` BIGINT, `F1` STRING, `F2` BIGINT", + "keyFormat" : { + "format" : "KAFKA" + }, + "valueFormat" : { + "format" : "PROTOBUF_NOSR" + } + }, + "CSAS_OUTPUT_0.KafkaTopic_Left.Source" : { + "schema" : "`ID` BIGINT KEY, `NAME` STRING, `VALUE` BIGINT", + "keyFormat" : { + "format" : "KAFKA" + }, + "valueFormat" : { + "format" : "PROTOBUF_NOSR" + } + }, + "CSAS_OUTPUT_0.Join" : { + "schema" : "`T_ID` BIGINT KEY, `T_NAME` STRING, `T_VALUE` BIGINT, `T_ROWTIME` BIGINT, `T_ROWPARTITION` INTEGER, `T_ROWOFFSET` BIGINT, `T_ID` BIGINT", + "keyFormat" : { + "format" : "KAFKA" + } + }, + "CSAS_OUTPUT_0.Join.Right" : { + "schema" : "`TT_ID` BIGINT KEY, `TT_F1` STRING, `TT_F2` BIGINT, `TT_ROWTIME` BIGINT, `TT_ROWPARTITION` INTEGER, `TT_ROWOFFSET` BIGINT, `TT_ID` BIGINT", + "valueFormat" : { + "format" : "PROTOBUF_NOSR" + } + } + }, + "testCase" : { + "name" : "stream stream left join - PROTOBUF - PROTOBUF_NOSR", + "inputs" : [ { + "topic" : "left_topic", + "key" : 0, + "value" : { + "NAME" : "zero", + "VALUE" : 0 + }, + "timestamp" : 0 + }, { + "topic" : "right_topic", + "key" : 0, + "value" : { + "F1" : "blah", + "F2" : 50 + }, + "timestamp" : 10000 + }, { + "topic" : "left_topic", + "key" : 10, + "value" : { + "NAME" : "100", + "VALUE" : 5 + }, + "timestamp" : 11000 + }, { + "topic" : "left_topic", + "key" : 0, + "value" : { + "NAME" : "foo", + "VALUE" : 100 + }, + "timestamp" : 13000 + }, { + "topic" : "right_topic", + "key" : 0, + "value" : { + "F1" : "a", + "F2" : 10 + }, + "timestamp" : 15000 + }, { + "topic" : "right_topic", + "key" : 100, + "value" : { + "F1" : "newblah", + "F2" : 150 + }, + "timestamp" : 16000 + }, { + "topic" : "left_topic", + "key" : 90, + "value" : { + "NAME" : "ninety", + "VALUE" : 90 + }, + "timestamp" : 17000 + }, { + "topic" : "left_topic", + "key" : 0, + "value" : { + "NAME" : "bar", + "VALUE" : 99 + }, + "timestamp" : 30000 + } ], + "outputs" : [ { + "topic" : "OUTPUT", + "key" : 0, + "value" : { + "NAME" : "zero", + "VALUE" : 0, + "F1" : "", + "F2" : 0 + }, + "timestamp" : 0 + }, { + "topic" : "OUTPUT", + "key" : 0, + "value" : { + "NAME" : "zero", + "VALUE" : 0, + "F1" : "blah", + "F2" : 50 + }, + "timestamp" : 10000 + }, { + "topic" : "OUTPUT", + "key" : 10, + "value" : { + "NAME" : "100", + "VALUE" : 5, + "F1" : "", + "F2" : 0 + }, + "timestamp" : 11000 + }, { + "topic" : "OUTPUT", + "key" : 0, + "value" : { + "NAME" : "foo", + "VALUE" : 100, + "F1" : "blah", + "F2" : 50 + }, + "timestamp" : 13000 + }, { + "topic" : "OUTPUT", + "key" : 0, + "value" : { + "NAME" : "foo", + "VALUE" : 100, + "F1" : "a", + "F2" : 10 + }, + "timestamp" : 15000 + }, { + "topic" : "OUTPUT", + "key" : 90, + "value" : { + "NAME" : "ninety", + "VALUE" : 90, + "F1" : "", + "F2" : 0 + }, + "timestamp" : 17000 + }, { + "topic" : "OUTPUT", + "key" : 0, + "value" : { + "NAME" : "bar", + "VALUE" : 99, + "F1" : "", + "F2" : 0 + }, + "timestamp" : 30000 + } ], + "topics" : [ { + "name" : "right_topic", + "replicas" : 1, + "numPartitions" : 4 + }, { + "name" : "OUTPUT", + "replicas" : 1, + "numPartitions" : 4 + }, { + "name" : "left_topic", + "replicas" : 1, + "numPartitions" : 4 + } ], + "statements" : [ "CREATE STREAM TEST (ID BIGINT KEY, NAME varchar, VALUE bigint) WITH (kafka_topic='left_topic', value_format='PROTOBUF_NOSR');", "CREATE STREAM TEST_STREAM (ID BIGINT KEY, F1 varchar, F2 bigint) WITH (kafka_topic='right_topic', value_format='PROTOBUF_NOSR');", "CREATE STREAM OUTPUT as SELECT t.id, name, value, f1, f2 FROM test t left join TEST_STREAM tt WITHIN 11 seconds ON t.id = tt.id;" ], + "post" : { + "sources" : [ { + "name" : "OUTPUT", + "type" : "STREAM", + "schema" : "`T_ID` BIGINT KEY, `NAME` STRING, `VALUE` BIGINT, `F1` STRING, `F2` BIGINT", + "keyFormat" : { + "format" : "KAFKA" + }, + "valueFormat" : "PROTOBUF_NOSR", + "keyFeatures" : [ ], + "valueFeatures" : [ ], + "isSource" : false + }, { + "name" : "TEST", + "type" : "STREAM", + "schema" : "`ID` BIGINT KEY, `NAME` STRING, `VALUE` BIGINT", + "keyFormat" : { + "format" : "KAFKA" + }, + "valueFormat" : "PROTOBUF_NOSR", + "keyFeatures" : [ ], + "valueFeatures" : [ ], + "isSource" : false + }, { + "name" : "TEST_STREAM", + "type" : "STREAM", + "schema" : "`ID` BIGINT KEY, `F1` STRING, `F2` BIGINT", + "keyFormat" : { + "format" : "KAFKA" + }, + "valueFormat" : "PROTOBUF_NOSR", + "keyFeatures" : [ ], + "valueFeatures" : [ ], + "isSource" : false + } ], + "topics" : { + "topics" : [ { + "name" : "_confluent-ksql-some.ksql.service.idquery_CSAS_OUTPUT_0-KSTREAM-JOINTHIS-0000000008-store-changelog", + "keyFormat" : { + "format" : "KAFKA" + }, + "valueFormat" : { + "format" : "PROTOBUF_NOSR" + } + }, { + "name" : "left_topic", + "keyFormat" : { + "format" : "KAFKA" + }, + "valueFormat" : { + "format" : "PROTOBUF_NOSR" + }, + "partitions" : 4 + }, { + "name" : "OUTPUT", + "keyFormat" : { + "format" : "KAFKA" + }, + "valueFormat" : { + "format" : "PROTOBUF_NOSR" + }, + "partitions" : 4 + }, { + "name" : "right_topic", + "keyFormat" : { + "format" : "KAFKA" + }, + "valueFormat" : { + "format" : "PROTOBUF_NOSR" + }, + "partitions" : 4 + }, { + "name" : "_confluent-ksql-some.ksql.service.idquery_CSAS_OUTPUT_0-KSTREAM-OUTEROTHER-0000000009-store-changelog", + "keyFormat" : { + "format" : "KAFKA" + }, + "valueFormat" : { + "format" : "PROTOBUF_NOSR" + } + } ], + "blackList" : ".*-repartition" + } + } + } +} \ No newline at end of file diff --git a/ksqldb-functional-tests/src/test/resources/historical_plans/joins_-_stream_stream_left_join_-_PROTOBUF_-_PROTOBUF_NOSR/7.3.0_1651071276997/topology b/ksqldb-functional-tests/src/test/resources/historical_plans/joins_-_stream_stream_left_join_-_PROTOBUF_-_PROTOBUF_NOSR/7.3.0_1651071276997/topology new file mode 100644 index 000000000000..301f2f8fd103 --- /dev/null +++ b/ksqldb-functional-tests/src/test/resources/historical_plans/joins_-_stream_stream_left_join_-_PROTOBUF_-_PROTOBUF_NOSR/7.3.0_1651071276997/topology @@ -0,0 +1,39 @@ +Topologies: + Sub-topology: 0 + Source: KSTREAM-SOURCE-0000000000 (topics: [left_topic]) + --> KSTREAM-TRANSFORMVALUES-0000000001 + Source: KSTREAM-SOURCE-0000000003 (topics: [right_topic]) + --> KSTREAM-TRANSFORMVALUES-0000000004 + Processor: KSTREAM-TRANSFORMVALUES-0000000001 (stores: []) + --> PrependAliasLeft + <-- KSTREAM-SOURCE-0000000000 + Processor: KSTREAM-TRANSFORMVALUES-0000000004 (stores: []) + --> PrependAliasRight + <-- KSTREAM-SOURCE-0000000003 + Processor: PrependAliasLeft (stores: []) + --> Join-this-windowed + <-- KSTREAM-TRANSFORMVALUES-0000000001 + Processor: PrependAliasRight (stores: []) + --> Join-other-windowed + <-- KSTREAM-TRANSFORMVALUES-0000000004 + Processor: Join-other-windowed (stores: [KSTREAM-OUTEROTHER-0000000009-store]) + --> Join-outer-other-join + <-- PrependAliasRight + Processor: Join-this-windowed (stores: [KSTREAM-JOINTHIS-0000000008-store]) + --> Join-this-join + <-- PrependAliasLeft + Processor: Join-outer-other-join (stores: [KSTREAM-JOINTHIS-0000000008-store]) + --> Join-merge + <-- Join-other-windowed + Processor: Join-this-join (stores: [KSTREAM-OUTEROTHER-0000000009-store]) + --> Join-merge + <-- Join-this-windowed + Processor: Join-merge (stores: []) + --> Project + <-- Join-this-join, Join-outer-other-join + Processor: Project (stores: []) + --> KSTREAM-SINK-0000000012 + <-- Join-merge + Sink: KSTREAM-SINK-0000000012 (topic: OUTPUT) + <-- Project + diff --git a/ksqldb-functional-tests/src/test/resources/historical_plans/joins_-_stream_stream_left_join_-_rekey_-_PROTOBUF_NOSR/7.3.0_1651071286065/plan.json b/ksqldb-functional-tests/src/test/resources/historical_plans/joins_-_stream_stream_left_join_-_rekey_-_PROTOBUF_NOSR/7.3.0_1651071286065/plan.json new file mode 100644 index 000000000000..3a06b25bff1b --- /dev/null +++ b/ksqldb-functional-tests/src/test/resources/historical_plans/joins_-_stream_stream_left_join_-_rekey_-_PROTOBUF_NOSR/7.3.0_1651071286065/plan.json @@ -0,0 +1,310 @@ +{ + "plan" : [ { + "@type" : "ksqlPlanV1", + "statementText" : "CREATE STREAM TEST (K STRING KEY, ID BIGINT, NAME STRING, VALUE BIGINT) WITH (KAFKA_TOPIC='left_topic', KEY_FORMAT='KAFKA', VALUE_FORMAT='PROTOBUF_NOSR');", + "ddlCommand" : { + "@type" : "createStreamV1", + "sourceName" : "TEST", + "schema" : "`K` STRING KEY, `ID` BIGINT, `NAME` STRING, `VALUE` BIGINT", + "timestampColumn" : null, + "topicName" : "left_topic", + "formats" : { + "keyFormat" : { + "format" : "KAFKA", + "properties" : { } + }, + "valueFormat" : { + "format" : "PROTOBUF_NOSR", + "properties" : { } + } + }, + "windowInfo" : null, + "orReplace" : false, + "isSource" : false + }, + "queryPlan" : null + }, { + "@type" : "ksqlPlanV1", + "statementText" : "CREATE STREAM TEST_STREAM (K STRING KEY, ID BIGINT, F1 STRING, F2 BIGINT) WITH (KAFKA_TOPIC='right_topic', KEY_FORMAT='KAFKA', VALUE_FORMAT='PROTOBUF_NOSR');", + "ddlCommand" : { + "@type" : "createStreamV1", + "sourceName" : "TEST_STREAM", + "schema" : "`K` STRING KEY, `ID` BIGINT, `F1` STRING, `F2` BIGINT", + "timestampColumn" : null, + "topicName" : "right_topic", + "formats" : { + "keyFormat" : { + "format" : "KAFKA", + "properties" : { } + }, + "valueFormat" : { + "format" : "PROTOBUF_NOSR", + "properties" : { } + } + }, + "windowInfo" : null, + "orReplace" : false, + "isSource" : false + }, + "queryPlan" : null + }, { + "@type" : "ksqlPlanV1", + "statementText" : "CREATE STREAM OUTPUT AS SELECT\n T.ID T_ID,\n T.NAME NAME,\n T.VALUE VALUE,\n TT.F1 F1,\n TT.F2 F2\nFROM TEST T\nLEFT OUTER JOIN TEST_STREAM TT WITHIN 11 SECONDS ON ((T.ID = TT.ID))\nEMIT CHANGES", + "ddlCommand" : { + "@type" : "createStreamV1", + "sourceName" : "OUTPUT", + "schema" : "`T_ID` BIGINT KEY, `NAME` STRING, `VALUE` BIGINT, `F1` STRING, `F2` BIGINT", + "timestampColumn" : null, + "topicName" : "OUTPUT", + "formats" : { + "keyFormat" : { + "format" : "KAFKA", + "properties" : { } + }, + "valueFormat" : { + "format" : "PROTOBUF_NOSR", + "properties" : { } + } + }, + "windowInfo" : null, + "orReplace" : false, + "isSource" : false + }, + "queryPlan" : { + "sources" : [ "TEST", "TEST_STREAM" ], + "sink" : "OUTPUT", + "physicalPlan" : { + "@type" : "streamSinkV1", + "properties" : { + "queryContext" : "OUTPUT" + }, + "source" : { + "@type" : "streamSelectV1", + "properties" : { + "queryContext" : "Project" + }, + "source" : { + "@type" : "streamStreamJoinV1", + "properties" : { + "queryContext" : "Join" + }, + "joinType" : "LEFT", + "leftInternalFormats" : { + "keyFormat" : { + "format" : "KAFKA", + "properties" : { } + }, + "valueFormat" : { + "format" : "PROTOBUF_NOSR", + "properties" : { } + } + }, + "rightInternalFormats" : { + "keyFormat" : { + "format" : "KAFKA", + "properties" : { } + }, + "valueFormat" : { + "format" : "PROTOBUF_NOSR", + "properties" : { } + } + }, + "leftSource" : { + "@type" : "streamSelectV1", + "properties" : { + "queryContext" : "PrependAliasLeft" + }, + "source" : { + "@type" : "streamSelectKeyV2", + "properties" : { + "queryContext" : "LeftSourceKeyed" + }, + "source" : { + "@type" : "streamSourceV1", + "properties" : { + "queryContext" : "KafkaTopic_Left/Source" + }, + "topicName" : "left_topic", + "formats" : { + "keyFormat" : { + "format" : "KAFKA", + "properties" : { } + }, + "valueFormat" : { + "format" : "PROTOBUF_NOSR", + "properties" : { } + } + }, + "timestampColumn" : null, + "sourceSchema" : "`K` STRING KEY, `ID` BIGINT, `NAME` STRING, `VALUE` BIGINT", + "pseudoColumnVersion" : 1 + }, + "keyExpression" : [ "ID" ] + }, + "keyColumnNames" : [ "T_ID" ], + "selectedKeys" : null, + "selectExpressions" : [ "ID AS T_ID", "NAME AS T_NAME", "VALUE AS T_VALUE", "ROWTIME AS T_ROWTIME", "ROWPARTITION AS T_ROWPARTITION", "ROWOFFSET AS T_ROWOFFSET", "K AS T_K" ] + }, + "rightSource" : { + "@type" : "streamSelectV1", + "properties" : { + "queryContext" : "PrependAliasRight" + }, + "source" : { + "@type" : "streamSelectKeyV2", + "properties" : { + "queryContext" : "RightSourceKeyed" + }, + "source" : { + "@type" : "streamSourceV1", + "properties" : { + "queryContext" : "KafkaTopic_Right/Source" + }, + "topicName" : "right_topic", + "formats" : { + "keyFormat" : { + "format" : "KAFKA", + "properties" : { } + }, + "valueFormat" : { + "format" : "PROTOBUF_NOSR", + "properties" : { } + } + }, + "timestampColumn" : null, + "sourceSchema" : "`K` STRING KEY, `ID` BIGINT, `F1` STRING, `F2` BIGINT", + "pseudoColumnVersion" : 1 + }, + "keyExpression" : [ "ID" ] + }, + "keyColumnNames" : [ "TT_ID" ], + "selectedKeys" : null, + "selectExpressions" : [ "ID AS TT_ID", "F1 AS TT_F1", "F2 AS TT_F2", "ROWTIME AS TT_ROWTIME", "ROWPARTITION AS TT_ROWPARTITION", "ROWOFFSET AS TT_ROWOFFSET", "K AS TT_K" ] + }, + "beforeMillis" : 11.000000000, + "afterMillis" : 11.000000000, + "graceMillis" : null, + "keyColName" : "T_ID" + }, + "keyColumnNames" : [ "T_ID" ], + "selectedKeys" : null, + "selectExpressions" : [ "T_NAME AS NAME", "T_VALUE AS VALUE", "TT_F1 AS F1", "TT_F2 AS F2" ] + }, + "formats" : { + "keyFormat" : { + "format" : "KAFKA", + "properties" : { } + }, + "valueFormat" : { + "format" : "PROTOBUF_NOSR", + "properties" : { } + } + }, + "topicName" : "OUTPUT", + "timestampColumn" : null + }, + "queryId" : "CSAS_OUTPUT_0", + "runtimeId" : null + } + } ], + "configs" : { + "ksql.extension.dir" : "ext", + "ksql.streams.cache.max.bytes.buffering" : "0", + "metric.reporters" : "", + "ksql.query.status.running.threshold.seconds" : "300", + "ksql.connect.basic.auth.credentials.reload" : "false", + "ksql.output.topic.name.prefix" : "", + "ksql.query.pull.stream.enabled" : "true", + "ksql.query.push.v2.interpreter.enabled" : "true", + "ksql.queryanonymizer.logs_enabled" : "true", + "ksql.variable.substitution.enable" : "true", + "ksql.streams.shutdown.timeout.ms" : "300000", + "ksql.query.pull.max.allowed.offset.lag" : "9223372036854775807", + "ksql.query.pull.max.qps" : "2147483647", + "ksql.access.validator.enable" : "auto", + "ksql.streams.bootstrap.servers" : "localhost:0", + "ksql.query.pull.metrics.enabled" : "true", + "ksql.metrics.extension" : null, + "ksql.query.push.v2.alos.enabled" : "true", + "ksql.query.push.v2.max.hourly.bandwidth.megabytes" : "2147483647", + "ksql.query.pull.range.scan.enabled" : "true", + "ksql.transient.query.cleanup.service.initial.delay.seconds" : "600", + "ksql.hidden.topics" : "_confluent.*,__confluent.*,_schemas,__consumer_offsets,__transaction_state,connect-configs,connect-offsets,connect-status,connect-statuses", + "ksql.lambdas.enabled" : "true", + "ksql.source.table.materialization.enabled" : "true", + "ksql.query.pull.max.hourly.bandwidth.megabytes" : "2147483647", + "ksql.sink.window.change.log.additional.retention" : "1000000", + "ksql.query.persistent.active.limit" : "2147483647", + "ksql.persistence.wrap.single.values" : null, + "ksql.query.transient.max.bytes.buffering.total" : "-1", + "ksql.connect.basic.auth.credentials.source" : "NONE", + "ksql.schema.registry.url" : "schema_registry.url:0", + "ksql.properties.overrides.denylist" : "", + "ksql.service.id" : "some.ksql.service.id", + "ksql.query.push.v2.max.catchup.consumers" : "5", + "ksql.query.push.v2.enabled" : "false", + "ksql.transient.query.cleanup.service.enable" : "true", + "ksql.query.push.v2.metrics.enabled" : "true", + "ksql.rowpartition.rowoffset.enabled" : "true", + "ksql.streams.commit.interval.ms" : "2000", + "ksql.query.pull.table.scan.enabled" : "true", + "ksql.streams.auto.commit.interval.ms" : "0", + "ksql.streams.topology.optimization" : "all", + "ksql.endpoint.migrate.query" : "true", + "ksql.query.push.v2.registry.installed" : "false", + "ksql.streams.num.stream.threads" : "4", + "ksql.metrics.tags.custom" : "", + "ksql.query.push.v2.catchup.consumer.msg.window" : "50", + "ksql.runtime.feature.shared.enabled" : "false", + "ksql.udf.collect.metrics" : "false", + "ksql.new.query.planner.enabled" : "false", + "ksql.connect.request.headers.plugin" : null, + "ksql.security.extension.class" : null, + "ksql.transient.prefix" : "transient_", + "ksql.headers.columns.enabled" : "true", + "ksql.streams.default.deserialization.exception.handler" : "io.confluent.ksql.errors.LogMetricAndContinueExceptionHandler", + "ksql.connect.request.timeout.ms" : "5000", + "ksql.query.pull.enable.standby.reads" : "false", + "ksql.persistence.default.format.key" : "KAFKA", + "ksql.query.persistent.max.bytes.buffering.total" : "-1", + "ksql.query.error.max.queue.size" : "10", + "ksql.query.cleanup.shutdown.timeout.ms" : "30000", + "ksql.internal.topic.min.insync.replicas" : "1", + "ksql.internal.topic.replicas" : "1", + "ksql.insert.into.values.enabled" : "true", + "ksql.queryanonymizer.cluster_namespace" : null, + "ksql.create.or.replace.enabled" : "true", + "ksql.shared.runtimes.count" : "2", + "ksql.cast.strings.preserve.nulls" : "true", + "ksql.authorization.cache.max.entries" : "10000", + "ksql.pull.queries.enable" : "true", + "ksql.transient.query.cleanup.service.period.seconds" : "600", + "ksql.suppress.enabled" : "false", + "ksql.readonly.topics" : "_confluent.*,__confluent.*,_schemas,__consumer_offsets,__transaction_state,connect-configs,connect-offsets,connect-status,connect-statuses", + "ksql.connect.basic.auth.credentials.file" : "", + "ksql.authorization.cache.expiry.time.secs" : "30", + "ksql.query.retry.backoff.initial.ms" : "15000", + "ksql.query.pull.max.concurrent.requests" : "2147483647", + "ksql.streams.auto.offset.reset" : "earliest", + "ksql.connect.url" : "http://localhost:8083", + "ksql.query.push.v2.new.latest.delay.ms" : "5000", + "ksql.query.push.v2.latest.reset.age.ms" : "30000", + "ksql.streams.default.production.exception.handler" : "io.confluent.ksql.errors.ProductionExceptionHandlerUtil$LogAndFailProductionExceptionHandler", + "ksql.query.pull.interpreter.enabled" : "true", + "ksql.query.pull.limit.clause.enabled" : "true", + "ksql.query.pull.router.thread.pool.size" : "50", + "ksql.query.push.v2.continuation.tokens.enabled" : "false", + "ksql.query.retry.backoff.max.ms" : "900000", + "ksql.timestamp.throw.on.invalid" : "false", + "ksql.persistence.default.format.value" : null, + "ksql.udfs.enabled" : "true", + "ksql.udf.enable.security.manager" : "true", + "ksql.connect.worker.config" : "", + "ksql.nested.error.set.null" : "true", + "ksql.query.pull.thread.pool.size" : "50", + "ksql.persistent.prefix" : "query_", + "ksql.metastore.backup.location" : "", + "ksql.error.classifier.regex" : "", + "ksql.suppress.buffer.size.bytes" : "-1" + } +} \ No newline at end of file diff --git a/ksqldb-functional-tests/src/test/resources/historical_plans/joins_-_stream_stream_left_join_-_rekey_-_PROTOBUF_NOSR/7.3.0_1651071286065/spec.json b/ksqldb-functional-tests/src/test/resources/historical_plans/joins_-_stream_stream_left_join_-_rekey_-_PROTOBUF_NOSR/7.3.0_1651071286065/spec.json new file mode 100644 index 000000000000..62b130f7e038 --- /dev/null +++ b/ksqldb-functional-tests/src/test/resources/historical_plans/joins_-_stream_stream_left_join_-_rekey_-_PROTOBUF_NOSR/7.3.0_1651071286065/spec.json @@ -0,0 +1,311 @@ +{ + "version" : "7.3.0", + "timestamp" : 1651071286065, + "path" : "query-validation-tests/joins.json", + "schemas" : { + "CSAS_OUTPUT_0.KafkaTopic_Right.Source" : { + "schema" : "`K` STRING KEY, `ID` BIGINT, `F1` STRING, `F2` BIGINT", + "keyFormat" : { + "format" : "KAFKA" + }, + "valueFormat" : { + "format" : "PROTOBUF_NOSR" + } + }, + "CSAS_OUTPUT_0.Join.Left" : { + "schema" : "`T_ID` BIGINT KEY, `T_ID` BIGINT, `T_NAME` STRING, `T_VALUE` BIGINT, `T_ROWTIME` BIGINT, `T_ROWPARTITION` INTEGER, `T_ROWOFFSET` BIGINT, `T_K` STRING", + "valueFormat" : { + "format" : "PROTOBUF_NOSR" + } + }, + "CSAS_OUTPUT_0.OUTPUT" : { + "schema" : "`T_ID` BIGINT KEY, `NAME` STRING, `VALUE` BIGINT, `F1` STRING, `F2` BIGINT", + "keyFormat" : { + "format" : "KAFKA" + }, + "valueFormat" : { + "format" : "PROTOBUF_NOSR" + } + }, + "CSAS_OUTPUT_0.KafkaTopic_Left.Source" : { + "schema" : "`K` STRING KEY, `ID` BIGINT, `NAME` STRING, `VALUE` BIGINT", + "keyFormat" : { + "format" : "KAFKA" + }, + "valueFormat" : { + "format" : "PROTOBUF_NOSR" + } + }, + "CSAS_OUTPUT_0.Join" : { + "schema" : "`T_ID` BIGINT KEY, `T_ID` BIGINT, `T_NAME` STRING, `T_VALUE` BIGINT, `T_ROWTIME` BIGINT, `T_ROWPARTITION` INTEGER, `T_ROWOFFSET` BIGINT, `T_K` STRING", + "keyFormat" : { + "format" : "KAFKA" + } + }, + "CSAS_OUTPUT_0.Join.Right" : { + "schema" : "`TT_ID` BIGINT KEY, `TT_ID` BIGINT, `TT_F1` STRING, `TT_F2` BIGINT, `TT_ROWTIME` BIGINT, `TT_ROWPARTITION` INTEGER, `TT_ROWOFFSET` BIGINT, `TT_K` STRING", + "valueFormat" : { + "format" : "PROTOBUF_NOSR" + } + } + }, + "testCase" : { + "name" : "stream stream left join - rekey - PROTOBUF_NOSR", + "inputs" : [ { + "topic" : "left_topic", + "key" : null, + "value" : { + "ID" : 0, + "NAME" : "zero", + "VALUE" : 0 + }, + "timestamp" : 0 + }, { + "topic" : "right_topic", + "key" : null, + "value" : { + "ID" : 0, + "F1" : "blah", + "F2" : 50 + }, + "timestamp" : 10000 + }, { + "topic" : "left_topic", + "key" : null, + "value" : { + "ID" : 10, + "NAME" : "100", + "VALUE" : 5 + }, + "timestamp" : 11000 + }, { + "topic" : "left_topic", + "key" : null, + "value" : { + "ID" : 0, + "NAME" : "foo", + "VALUE" : 100 + }, + "timestamp" : 13000 + }, { + "topic" : "right_topic", + "key" : null, + "value" : { + "ID" : 0, + "F1" : "a", + "F2" : 10 + }, + "timestamp" : 15000 + }, { + "topic" : "right_topic", + "key" : null, + "value" : { + "ID" : 100, + "F1" : "newblah", + "F2" : 150 + }, + "timestamp" : 16000 + }, { + "topic" : "left_topic", + "key" : null, + "value" : { + "ID" : 90, + "NAME" : "ninety", + "VALUE" : 90 + }, + "timestamp" : 17000 + }, { + "topic" : "left_topic", + "key" : null, + "value" : { + "ID" : 0, + "NAME" : "bar", + "VALUE" : 99 + }, + "timestamp" : 30000 + } ], + "outputs" : [ { + "topic" : "OUTPUT", + "key" : 0, + "value" : { + "NAME" : "zero", + "VALUE" : 0, + "F1" : "", + "F2" : 0 + }, + "timestamp" : 0 + }, { + "topic" : "OUTPUT", + "key" : 0, + "value" : { + "NAME" : "zero", + "VALUE" : 0, + "F1" : "blah", + "F2" : 50 + }, + "timestamp" : 10000 + }, { + "topic" : "OUTPUT", + "key" : 10, + "value" : { + "NAME" : "100", + "VALUE" : 5, + "F1" : "", + "F2" : 0 + }, + "timestamp" : 11000 + }, { + "topic" : "OUTPUT", + "key" : 0, + "value" : { + "NAME" : "foo", + "VALUE" : 100, + "F1" : "blah", + "F2" : 50 + }, + "timestamp" : 13000 + }, { + "topic" : "OUTPUT", + "key" : 0, + "value" : { + "NAME" : "foo", + "VALUE" : 100, + "F1" : "a", + "F2" : 10 + }, + "timestamp" : 15000 + }, { + "topic" : "OUTPUT", + "key" : 90, + "value" : { + "NAME" : "ninety", + "VALUE" : 90, + "F1" : "", + "F2" : 0 + }, + "timestamp" : 17000 + }, { + "topic" : "OUTPUT", + "key" : 0, + "value" : { + "NAME" : "bar", + "VALUE" : 99, + "F1" : "", + "F2" : 0 + }, + "timestamp" : 30000 + } ], + "topics" : [ { + "name" : "right_topic", + "replicas" : 1, + "numPartitions" : 4 + }, { + "name" : "OUTPUT", + "replicas" : 1, + "numPartitions" : 4 + }, { + "name" : "left_topic", + "replicas" : 1, + "numPartitions" : 4 + } ], + "statements" : [ "CREATE STREAM TEST (K STRING KEY, ID bigint, NAME varchar, VALUE bigint) WITH (kafka_topic='left_topic', value_format='PROTOBUF_NOSR');", "CREATE STREAM TEST_STREAM (K STRING KEY, ID bigint, F1 varchar, F2 bigint) WITH (kafka_topic='right_topic', value_format='PROTOBUF_NOSR');", "CREATE STREAM OUTPUT as SELECT t.id, name, value, f1, f2 FROM test t left join TEST_STREAM tt WITHIN 11 seconds ON t.id = tt.id;" ], + "post" : { + "sources" : [ { + "name" : "OUTPUT", + "type" : "STREAM", + "schema" : "`T_ID` BIGINT KEY, `NAME` STRING, `VALUE` BIGINT, `F1` STRING, `F2` BIGINT", + "keyFormat" : { + "format" : "KAFKA" + }, + "valueFormat" : "PROTOBUF_NOSR", + "keyFeatures" : [ ], + "valueFeatures" : [ ], + "isSource" : false + }, { + "name" : "TEST", + "type" : "STREAM", + "schema" : "`K` STRING KEY, `ID` BIGINT, `NAME` STRING, `VALUE` BIGINT", + "keyFormat" : { + "format" : "KAFKA" + }, + "valueFormat" : "PROTOBUF_NOSR", + "keyFeatures" : [ ], + "valueFeatures" : [ ], + "isSource" : false + }, { + "name" : "TEST_STREAM", + "type" : "STREAM", + "schema" : "`K` STRING KEY, `ID` BIGINT, `F1` STRING, `F2` BIGINT", + "keyFormat" : { + "format" : "KAFKA" + }, + "valueFormat" : "PROTOBUF_NOSR", + "keyFeatures" : [ ], + "valueFeatures" : [ ], + "isSource" : false + } ], + "topics" : { + "topics" : [ { + "name" : "_confluent-ksql-some.ksql.service.idquery_CSAS_OUTPUT_0-Join-left-repartition", + "keyFormat" : { + "format" : "KAFKA" + }, + "valueFormat" : { + "format" : "PROTOBUF_NOSR" + } + }, { + "name" : "_confluent-ksql-some.ksql.service.idquery_CSAS_OUTPUT_0-Join-right-repartition", + "keyFormat" : { + "format" : "KAFKA" + }, + "valueFormat" : { + "format" : "PROTOBUF_NOSR" + } + }, { + "name" : "_confluent-ksql-some.ksql.service.idquery_CSAS_OUTPUT_0-KSTREAM-JOINTHIS-0000000016-store-changelog", + "keyFormat" : { + "format" : "KAFKA" + }, + "valueFormat" : { + "format" : "PROTOBUF_NOSR" + } + }, { + "name" : "left_topic", + "keyFormat" : { + "format" : "KAFKA" + }, + "valueFormat" : { + "format" : "PROTOBUF_NOSR" + }, + "partitions" : 4 + }, { + "name" : "_confluent-ksql-some.ksql.service.idquery_CSAS_OUTPUT_0-KSTREAM-OUTEROTHER-0000000017-store-changelog", + "keyFormat" : { + "format" : "KAFKA" + }, + "valueFormat" : { + "format" : "PROTOBUF_NOSR" + } + }, { + "name" : "OUTPUT", + "keyFormat" : { + "format" : "KAFKA" + }, + "valueFormat" : { + "format" : "PROTOBUF_NOSR" + }, + "partitions" : 4 + }, { + "name" : "right_topic", + "keyFormat" : { + "format" : "KAFKA" + }, + "valueFormat" : { + "format" : "PROTOBUF_NOSR" + }, + "partitions" : 4 + } ] + } + } + } +} \ No newline at end of file diff --git a/ksqldb-functional-tests/src/test/resources/historical_plans/joins_-_stream_stream_left_join_-_rekey_-_PROTOBUF_NOSR/7.3.0_1651071286065/topology b/ksqldb-functional-tests/src/test/resources/historical_plans/joins_-_stream_stream_left_join_-_rekey_-_PROTOBUF_NOSR/7.3.0_1651071286065/topology new file mode 100644 index 000000000000..4c5f7fa6ca95 --- /dev/null +++ b/ksqldb-functional-tests/src/test/resources/historical_plans/joins_-_stream_stream_left_join_-_rekey_-_PROTOBUF_NOSR/7.3.0_1651071286065/topology @@ -0,0 +1,63 @@ +Topologies: + Sub-topology: 0 + Source: KSTREAM-SOURCE-0000000000 (topics: [left_topic]) + --> KSTREAM-TRANSFORMVALUES-0000000001 + Processor: KSTREAM-TRANSFORMVALUES-0000000001 (stores: []) + --> LeftSourceKeyed-SelectKey + <-- KSTREAM-SOURCE-0000000000 + Processor: LeftSourceKeyed-SelectKey (stores: []) + --> PrependAliasLeft + <-- KSTREAM-TRANSFORMVALUES-0000000001 + Processor: PrependAliasLeft (stores: []) + --> Join-left-repartition-filter + <-- LeftSourceKeyed-SelectKey + Processor: Join-left-repartition-filter (stores: []) + --> Join-left-repartition-sink + <-- PrependAliasLeft + Sink: Join-left-repartition-sink (topic: Join-left-repartition) + <-- Join-left-repartition-filter + + Sub-topology: 1 + Source: KSTREAM-SOURCE-0000000004 (topics: [right_topic]) + --> KSTREAM-TRANSFORMVALUES-0000000005 + Processor: KSTREAM-TRANSFORMVALUES-0000000005 (stores: []) + --> RightSourceKeyed-SelectKey + <-- KSTREAM-SOURCE-0000000004 + Processor: RightSourceKeyed-SelectKey (stores: []) + --> PrependAliasRight + <-- KSTREAM-TRANSFORMVALUES-0000000005 + Processor: PrependAliasRight (stores: []) + --> Join-right-repartition-filter + <-- RightSourceKeyed-SelectKey + Processor: Join-right-repartition-filter (stores: []) + --> Join-right-repartition-sink + <-- PrependAliasRight + Sink: Join-right-repartition-sink (topic: Join-right-repartition) + <-- Join-right-repartition-filter + + Sub-topology: 2 + Source: Join-left-repartition-source (topics: [Join-left-repartition]) + --> Join-this-windowed + Source: Join-right-repartition-source (topics: [Join-right-repartition]) + --> Join-other-windowed + Processor: Join-other-windowed (stores: [KSTREAM-OUTEROTHER-0000000017-store]) + --> Join-outer-other-join + <-- Join-right-repartition-source + Processor: Join-this-windowed (stores: [KSTREAM-JOINTHIS-0000000016-store]) + --> Join-this-join + <-- Join-left-repartition-source + Processor: Join-outer-other-join (stores: [KSTREAM-JOINTHIS-0000000016-store]) + --> Join-merge + <-- Join-other-windowed + Processor: Join-this-join (stores: [KSTREAM-OUTEROTHER-0000000017-store]) + --> Join-merge + <-- Join-this-windowed + Processor: Join-merge (stores: []) + --> Project + <-- Join-this-join, Join-outer-other-join + Processor: Project (stores: []) + --> KSTREAM-SINK-0000000020 + <-- Join-merge + Sink: KSTREAM-SINK-0000000020 (topic: OUTPUT) + <-- Project + diff --git a/ksqldb-functional-tests/src/test/resources/historical_plans/joins_-_stream_stream_left_join_with_key_in_projection_-_rekey_-_PROTOBUF_NOSR/7.3.0_1651071281568/plan.json b/ksqldb-functional-tests/src/test/resources/historical_plans/joins_-_stream_stream_left_join_with_key_in_projection_-_rekey_-_PROTOBUF_NOSR/7.3.0_1651071281568/plan.json new file mode 100644 index 000000000000..3b98a429bbb2 --- /dev/null +++ b/ksqldb-functional-tests/src/test/resources/historical_plans/joins_-_stream_stream_left_join_with_key_in_projection_-_rekey_-_PROTOBUF_NOSR/7.3.0_1651071281568/plan.json @@ -0,0 +1,310 @@ +{ + "plan" : [ { + "@type" : "ksqlPlanV1", + "statementText" : "CREATE STREAM TEST (K STRING KEY, ID BIGINT, NAME STRING, VALUE BIGINT) WITH (KAFKA_TOPIC='left_topic', KEY_FORMAT='KAFKA', VALUE_FORMAT='PROTOBUF_NOSR');", + "ddlCommand" : { + "@type" : "createStreamV1", + "sourceName" : "TEST", + "schema" : "`K` STRING KEY, `ID` BIGINT, `NAME` STRING, `VALUE` BIGINT", + "timestampColumn" : null, + "topicName" : "left_topic", + "formats" : { + "keyFormat" : { + "format" : "KAFKA", + "properties" : { } + }, + "valueFormat" : { + "format" : "PROTOBUF_NOSR", + "properties" : { } + } + }, + "windowInfo" : null, + "orReplace" : false, + "isSource" : false + }, + "queryPlan" : null + }, { + "@type" : "ksqlPlanV1", + "statementText" : "CREATE STREAM TEST_STREAM (K STRING KEY, ID BIGINT, F1 STRING, F2 BIGINT) WITH (KAFKA_TOPIC='right_topic', KEY_FORMAT='KAFKA', VALUE_FORMAT='PROTOBUF_NOSR');", + "ddlCommand" : { + "@type" : "createStreamV1", + "sourceName" : "TEST_STREAM", + "schema" : "`K` STRING KEY, `ID` BIGINT, `F1` STRING, `F2` BIGINT", + "timestampColumn" : null, + "topicName" : "right_topic", + "formats" : { + "keyFormat" : { + "format" : "KAFKA", + "properties" : { } + }, + "valueFormat" : { + "format" : "PROTOBUF_NOSR", + "properties" : { } + } + }, + "windowInfo" : null, + "orReplace" : false, + "isSource" : false + }, + "queryPlan" : null + }, { + "@type" : "ksqlPlanV1", + "statementText" : "CREATE STREAM OUTPUT AS SELECT\n T.ID T_ID,\n T.K T_K,\n T.NAME NAME,\n T.VALUE VALUE,\n TT.F1 F1,\n TT.F2 F2\nFROM TEST T\nLEFT OUTER JOIN TEST_STREAM TT WITHIN 11 SECONDS ON ((T.ID = TT.ID))\nEMIT CHANGES", + "ddlCommand" : { + "@type" : "createStreamV1", + "sourceName" : "OUTPUT", + "schema" : "`T_ID` BIGINT KEY, `T_K` STRING, `NAME` STRING, `VALUE` BIGINT, `F1` STRING, `F2` BIGINT", + "timestampColumn" : null, + "topicName" : "OUTPUT", + "formats" : { + "keyFormat" : { + "format" : "KAFKA", + "properties" : { } + }, + "valueFormat" : { + "format" : "PROTOBUF_NOSR", + "properties" : { } + } + }, + "windowInfo" : null, + "orReplace" : false, + "isSource" : false + }, + "queryPlan" : { + "sources" : [ "TEST", "TEST_STREAM" ], + "sink" : "OUTPUT", + "physicalPlan" : { + "@type" : "streamSinkV1", + "properties" : { + "queryContext" : "OUTPUT" + }, + "source" : { + "@type" : "streamSelectV1", + "properties" : { + "queryContext" : "Project" + }, + "source" : { + "@type" : "streamStreamJoinV1", + "properties" : { + "queryContext" : "Join" + }, + "joinType" : "LEFT", + "leftInternalFormats" : { + "keyFormat" : { + "format" : "KAFKA", + "properties" : { } + }, + "valueFormat" : { + "format" : "PROTOBUF_NOSR", + "properties" : { } + } + }, + "rightInternalFormats" : { + "keyFormat" : { + "format" : "KAFKA", + "properties" : { } + }, + "valueFormat" : { + "format" : "PROTOBUF_NOSR", + "properties" : { } + } + }, + "leftSource" : { + "@type" : "streamSelectV1", + "properties" : { + "queryContext" : "PrependAliasLeft" + }, + "source" : { + "@type" : "streamSelectKeyV2", + "properties" : { + "queryContext" : "LeftSourceKeyed" + }, + "source" : { + "@type" : "streamSourceV1", + "properties" : { + "queryContext" : "KafkaTopic_Left/Source" + }, + "topicName" : "left_topic", + "formats" : { + "keyFormat" : { + "format" : "KAFKA", + "properties" : { } + }, + "valueFormat" : { + "format" : "PROTOBUF_NOSR", + "properties" : { } + } + }, + "timestampColumn" : null, + "sourceSchema" : "`K` STRING KEY, `ID` BIGINT, `NAME` STRING, `VALUE` BIGINT", + "pseudoColumnVersion" : 1 + }, + "keyExpression" : [ "ID" ] + }, + "keyColumnNames" : [ "T_ID" ], + "selectedKeys" : null, + "selectExpressions" : [ "ID AS T_ID", "NAME AS T_NAME", "VALUE AS T_VALUE", "ROWTIME AS T_ROWTIME", "ROWPARTITION AS T_ROWPARTITION", "ROWOFFSET AS T_ROWOFFSET", "K AS T_K" ] + }, + "rightSource" : { + "@type" : "streamSelectV1", + "properties" : { + "queryContext" : "PrependAliasRight" + }, + "source" : { + "@type" : "streamSelectKeyV2", + "properties" : { + "queryContext" : "RightSourceKeyed" + }, + "source" : { + "@type" : "streamSourceV1", + "properties" : { + "queryContext" : "KafkaTopic_Right/Source" + }, + "topicName" : "right_topic", + "formats" : { + "keyFormat" : { + "format" : "KAFKA", + "properties" : { } + }, + "valueFormat" : { + "format" : "PROTOBUF_NOSR", + "properties" : { } + } + }, + "timestampColumn" : null, + "sourceSchema" : "`K` STRING KEY, `ID` BIGINT, `F1` STRING, `F2` BIGINT", + "pseudoColumnVersion" : 1 + }, + "keyExpression" : [ "ID" ] + }, + "keyColumnNames" : [ "TT_ID" ], + "selectedKeys" : null, + "selectExpressions" : [ "ID AS TT_ID", "F1 AS TT_F1", "F2 AS TT_F2", "ROWTIME AS TT_ROWTIME", "ROWPARTITION AS TT_ROWPARTITION", "ROWOFFSET AS TT_ROWOFFSET", "K AS TT_K" ] + }, + "beforeMillis" : 11.000000000, + "afterMillis" : 11.000000000, + "graceMillis" : null, + "keyColName" : "T_ID" + }, + "keyColumnNames" : [ "T_ID" ], + "selectedKeys" : null, + "selectExpressions" : [ "T_K AS T_K", "T_NAME AS NAME", "T_VALUE AS VALUE", "TT_F1 AS F1", "TT_F2 AS F2" ] + }, + "formats" : { + "keyFormat" : { + "format" : "KAFKA", + "properties" : { } + }, + "valueFormat" : { + "format" : "PROTOBUF_NOSR", + "properties" : { } + } + }, + "topicName" : "OUTPUT", + "timestampColumn" : null + }, + "queryId" : "CSAS_OUTPUT_0", + "runtimeId" : null + } + } ], + "configs" : { + "ksql.extension.dir" : "ext", + "ksql.streams.cache.max.bytes.buffering" : "0", + "metric.reporters" : "", + "ksql.query.status.running.threshold.seconds" : "300", + "ksql.connect.basic.auth.credentials.reload" : "false", + "ksql.output.topic.name.prefix" : "", + "ksql.query.pull.stream.enabled" : "true", + "ksql.query.push.v2.interpreter.enabled" : "true", + "ksql.queryanonymizer.logs_enabled" : "true", + "ksql.variable.substitution.enable" : "true", + "ksql.streams.shutdown.timeout.ms" : "300000", + "ksql.query.pull.max.allowed.offset.lag" : "9223372036854775807", + "ksql.query.pull.max.qps" : "2147483647", + "ksql.access.validator.enable" : "auto", + "ksql.streams.bootstrap.servers" : "localhost:0", + "ksql.query.pull.metrics.enabled" : "true", + "ksql.metrics.extension" : null, + "ksql.query.push.v2.alos.enabled" : "true", + "ksql.query.push.v2.max.hourly.bandwidth.megabytes" : "2147483647", + "ksql.query.pull.range.scan.enabled" : "true", + "ksql.transient.query.cleanup.service.initial.delay.seconds" : "600", + "ksql.hidden.topics" : "_confluent.*,__confluent.*,_schemas,__consumer_offsets,__transaction_state,connect-configs,connect-offsets,connect-status,connect-statuses", + "ksql.lambdas.enabled" : "true", + "ksql.source.table.materialization.enabled" : "true", + "ksql.query.pull.max.hourly.bandwidth.megabytes" : "2147483647", + "ksql.sink.window.change.log.additional.retention" : "1000000", + "ksql.query.persistent.active.limit" : "2147483647", + "ksql.persistence.wrap.single.values" : null, + "ksql.query.transient.max.bytes.buffering.total" : "-1", + "ksql.connect.basic.auth.credentials.source" : "NONE", + "ksql.schema.registry.url" : "schema_registry.url:0", + "ksql.properties.overrides.denylist" : "", + "ksql.service.id" : "some.ksql.service.id", + "ksql.query.push.v2.max.catchup.consumers" : "5", + "ksql.query.push.v2.enabled" : "false", + "ksql.transient.query.cleanup.service.enable" : "true", + "ksql.query.push.v2.metrics.enabled" : "true", + "ksql.rowpartition.rowoffset.enabled" : "true", + "ksql.streams.commit.interval.ms" : "2000", + "ksql.query.pull.table.scan.enabled" : "true", + "ksql.streams.auto.commit.interval.ms" : "0", + "ksql.streams.topology.optimization" : "all", + "ksql.endpoint.migrate.query" : "true", + "ksql.query.push.v2.registry.installed" : "false", + "ksql.streams.num.stream.threads" : "4", + "ksql.metrics.tags.custom" : "", + "ksql.query.push.v2.catchup.consumer.msg.window" : "50", + "ksql.runtime.feature.shared.enabled" : "false", + "ksql.udf.collect.metrics" : "false", + "ksql.new.query.planner.enabled" : "false", + "ksql.connect.request.headers.plugin" : null, + "ksql.security.extension.class" : null, + "ksql.transient.prefix" : "transient_", + "ksql.headers.columns.enabled" : "true", + "ksql.streams.default.deserialization.exception.handler" : "io.confluent.ksql.errors.LogMetricAndContinueExceptionHandler", + "ksql.connect.request.timeout.ms" : "5000", + "ksql.query.pull.enable.standby.reads" : "false", + "ksql.persistence.default.format.key" : "KAFKA", + "ksql.query.persistent.max.bytes.buffering.total" : "-1", + "ksql.query.error.max.queue.size" : "10", + "ksql.query.cleanup.shutdown.timeout.ms" : "30000", + "ksql.internal.topic.min.insync.replicas" : "1", + "ksql.internal.topic.replicas" : "1", + "ksql.insert.into.values.enabled" : "true", + "ksql.queryanonymizer.cluster_namespace" : null, + "ksql.create.or.replace.enabled" : "true", + "ksql.shared.runtimes.count" : "2", + "ksql.cast.strings.preserve.nulls" : "true", + "ksql.authorization.cache.max.entries" : "10000", + "ksql.pull.queries.enable" : "true", + "ksql.transient.query.cleanup.service.period.seconds" : "600", + "ksql.suppress.enabled" : "false", + "ksql.readonly.topics" : "_confluent.*,__confluent.*,_schemas,__consumer_offsets,__transaction_state,connect-configs,connect-offsets,connect-status,connect-statuses", + "ksql.connect.basic.auth.credentials.file" : "", + "ksql.authorization.cache.expiry.time.secs" : "30", + "ksql.query.retry.backoff.initial.ms" : "15000", + "ksql.query.pull.max.concurrent.requests" : "2147483647", + "ksql.streams.auto.offset.reset" : "earliest", + "ksql.connect.url" : "http://localhost:8083", + "ksql.query.push.v2.new.latest.delay.ms" : "5000", + "ksql.query.push.v2.latest.reset.age.ms" : "30000", + "ksql.streams.default.production.exception.handler" : "io.confluent.ksql.errors.ProductionExceptionHandlerUtil$LogAndFailProductionExceptionHandler", + "ksql.query.pull.interpreter.enabled" : "true", + "ksql.query.pull.limit.clause.enabled" : "true", + "ksql.query.pull.router.thread.pool.size" : "50", + "ksql.query.push.v2.continuation.tokens.enabled" : "false", + "ksql.query.retry.backoff.max.ms" : "900000", + "ksql.timestamp.throw.on.invalid" : "false", + "ksql.persistence.default.format.value" : null, + "ksql.udfs.enabled" : "true", + "ksql.udf.enable.security.manager" : "true", + "ksql.connect.worker.config" : "", + "ksql.nested.error.set.null" : "true", + "ksql.query.pull.thread.pool.size" : "50", + "ksql.persistent.prefix" : "query_", + "ksql.metastore.backup.location" : "", + "ksql.error.classifier.regex" : "", + "ksql.suppress.buffer.size.bytes" : "-1" + } +} \ No newline at end of file diff --git a/ksqldb-functional-tests/src/test/resources/historical_plans/joins_-_stream_stream_left_join_with_key_in_projection_-_rekey_-_PROTOBUF_NOSR/7.3.0_1651071281568/spec.json b/ksqldb-functional-tests/src/test/resources/historical_plans/joins_-_stream_stream_left_join_with_key_in_projection_-_rekey_-_PROTOBUF_NOSR/7.3.0_1651071281568/spec.json new file mode 100644 index 000000000000..7a92f6e5f556 --- /dev/null +++ b/ksqldb-functional-tests/src/test/resources/historical_plans/joins_-_stream_stream_left_join_with_key_in_projection_-_rekey_-_PROTOBUF_NOSR/7.3.0_1651071281568/spec.json @@ -0,0 +1,318 @@ +{ + "version" : "7.3.0", + "timestamp" : 1651071281568, + "path" : "query-validation-tests/joins.json", + "schemas" : { + "CSAS_OUTPUT_0.KafkaTopic_Right.Source" : { + "schema" : "`K` STRING KEY, `ID` BIGINT, `F1` STRING, `F2` BIGINT", + "keyFormat" : { + "format" : "KAFKA" + }, + "valueFormat" : { + "format" : "PROTOBUF_NOSR" + } + }, + "CSAS_OUTPUT_0.Join.Left" : { + "schema" : "`T_ID` BIGINT KEY, `T_ID` BIGINT, `T_NAME` STRING, `T_VALUE` BIGINT, `T_ROWTIME` BIGINT, `T_ROWPARTITION` INTEGER, `T_ROWOFFSET` BIGINT, `T_K` STRING", + "valueFormat" : { + "format" : "PROTOBUF_NOSR" + } + }, + "CSAS_OUTPUT_0.OUTPUT" : { + "schema" : "`T_ID` BIGINT KEY, `T_K` STRING, `NAME` STRING, `VALUE` BIGINT, `F1` STRING, `F2` BIGINT", + "keyFormat" : { + "format" : "KAFKA" + }, + "valueFormat" : { + "format" : "PROTOBUF_NOSR" + } + }, + "CSAS_OUTPUT_0.KafkaTopic_Left.Source" : { + "schema" : "`K` STRING KEY, `ID` BIGINT, `NAME` STRING, `VALUE` BIGINT", + "keyFormat" : { + "format" : "KAFKA" + }, + "valueFormat" : { + "format" : "PROTOBUF_NOSR" + } + }, + "CSAS_OUTPUT_0.Join" : { + "schema" : "`T_ID` BIGINT KEY, `T_ID` BIGINT, `T_NAME` STRING, `T_VALUE` BIGINT, `T_ROWTIME` BIGINT, `T_ROWPARTITION` INTEGER, `T_ROWOFFSET` BIGINT, `T_K` STRING", + "keyFormat" : { + "format" : "KAFKA" + } + }, + "CSAS_OUTPUT_0.Join.Right" : { + "schema" : "`TT_ID` BIGINT KEY, `TT_ID` BIGINT, `TT_F1` STRING, `TT_F2` BIGINT, `TT_ROWTIME` BIGINT, `TT_ROWPARTITION` INTEGER, `TT_ROWOFFSET` BIGINT, `TT_K` STRING", + "valueFormat" : { + "format" : "PROTOBUF_NOSR" + } + } + }, + "testCase" : { + "name" : "stream stream left join with key in projection - rekey - PROTOBUF_NOSR", + "inputs" : [ { + "topic" : "left_topic", + "key" : "foo", + "value" : { + "ID" : 0, + "NAME" : "zero", + "VALUE" : 0 + }, + "timestamp" : 0 + }, { + "topic" : "right_topic", + "key" : "foo", + "value" : { + "ID" : 0, + "F1" : "blah", + "F2" : 50 + }, + "timestamp" : 10000 + }, { + "topic" : "left_topic", + "key" : "foo", + "value" : { + "ID" : 10, + "NAME" : "100", + "VALUE" : 5 + }, + "timestamp" : 11000 + }, { + "topic" : "left_topic", + "key" : "foo", + "value" : { + "ID" : 0, + "NAME" : "foo", + "VALUE" : 100 + }, + "timestamp" : 13000 + }, { + "topic" : "right_topic", + "key" : "foo", + "value" : { + "ID" : 0, + "F1" : "a", + "F2" : 10 + }, + "timestamp" : 15000 + }, { + "topic" : "right_topic", + "key" : "foo", + "value" : { + "ID" : 100, + "F1" : "newblah", + "F2" : 150 + }, + "timestamp" : 16000 + }, { + "topic" : "left_topic", + "key" : "foo", + "value" : { + "ID" : 90, + "NAME" : "ninety", + "VALUE" : 90 + }, + "timestamp" : 17000 + }, { + "topic" : "left_topic", + "key" : "foo", + "value" : { + "ID" : 0, + "NAME" : "bar", + "VALUE" : 99 + }, + "timestamp" : 30000 + } ], + "outputs" : [ { + "topic" : "OUTPUT", + "key" : 0, + "value" : { + "T_K" : "foo", + "NAME" : "zero", + "VALUE" : 0, + "F1" : "", + "F2" : 0 + }, + "timestamp" : 0 + }, { + "topic" : "OUTPUT", + "key" : 0, + "value" : { + "T_K" : "foo", + "NAME" : "zero", + "VALUE" : 0, + "F1" : "blah", + "F2" : 50 + }, + "timestamp" : 10000 + }, { + "topic" : "OUTPUT", + "key" : 10, + "value" : { + "T_K" : "foo", + "NAME" : "100", + "VALUE" : 5, + "F1" : "", + "F2" : 0 + }, + "timestamp" : 11000 + }, { + "topic" : "OUTPUT", + "key" : 0, + "value" : { + "T_K" : "foo", + "NAME" : "foo", + "VALUE" : 100, + "F1" : "blah", + "F2" : 50 + }, + "timestamp" : 13000 + }, { + "topic" : "OUTPUT", + "key" : 0, + "value" : { + "T_K" : "foo", + "NAME" : "foo", + "VALUE" : 100, + "F1" : "a", + "F2" : 10 + }, + "timestamp" : 15000 + }, { + "topic" : "OUTPUT", + "key" : 90, + "value" : { + "T_K" : "foo", + "NAME" : "ninety", + "VALUE" : 90, + "F1" : "", + "F2" : 0 + }, + "timestamp" : 17000 + }, { + "topic" : "OUTPUT", + "key" : 0, + "value" : { + "T_K" : "foo", + "NAME" : "bar", + "VALUE" : 99, + "F1" : "", + "F2" : 0 + }, + "timestamp" : 30000 + } ], + "topics" : [ { + "name" : "right_topic", + "replicas" : 1, + "numPartitions" : 4 + }, { + "name" : "OUTPUT", + "replicas" : 1, + "numPartitions" : 4 + }, { + "name" : "left_topic", + "replicas" : 1, + "numPartitions" : 4 + } ], + "statements" : [ "CREATE STREAM TEST (K STRING KEY, ID bigint, NAME varchar, VALUE bigint) WITH (kafka_topic='left_topic', value_format='PROTOBUF_NOSR');", "CREATE STREAM TEST_STREAM (K STRING KEY, ID bigint, F1 varchar, F2 bigint) WITH (kafka_topic='right_topic', value_format='PROTOBUF_NOSR');", "CREATE STREAM OUTPUT as SELECT t.id, t.k, name, value, f1, f2 FROM test t left join TEST_STREAM tt WITHIN 11 seconds ON t.id = tt.id;" ], + "post" : { + "sources" : [ { + "name" : "OUTPUT", + "type" : "STREAM", + "schema" : "`T_ID` BIGINT KEY, `T_K` STRING, `NAME` STRING, `VALUE` BIGINT, `F1` STRING, `F2` BIGINT", + "keyFormat" : { + "format" : "KAFKA" + }, + "valueFormat" : "PROTOBUF_NOSR", + "keyFeatures" : [ ], + "valueFeatures" : [ ], + "isSource" : false + }, { + "name" : "TEST", + "type" : "STREAM", + "schema" : "`K` STRING KEY, `ID` BIGINT, `NAME` STRING, `VALUE` BIGINT", + "keyFormat" : { + "format" : "KAFKA" + }, + "valueFormat" : "PROTOBUF_NOSR", + "keyFeatures" : [ ], + "valueFeatures" : [ ], + "isSource" : false + }, { + "name" : "TEST_STREAM", + "type" : "STREAM", + "schema" : "`K` STRING KEY, `ID` BIGINT, `F1` STRING, `F2` BIGINT", + "keyFormat" : { + "format" : "KAFKA" + }, + "valueFormat" : "PROTOBUF_NOSR", + "keyFeatures" : [ ], + "valueFeatures" : [ ], + "isSource" : false + } ], + "topics" : { + "topics" : [ { + "name" : "_confluent-ksql-some.ksql.service.idquery_CSAS_OUTPUT_0-Join-left-repartition", + "keyFormat" : { + "format" : "KAFKA" + }, + "valueFormat" : { + "format" : "PROTOBUF_NOSR" + } + }, { + "name" : "_confluent-ksql-some.ksql.service.idquery_CSAS_OUTPUT_0-Join-right-repartition", + "keyFormat" : { + "format" : "KAFKA" + }, + "valueFormat" : { + "format" : "PROTOBUF_NOSR" + } + }, { + "name" : "_confluent-ksql-some.ksql.service.idquery_CSAS_OUTPUT_0-KSTREAM-JOINTHIS-0000000016-store-changelog", + "keyFormat" : { + "format" : "KAFKA" + }, + "valueFormat" : { + "format" : "PROTOBUF_NOSR" + } + }, { + "name" : "left_topic", + "keyFormat" : { + "format" : "KAFKA" + }, + "valueFormat" : { + "format" : "PROTOBUF_NOSR" + }, + "partitions" : 4 + }, { + "name" : "_confluent-ksql-some.ksql.service.idquery_CSAS_OUTPUT_0-KSTREAM-OUTEROTHER-0000000017-store-changelog", + "keyFormat" : { + "format" : "KAFKA" + }, + "valueFormat" : { + "format" : "PROTOBUF_NOSR" + } + }, { + "name" : "OUTPUT", + "keyFormat" : { + "format" : "KAFKA" + }, + "valueFormat" : { + "format" : "PROTOBUF_NOSR" + }, + "partitions" : 4 + }, { + "name" : "right_topic", + "keyFormat" : { + "format" : "KAFKA" + }, + "valueFormat" : { + "format" : "PROTOBUF_NOSR" + }, + "partitions" : 4 + } ] + } + } + } +} \ No newline at end of file diff --git a/ksqldb-functional-tests/src/test/resources/historical_plans/joins_-_stream_stream_left_join_with_key_in_projection_-_rekey_-_PROTOBUF_NOSR/7.3.0_1651071281568/topology b/ksqldb-functional-tests/src/test/resources/historical_plans/joins_-_stream_stream_left_join_with_key_in_projection_-_rekey_-_PROTOBUF_NOSR/7.3.0_1651071281568/topology new file mode 100644 index 000000000000..4c5f7fa6ca95 --- /dev/null +++ b/ksqldb-functional-tests/src/test/resources/historical_plans/joins_-_stream_stream_left_join_with_key_in_projection_-_rekey_-_PROTOBUF_NOSR/7.3.0_1651071281568/topology @@ -0,0 +1,63 @@ +Topologies: + Sub-topology: 0 + Source: KSTREAM-SOURCE-0000000000 (topics: [left_topic]) + --> KSTREAM-TRANSFORMVALUES-0000000001 + Processor: KSTREAM-TRANSFORMVALUES-0000000001 (stores: []) + --> LeftSourceKeyed-SelectKey + <-- KSTREAM-SOURCE-0000000000 + Processor: LeftSourceKeyed-SelectKey (stores: []) + --> PrependAliasLeft + <-- KSTREAM-TRANSFORMVALUES-0000000001 + Processor: PrependAliasLeft (stores: []) + --> Join-left-repartition-filter + <-- LeftSourceKeyed-SelectKey + Processor: Join-left-repartition-filter (stores: []) + --> Join-left-repartition-sink + <-- PrependAliasLeft + Sink: Join-left-repartition-sink (topic: Join-left-repartition) + <-- Join-left-repartition-filter + + Sub-topology: 1 + Source: KSTREAM-SOURCE-0000000004 (topics: [right_topic]) + --> KSTREAM-TRANSFORMVALUES-0000000005 + Processor: KSTREAM-TRANSFORMVALUES-0000000005 (stores: []) + --> RightSourceKeyed-SelectKey + <-- KSTREAM-SOURCE-0000000004 + Processor: RightSourceKeyed-SelectKey (stores: []) + --> PrependAliasRight + <-- KSTREAM-TRANSFORMVALUES-0000000005 + Processor: PrependAliasRight (stores: []) + --> Join-right-repartition-filter + <-- RightSourceKeyed-SelectKey + Processor: Join-right-repartition-filter (stores: []) + --> Join-right-repartition-sink + <-- PrependAliasRight + Sink: Join-right-repartition-sink (topic: Join-right-repartition) + <-- Join-right-repartition-filter + + Sub-topology: 2 + Source: Join-left-repartition-source (topics: [Join-left-repartition]) + --> Join-this-windowed + Source: Join-right-repartition-source (topics: [Join-right-repartition]) + --> Join-other-windowed + Processor: Join-other-windowed (stores: [KSTREAM-OUTEROTHER-0000000017-store]) + --> Join-outer-other-join + <-- Join-right-repartition-source + Processor: Join-this-windowed (stores: [KSTREAM-JOINTHIS-0000000016-store]) + --> Join-this-join + <-- Join-left-repartition-source + Processor: Join-outer-other-join (stores: [KSTREAM-JOINTHIS-0000000016-store]) + --> Join-merge + <-- Join-other-windowed + Processor: Join-this-join (stores: [KSTREAM-OUTEROTHER-0000000017-store]) + --> Join-merge + <-- Join-this-windowed + Processor: Join-merge (stores: []) + --> Project + <-- Join-this-join, Join-outer-other-join + Processor: Project (stores: []) + --> KSTREAM-SINK-0000000020 + <-- Join-merge + Sink: KSTREAM-SINK-0000000020 (topic: OUTPUT) + <-- Project + diff --git a/ksqldb-functional-tests/src/test/resources/historical_plans/joins_-_stream_stream_outer_join_-_PROTOBUF_-_PROTOBUF/7.3.0_1651071348899/plan.json b/ksqldb-functional-tests/src/test/resources/historical_plans/joins_-_stream_stream_outer_join_-_PROTOBUF_-_PROTOBUF/7.3.0_1651071348899/plan.json new file mode 100644 index 000000000000..98611330aa79 --- /dev/null +++ b/ksqldb-functional-tests/src/test/resources/historical_plans/joins_-_stream_stream_outer_join_-_PROTOBUF_-_PROTOBUF/7.3.0_1651071348899/plan.json @@ -0,0 +1,312 @@ +{ + "plan" : [ { + "@type" : "ksqlPlanV1", + "statementText" : "CREATE STREAM TEST (ID BIGINT KEY, NAME STRING, VALUE BIGINT) WITH (KAFKA_TOPIC='left_topic', KEY_FORMAT='KAFKA', VALUE_FORMAT='PROTOBUF');", + "ddlCommand" : { + "@type" : "createStreamV1", + "sourceName" : "TEST", + "schema" : "`ID` BIGINT KEY, `NAME` STRING, `VALUE` BIGINT", + "timestampColumn" : null, + "topicName" : "left_topic", + "formats" : { + "keyFormat" : { + "format" : "KAFKA", + "properties" : { } + }, + "valueFormat" : { + "format" : "PROTOBUF", + "properties" : { + "unwrapPrimitives" : "true" + } + } + }, + "windowInfo" : null, + "orReplace" : false, + "isSource" : false + }, + "queryPlan" : null + }, { + "@type" : "ksqlPlanV1", + "statementText" : "CREATE STREAM TEST_STREAM (ID BIGINT KEY, F1 STRING, F2 BIGINT) WITH (KAFKA_TOPIC='right_topic', KEY_FORMAT='KAFKA', VALUE_FORMAT='PROTOBUF');", + "ddlCommand" : { + "@type" : "createStreamV1", + "sourceName" : "TEST_STREAM", + "schema" : "`ID` BIGINT KEY, `F1` STRING, `F2` BIGINT", + "timestampColumn" : null, + "topicName" : "right_topic", + "formats" : { + "keyFormat" : { + "format" : "KAFKA", + "properties" : { } + }, + "valueFormat" : { + "format" : "PROTOBUF", + "properties" : { + "unwrapPrimitives" : "true" + } + } + }, + "windowInfo" : null, + "orReplace" : false, + "isSource" : false + }, + "queryPlan" : null + }, { + "@type" : "ksqlPlanV1", + "statementText" : "CREATE STREAM OUTPUT AS SELECT\n ROWKEY ID,\n T.NAME NAME,\n T.VALUE VALUE,\n TT.F1 F1,\n TT.F2 F2\nFROM TEST T\nFULL OUTER JOIN TEST_STREAM TT WITHIN 11 SECONDS ON ((T.ID = TT.ID))\nEMIT CHANGES", + "ddlCommand" : { + "@type" : "createStreamV1", + "sourceName" : "OUTPUT", + "schema" : "`ID` BIGINT KEY, `NAME` STRING, `VALUE` BIGINT, `F1` STRING, `F2` BIGINT", + "timestampColumn" : null, + "topicName" : "OUTPUT", + "formats" : { + "keyFormat" : { + "format" : "KAFKA", + "properties" : { } + }, + "valueFormat" : { + "format" : "PROTOBUF", + "properties" : { + "unwrapPrimitives" : "true" + } + } + }, + "windowInfo" : null, + "orReplace" : false, + "isSource" : false + }, + "queryPlan" : { + "sources" : [ "TEST", "TEST_STREAM" ], + "sink" : "OUTPUT", + "physicalPlan" : { + "@type" : "streamSinkV1", + "properties" : { + "queryContext" : "OUTPUT" + }, + "source" : { + "@type" : "streamSelectV1", + "properties" : { + "queryContext" : "Project" + }, + "source" : { + "@type" : "streamStreamJoinV1", + "properties" : { + "queryContext" : "Join" + }, + "joinType" : "OUTER", + "leftInternalFormats" : { + "keyFormat" : { + "format" : "KAFKA", + "properties" : { } + }, + "valueFormat" : { + "format" : "PROTOBUF", + "properties" : { + "unwrapPrimitives" : "true" + } + } + }, + "rightInternalFormats" : { + "keyFormat" : { + "format" : "KAFKA", + "properties" : { } + }, + "valueFormat" : { + "format" : "PROTOBUF", + "properties" : { + "unwrapPrimitives" : "true" + } + } + }, + "leftSource" : { + "@type" : "streamSelectV1", + "properties" : { + "queryContext" : "PrependAliasLeft" + }, + "source" : { + "@type" : "streamSourceV1", + "properties" : { + "queryContext" : "KafkaTopic_Left/Source" + }, + "topicName" : "left_topic", + "formats" : { + "keyFormat" : { + "format" : "KAFKA", + "properties" : { } + }, + "valueFormat" : { + "format" : "PROTOBUF", + "properties" : { + "unwrapPrimitives" : "true" + } + } + }, + "timestampColumn" : null, + "sourceSchema" : "`ID` BIGINT KEY, `NAME` STRING, `VALUE` BIGINT", + "pseudoColumnVersion" : 1 + }, + "keyColumnNames" : [ "T_ID" ], + "selectedKeys" : null, + "selectExpressions" : [ "NAME AS T_NAME", "VALUE AS T_VALUE", "ROWTIME AS T_ROWTIME", "ROWPARTITION AS T_ROWPARTITION", "ROWOFFSET AS T_ROWOFFSET", "ID AS T_ID" ] + }, + "rightSource" : { + "@type" : "streamSelectV1", + "properties" : { + "queryContext" : "PrependAliasRight" + }, + "source" : { + "@type" : "streamSourceV1", + "properties" : { + "queryContext" : "KafkaTopic_Right/Source" + }, + "topicName" : "right_topic", + "formats" : { + "keyFormat" : { + "format" : "KAFKA", + "properties" : { } + }, + "valueFormat" : { + "format" : "PROTOBUF", + "properties" : { + "unwrapPrimitives" : "true" + } + } + }, + "timestampColumn" : null, + "sourceSchema" : "`ID` BIGINT KEY, `F1` STRING, `F2` BIGINT", + "pseudoColumnVersion" : 1 + }, + "keyColumnNames" : [ "TT_ID" ], + "selectedKeys" : null, + "selectExpressions" : [ "F1 AS TT_F1", "F2 AS TT_F2", "ROWTIME AS TT_ROWTIME", "ROWPARTITION AS TT_ROWPARTITION", "ROWOFFSET AS TT_ROWOFFSET", "ID AS TT_ID" ] + }, + "beforeMillis" : 11.000000000, + "afterMillis" : 11.000000000, + "graceMillis" : null, + "keyColName" : "ROWKEY" + }, + "keyColumnNames" : [ "ID" ], + "selectedKeys" : null, + "selectExpressions" : [ "T_NAME AS NAME", "T_VALUE AS VALUE", "TT_F1 AS F1", "TT_F2 AS F2" ] + }, + "formats" : { + "keyFormat" : { + "format" : "KAFKA", + "properties" : { } + }, + "valueFormat" : { + "format" : "PROTOBUF", + "properties" : { + "unwrapPrimitives" : "true" + } + } + }, + "topicName" : "OUTPUT", + "timestampColumn" : null + }, + "queryId" : "CSAS_OUTPUT_0", + "runtimeId" : null + } + } ], + "configs" : { + "ksql.extension.dir" : "ext", + "ksql.streams.cache.max.bytes.buffering" : "0", + "metric.reporters" : "", + "ksql.query.status.running.threshold.seconds" : "300", + "ksql.connect.basic.auth.credentials.reload" : "false", + "ksql.output.topic.name.prefix" : "", + "ksql.query.pull.stream.enabled" : "true", + "ksql.query.push.v2.interpreter.enabled" : "true", + "ksql.queryanonymizer.logs_enabled" : "true", + "ksql.variable.substitution.enable" : "true", + "ksql.streams.shutdown.timeout.ms" : "300000", + "ksql.query.pull.max.allowed.offset.lag" : "9223372036854775807", + "ksql.query.pull.max.qps" : "2147483647", + "ksql.access.validator.enable" : "auto", + "ksql.streams.bootstrap.servers" : "localhost:0", + "ksql.query.pull.metrics.enabled" : "true", + "ksql.metrics.extension" : null, + "ksql.query.push.v2.alos.enabled" : "true", + "ksql.query.push.v2.max.hourly.bandwidth.megabytes" : "2147483647", + "ksql.query.pull.range.scan.enabled" : "true", + "ksql.transient.query.cleanup.service.initial.delay.seconds" : "600", + "ksql.hidden.topics" : "_confluent.*,__confluent.*,_schemas,__consumer_offsets,__transaction_state,connect-configs,connect-offsets,connect-status,connect-statuses", + "ksql.lambdas.enabled" : "true", + "ksql.source.table.materialization.enabled" : "true", + "ksql.query.pull.max.hourly.bandwidth.megabytes" : "2147483647", + "ksql.sink.window.change.log.additional.retention" : "1000000", + "ksql.query.persistent.active.limit" : "2147483647", + "ksql.persistence.wrap.single.values" : null, + "ksql.query.transient.max.bytes.buffering.total" : "-1", + "ksql.connect.basic.auth.credentials.source" : "NONE", + "ksql.schema.registry.url" : "schema_registry.url:0", + "ksql.properties.overrides.denylist" : "", + "ksql.service.id" : "some.ksql.service.id", + "ksql.query.push.v2.max.catchup.consumers" : "5", + "ksql.query.push.v2.enabled" : "false", + "ksql.transient.query.cleanup.service.enable" : "true", + "ksql.query.push.v2.metrics.enabled" : "true", + "ksql.rowpartition.rowoffset.enabled" : "true", + "ksql.streams.commit.interval.ms" : "2000", + "ksql.query.pull.table.scan.enabled" : "true", + "ksql.streams.auto.commit.interval.ms" : "0", + "ksql.streams.topology.optimization" : "all", + "ksql.endpoint.migrate.query" : "true", + "ksql.query.push.v2.registry.installed" : "false", + "ksql.streams.num.stream.threads" : "4", + "ksql.metrics.tags.custom" : "", + "ksql.query.push.v2.catchup.consumer.msg.window" : "50", + "ksql.runtime.feature.shared.enabled" : "false", + "ksql.udf.collect.metrics" : "false", + "ksql.new.query.planner.enabled" : "false", + "ksql.connect.request.headers.plugin" : null, + "ksql.security.extension.class" : null, + "ksql.transient.prefix" : "transient_", + "ksql.headers.columns.enabled" : "true", + "ksql.streams.default.deserialization.exception.handler" : "io.confluent.ksql.errors.LogMetricAndContinueExceptionHandler", + "ksql.connect.request.timeout.ms" : "5000", + "ksql.query.pull.enable.standby.reads" : "false", + "ksql.persistence.default.format.key" : "KAFKA", + "ksql.query.persistent.max.bytes.buffering.total" : "-1", + "ksql.query.error.max.queue.size" : "10", + "ksql.query.cleanup.shutdown.timeout.ms" : "30000", + "ksql.internal.topic.min.insync.replicas" : "1", + "ksql.internal.topic.replicas" : "1", + "ksql.insert.into.values.enabled" : "true", + "ksql.queryanonymizer.cluster_namespace" : null, + "ksql.create.or.replace.enabled" : "true", + "ksql.shared.runtimes.count" : "2", + "ksql.cast.strings.preserve.nulls" : "true", + "ksql.authorization.cache.max.entries" : "10000", + "ksql.pull.queries.enable" : "true", + "ksql.transient.query.cleanup.service.period.seconds" : "600", + "ksql.suppress.enabled" : "false", + "ksql.readonly.topics" : "_confluent.*,__confluent.*,_schemas,__consumer_offsets,__transaction_state,connect-configs,connect-offsets,connect-status,connect-statuses", + "ksql.connect.basic.auth.credentials.file" : "", + "ksql.authorization.cache.expiry.time.secs" : "30", + "ksql.query.retry.backoff.initial.ms" : "15000", + "ksql.query.pull.max.concurrent.requests" : "2147483647", + "ksql.streams.auto.offset.reset" : "earliest", + "ksql.connect.url" : "http://localhost:8083", + "ksql.query.push.v2.new.latest.delay.ms" : "5000", + "ksql.query.push.v2.latest.reset.age.ms" : "30000", + "ksql.streams.default.production.exception.handler" : "io.confluent.ksql.errors.ProductionExceptionHandlerUtil$LogAndFailProductionExceptionHandler", + "ksql.query.pull.interpreter.enabled" : "true", + "ksql.query.pull.limit.clause.enabled" : "true", + "ksql.query.pull.router.thread.pool.size" : "50", + "ksql.query.push.v2.continuation.tokens.enabled" : "false", + "ksql.query.retry.backoff.max.ms" : "900000", + "ksql.timestamp.throw.on.invalid" : "false", + "ksql.persistence.default.format.value" : null, + "ksql.udfs.enabled" : "true", + "ksql.udf.enable.security.manager" : "true", + "ksql.connect.worker.config" : "", + "ksql.nested.error.set.null" : "true", + "ksql.query.pull.thread.pool.size" : "50", + "ksql.persistent.prefix" : "query_", + "ksql.metastore.backup.location" : "", + "ksql.error.classifier.regex" : "", + "ksql.suppress.buffer.size.bytes" : "-1" + } +} \ No newline at end of file diff --git a/ksqldb-functional-tests/src/test/resources/historical_plans/joins_-_stream_stream_outer_join_-_PROTOBUF_-_PROTOBUF/7.3.0_1651071348899/spec.json b/ksqldb-functional-tests/src/test/resources/historical_plans/joins_-_stream_stream_outer_join_-_PROTOBUF_-_PROTOBUF/7.3.0_1651071348899/spec.json new file mode 100644 index 000000000000..0138aade7a9e --- /dev/null +++ b/ksqldb-functional-tests/src/test/resources/historical_plans/joins_-_stream_stream_outer_join_-_PROTOBUF_-_PROTOBUF/7.3.0_1651071348899/spec.json @@ -0,0 +1,337 @@ +{ + "version" : "7.3.0", + "timestamp" : 1651071348899, + "path" : "query-validation-tests/joins.json", + "schemas" : { + "CSAS_OUTPUT_0.KafkaTopic_Right.Source" : { + "schema" : "`ID` BIGINT KEY, `F1` STRING, `F2` BIGINT", + "keyFormat" : { + "format" : "KAFKA" + }, + "valueFormat" : { + "format" : "PROTOBUF", + "properties" : { + "unwrapPrimitives" : "true" + } + } + }, + "CSAS_OUTPUT_0.Join.Left" : { + "schema" : "`T_ID` BIGINT KEY, `T_NAME` STRING, `T_VALUE` BIGINT, `T_ROWTIME` BIGINT, `T_ROWPARTITION` INTEGER, `T_ROWOFFSET` BIGINT, `T_ID` BIGINT", + "valueFormat" : { + "format" : "PROTOBUF", + "properties" : { + "unwrapPrimitives" : "true" + } + } + }, + "CSAS_OUTPUT_0.OUTPUT" : { + "schema" : "`ID` BIGINT KEY, `NAME` STRING, `VALUE` BIGINT, `F1` STRING, `F2` BIGINT", + "keyFormat" : { + "format" : "KAFKA" + }, + "valueFormat" : { + "format" : "PROTOBUF", + "properties" : { + "unwrapPrimitives" : "true" + } + } + }, + "CSAS_OUTPUT_0.KafkaTopic_Left.Source" : { + "schema" : "`ID` BIGINT KEY, `NAME` STRING, `VALUE` BIGINT", + "keyFormat" : { + "format" : "KAFKA" + }, + "valueFormat" : { + "format" : "PROTOBUF", + "properties" : { + "unwrapPrimitives" : "true" + } + } + }, + "CSAS_OUTPUT_0.Join" : { + "schema" : "`T_ID` BIGINT KEY, `T_NAME` STRING, `T_VALUE` BIGINT, `T_ROWTIME` BIGINT, `T_ROWPARTITION` INTEGER, `T_ROWOFFSET` BIGINT, `T_ID` BIGINT", + "keyFormat" : { + "format" : "KAFKA" + } + }, + "CSAS_OUTPUT_0.Join.Right" : { + "schema" : "`TT_ID` BIGINT KEY, `TT_F1` STRING, `TT_F2` BIGINT, `TT_ROWTIME` BIGINT, `TT_ROWPARTITION` INTEGER, `TT_ROWOFFSET` BIGINT, `TT_ID` BIGINT", + "valueFormat" : { + "format" : "PROTOBUF", + "properties" : { + "unwrapPrimitives" : "true" + } + } + } + }, + "testCase" : { + "name" : "stream stream outer join - PROTOBUF - PROTOBUF", + "inputs" : [ { + "topic" : "left_topic", + "key" : 0, + "value" : { + "NAME" : "zero", + "VALUE" : 0 + }, + "timestamp" : 0 + }, { + "topic" : "right_topic", + "key" : 0, + "value" : { + "F1" : "blah", + "F2" : 50 + }, + "timestamp" : 10000 + }, { + "topic" : "left_topic", + "key" : 10, + "value" : { + "NAME" : "100", + "VALUE" : 5 + }, + "timestamp" : 11000 + }, { + "topic" : "left_topic", + "key" : 0, + "value" : { + "NAME" : "foo", + "VALUE" : 100 + }, + "timestamp" : 13000 + }, { + "topic" : "right_topic", + "key" : 0, + "value" : { + "F1" : "a", + "F2" : 10 + }, + "timestamp" : 15000 + }, { + "topic" : "left_topic", + "key" : 0, + "value" : { + "NAME" : "bar", + "VALUE" : 99 + }, + "timestamp" : 30000 + }, { + "topic" : "left_topic", + "key" : 90, + "value" : { + "NAME" : "ninety", + "VALUE" : 90 + }, + "timestamp" : 17000 + }, { + "topic" : "right_topic", + "key" : 100, + "value" : { + "F1" : "newblah", + "F2" : 150 + }, + "timestamp" : 20000 + } ], + "outputs" : [ { + "topic" : "OUTPUT", + "key" : 0, + "value" : { + "NAME" : "zero", + "VALUE" : 0, + "F1" : "", + "F2" : 0 + }, + "timestamp" : 0 + }, { + "topic" : "OUTPUT", + "key" : 0, + "value" : { + "NAME" : "zero", + "VALUE" : 0, + "F1" : "blah", + "F2" : 50 + }, + "timestamp" : 10000 + }, { + "topic" : "OUTPUT", + "key" : 10, + "value" : { + "NAME" : "100", + "VALUE" : 5, + "F1" : "", + "F2" : 0 + }, + "timestamp" : 11000 + }, { + "topic" : "OUTPUT", + "key" : 0, + "value" : { + "NAME" : "foo", + "VALUE" : 100, + "F1" : "blah", + "F2" : 50 + }, + "timestamp" : 13000 + }, { + "topic" : "OUTPUT", + "key" : 0, + "value" : { + "NAME" : "foo", + "VALUE" : 100, + "F1" : "a", + "F2" : 10 + }, + "timestamp" : 15000 + }, { + "topic" : "OUTPUT", + "key" : 0, + "value" : { + "NAME" : "bar", + "VALUE" : 99, + "F1" : "", + "F2" : 0 + }, + "timestamp" : 30000 + }, { + "topic" : "OUTPUT", + "key" : 90, + "value" : { + "NAME" : "ninety", + "VALUE" : 90, + "F1" : "", + "F2" : 0 + }, + "timestamp" : 17000 + }, { + "topic" : "OUTPUT", + "key" : 100, + "value" : { + "NAME" : "", + "VALUE" : 0, + "F1" : "newblah", + "F2" : 150 + }, + "timestamp" : 20000 + } ], + "topics" : [ { + "name" : "right_topic", + "valueSchema" : "syntax = \"proto3\";\n\nmessage ConnectDefault1 {\n string F1 = 1;\n int64 F2 = 2;\n}\n", + "valueFormat" : "PROTOBUF", + "replicas" : 1, + "numPartitions" : 4 + }, { + "name" : "OUTPUT", + "replicas" : 1, + "numPartitions" : 4 + }, { + "name" : "left_topic", + "valueSchema" : "syntax = \"proto3\";\n\nmessage ConnectDefault1 {\n string NAME = 1;\n int64 VALUE = 2;\n}\n", + "valueFormat" : "PROTOBUF", + "replicas" : 1, + "numPartitions" : 4 + } ], + "statements" : [ "CREATE STREAM TEST (ID BIGINT KEY, NAME varchar, VALUE bigint) WITH (kafka_topic='left_topic', value_format='PROTOBUF');", "CREATE STREAM TEST_STREAM (ID BIGINT KEY, F1 varchar, F2 bigint) WITH (kafka_topic='right_topic', value_format='PROTOBUF');", "CREATE STREAM OUTPUT as SELECT ROWKEY as ID, name, value, f1, f2 FROM test t FULL OUTER join TEST_STREAM tt WITHIN 11 seconds on t.id = tt.id;" ], + "post" : { + "sources" : [ { + "name" : "OUTPUT", + "type" : "STREAM", + "schema" : "`ID` BIGINT KEY, `NAME` STRING, `VALUE` BIGINT, `F1` STRING, `F2` BIGINT", + "keyFormat" : { + "format" : "KAFKA" + }, + "valueFormat" : "PROTOBUF", + "keyFeatures" : [ ], + "valueFeatures" : [ ], + "isSource" : false + }, { + "name" : "TEST", + "type" : "STREAM", + "schema" : "`ID` BIGINT KEY, `NAME` STRING, `VALUE` BIGINT", + "keyFormat" : { + "format" : "KAFKA" + }, + "valueFormat" : "PROTOBUF", + "keyFeatures" : [ ], + "valueFeatures" : [ ], + "isSource" : false + }, { + "name" : "TEST_STREAM", + "type" : "STREAM", + "schema" : "`ID` BIGINT KEY, `F1` STRING, `F2` BIGINT", + "keyFormat" : { + "format" : "KAFKA" + }, + "valueFormat" : "PROTOBUF", + "keyFeatures" : [ ], + "valueFeatures" : [ ], + "isSource" : false + } ], + "topics" : { + "topics" : [ { + "name" : "_confluent-ksql-some.ksql.service.idquery_CSAS_OUTPUT_0-KSTREAM-OUTERTHIS-0000000008-store-changelog", + "keyFormat" : { + "format" : "KAFKA" + }, + "valueFormat" : { + "format" : "PROTOBUF", + "properties" : { + "unwrapPrimitives" : "true" + } + }, + "valueSchema" : "syntax = \"proto3\";\n\nmessage ConnectDefault1 {\n string T_NAME = 1;\n int64 T_VALUE = 2;\n int64 T_ROWTIME = 3;\n int32 T_ROWPARTITION = 4;\n int64 T_ROWOFFSET = 5;\n int64 T_ID = 6;\n}\n" + }, { + "name" : "left_topic", + "keyFormat" : { + "format" : "KAFKA" + }, + "valueFormat" : { + "format" : "PROTOBUF", + "properties" : { + "unwrapPrimitives" : "true" + } + }, + "partitions" : 4, + "valueSchema" : "syntax = \"proto3\";\n\nmessage ConnectDefault1 {\n string NAME = 1;\n int64 VALUE = 2;\n}\n" + }, { + "name" : "OUTPUT", + "keyFormat" : { + "format" : "KAFKA" + }, + "valueFormat" : { + "format" : "PROTOBUF", + "properties" : { + "unwrapPrimitives" : "true" + } + }, + "partitions" : 4, + "valueSchema" : "syntax = \"proto3\";\n\nmessage ConnectDefault1 {\n string NAME = 1;\n int64 VALUE = 2;\n string F1 = 3;\n int64 F2 = 4;\n}\n" + }, { + "name" : "right_topic", + "keyFormat" : { + "format" : "KAFKA" + }, + "valueFormat" : { + "format" : "PROTOBUF", + "properties" : { + "unwrapPrimitives" : "true" + } + }, + "partitions" : 4, + "valueSchema" : "syntax = \"proto3\";\n\nmessage ConnectDefault1 {\n string F1 = 1;\n int64 F2 = 2;\n}\n" + }, { + "name" : "_confluent-ksql-some.ksql.service.idquery_CSAS_OUTPUT_0-KSTREAM-OUTEROTHER-0000000009-store-changelog", + "keyFormat" : { + "format" : "KAFKA" + }, + "valueFormat" : { + "format" : "PROTOBUF", + "properties" : { + "unwrapPrimitives" : "true" + } + }, + "valueSchema" : "syntax = \"proto3\";\n\nmessage ConnectDefault1 {\n string TT_F1 = 1;\n int64 TT_F2 = 2;\n int64 TT_ROWTIME = 3;\n int32 TT_ROWPARTITION = 4;\n int64 TT_ROWOFFSET = 5;\n int64 TT_ID = 6;\n}\n" + } ], + "blackList" : ".*-repartition" + } + } + } +} \ No newline at end of file diff --git a/ksqldb-functional-tests/src/test/resources/historical_plans/joins_-_stream_stream_outer_join_-_PROTOBUF_-_PROTOBUF/7.3.0_1651071348899/topology b/ksqldb-functional-tests/src/test/resources/historical_plans/joins_-_stream_stream_outer_join_-_PROTOBUF_-_PROTOBUF/7.3.0_1651071348899/topology new file mode 100644 index 000000000000..7f807aeede02 --- /dev/null +++ b/ksqldb-functional-tests/src/test/resources/historical_plans/joins_-_stream_stream_outer_join_-_PROTOBUF_-_PROTOBUF/7.3.0_1651071348899/topology @@ -0,0 +1,39 @@ +Topologies: + Sub-topology: 0 + Source: KSTREAM-SOURCE-0000000000 (topics: [left_topic]) + --> KSTREAM-TRANSFORMVALUES-0000000001 + Source: KSTREAM-SOURCE-0000000003 (topics: [right_topic]) + --> KSTREAM-TRANSFORMVALUES-0000000004 + Processor: KSTREAM-TRANSFORMVALUES-0000000001 (stores: []) + --> PrependAliasLeft + <-- KSTREAM-SOURCE-0000000000 + Processor: KSTREAM-TRANSFORMVALUES-0000000004 (stores: []) + --> PrependAliasRight + <-- KSTREAM-SOURCE-0000000003 + Processor: PrependAliasLeft (stores: []) + --> Join-this-windowed + <-- KSTREAM-TRANSFORMVALUES-0000000001 + Processor: PrependAliasRight (stores: []) + --> Join-other-windowed + <-- KSTREAM-TRANSFORMVALUES-0000000004 + Processor: Join-other-windowed (stores: [KSTREAM-OUTEROTHER-0000000009-store]) + --> Join-outer-other-join + <-- PrependAliasRight + Processor: Join-this-windowed (stores: [KSTREAM-OUTERTHIS-0000000008-store]) + --> Join-outer-this-join + <-- PrependAliasLeft + Processor: Join-outer-other-join (stores: [KSTREAM-OUTERTHIS-0000000008-store]) + --> Join-merge + <-- Join-other-windowed + Processor: Join-outer-this-join (stores: [KSTREAM-OUTEROTHER-0000000009-store]) + --> Join-merge + <-- Join-this-windowed + Processor: Join-merge (stores: []) + --> Project + <-- Join-outer-this-join, Join-outer-other-join + Processor: Project (stores: []) + --> KSTREAM-SINK-0000000012 + <-- Join-merge + Sink: KSTREAM-SINK-0000000012 (topic: OUTPUT) + <-- Project + diff --git a/ksqldb-functional-tests/src/test/resources/historical_plans/joins_-_stream_stream_outer_join_-_PROTOBUF_-_PROTOBUF_NOSR/7.3.0_1651071349988/plan.json b/ksqldb-functional-tests/src/test/resources/historical_plans/joins_-_stream_stream_outer_join_-_PROTOBUF_-_PROTOBUF_NOSR/7.3.0_1651071349988/plan.json new file mode 100644 index 000000000000..729ab8df2250 --- /dev/null +++ b/ksqldb-functional-tests/src/test/resources/historical_plans/joins_-_stream_stream_outer_join_-_PROTOBUF_-_PROTOBUF_NOSR/7.3.0_1651071349988/plan.json @@ -0,0 +1,296 @@ +{ + "plan" : [ { + "@type" : "ksqlPlanV1", + "statementText" : "CREATE STREAM TEST (ID BIGINT KEY, NAME STRING, VALUE BIGINT) WITH (KAFKA_TOPIC='left_topic', KEY_FORMAT='KAFKA', VALUE_FORMAT='PROTOBUF_NOSR');", + "ddlCommand" : { + "@type" : "createStreamV1", + "sourceName" : "TEST", + "schema" : "`ID` BIGINT KEY, `NAME` STRING, `VALUE` BIGINT", + "timestampColumn" : null, + "topicName" : "left_topic", + "formats" : { + "keyFormat" : { + "format" : "KAFKA", + "properties" : { } + }, + "valueFormat" : { + "format" : "PROTOBUF_NOSR", + "properties" : { } + } + }, + "windowInfo" : null, + "orReplace" : false, + "isSource" : false + }, + "queryPlan" : null + }, { + "@type" : "ksqlPlanV1", + "statementText" : "CREATE STREAM TEST_STREAM (ID BIGINT KEY, F1 STRING, F2 BIGINT) WITH (KAFKA_TOPIC='right_topic', KEY_FORMAT='KAFKA', VALUE_FORMAT='PROTOBUF_NOSR');", + "ddlCommand" : { + "@type" : "createStreamV1", + "sourceName" : "TEST_STREAM", + "schema" : "`ID` BIGINT KEY, `F1` STRING, `F2` BIGINT", + "timestampColumn" : null, + "topicName" : "right_topic", + "formats" : { + "keyFormat" : { + "format" : "KAFKA", + "properties" : { } + }, + "valueFormat" : { + "format" : "PROTOBUF_NOSR", + "properties" : { } + } + }, + "windowInfo" : null, + "orReplace" : false, + "isSource" : false + }, + "queryPlan" : null + }, { + "@type" : "ksqlPlanV1", + "statementText" : "CREATE STREAM OUTPUT AS SELECT\n ROWKEY ID,\n T.NAME NAME,\n T.VALUE VALUE,\n TT.F1 F1,\n TT.F2 F2\nFROM TEST T\nFULL OUTER JOIN TEST_STREAM TT WITHIN 11 SECONDS ON ((T.ID = TT.ID))\nEMIT CHANGES", + "ddlCommand" : { + "@type" : "createStreamV1", + "sourceName" : "OUTPUT", + "schema" : "`ID` BIGINT KEY, `NAME` STRING, `VALUE` BIGINT, `F1` STRING, `F2` BIGINT", + "timestampColumn" : null, + "topicName" : "OUTPUT", + "formats" : { + "keyFormat" : { + "format" : "KAFKA", + "properties" : { } + }, + "valueFormat" : { + "format" : "PROTOBUF_NOSR", + "properties" : { } + } + }, + "windowInfo" : null, + "orReplace" : false, + "isSource" : false + }, + "queryPlan" : { + "sources" : [ "TEST", "TEST_STREAM" ], + "sink" : "OUTPUT", + "physicalPlan" : { + "@type" : "streamSinkV1", + "properties" : { + "queryContext" : "OUTPUT" + }, + "source" : { + "@type" : "streamSelectV1", + "properties" : { + "queryContext" : "Project" + }, + "source" : { + "@type" : "streamStreamJoinV1", + "properties" : { + "queryContext" : "Join" + }, + "joinType" : "OUTER", + "leftInternalFormats" : { + "keyFormat" : { + "format" : "KAFKA", + "properties" : { } + }, + "valueFormat" : { + "format" : "PROTOBUF_NOSR", + "properties" : { } + } + }, + "rightInternalFormats" : { + "keyFormat" : { + "format" : "KAFKA", + "properties" : { } + }, + "valueFormat" : { + "format" : "PROTOBUF_NOSR", + "properties" : { } + } + }, + "leftSource" : { + "@type" : "streamSelectV1", + "properties" : { + "queryContext" : "PrependAliasLeft" + }, + "source" : { + "@type" : "streamSourceV1", + "properties" : { + "queryContext" : "KafkaTopic_Left/Source" + }, + "topicName" : "left_topic", + "formats" : { + "keyFormat" : { + "format" : "KAFKA", + "properties" : { } + }, + "valueFormat" : { + "format" : "PROTOBUF_NOSR", + "properties" : { } + } + }, + "timestampColumn" : null, + "sourceSchema" : "`ID` BIGINT KEY, `NAME` STRING, `VALUE` BIGINT", + "pseudoColumnVersion" : 1 + }, + "keyColumnNames" : [ "T_ID" ], + "selectedKeys" : null, + "selectExpressions" : [ "NAME AS T_NAME", "VALUE AS T_VALUE", "ROWTIME AS T_ROWTIME", "ROWPARTITION AS T_ROWPARTITION", "ROWOFFSET AS T_ROWOFFSET", "ID AS T_ID" ] + }, + "rightSource" : { + "@type" : "streamSelectV1", + "properties" : { + "queryContext" : "PrependAliasRight" + }, + "source" : { + "@type" : "streamSourceV1", + "properties" : { + "queryContext" : "KafkaTopic_Right/Source" + }, + "topicName" : "right_topic", + "formats" : { + "keyFormat" : { + "format" : "KAFKA", + "properties" : { } + }, + "valueFormat" : { + "format" : "PROTOBUF_NOSR", + "properties" : { } + } + }, + "timestampColumn" : null, + "sourceSchema" : "`ID` BIGINT KEY, `F1` STRING, `F2` BIGINT", + "pseudoColumnVersion" : 1 + }, + "keyColumnNames" : [ "TT_ID" ], + "selectedKeys" : null, + "selectExpressions" : [ "F1 AS TT_F1", "F2 AS TT_F2", "ROWTIME AS TT_ROWTIME", "ROWPARTITION AS TT_ROWPARTITION", "ROWOFFSET AS TT_ROWOFFSET", "ID AS TT_ID" ] + }, + "beforeMillis" : 11.000000000, + "afterMillis" : 11.000000000, + "graceMillis" : null, + "keyColName" : "ROWKEY" + }, + "keyColumnNames" : [ "ID" ], + "selectedKeys" : null, + "selectExpressions" : [ "T_NAME AS NAME", "T_VALUE AS VALUE", "TT_F1 AS F1", "TT_F2 AS F2" ] + }, + "formats" : { + "keyFormat" : { + "format" : "KAFKA", + "properties" : { } + }, + "valueFormat" : { + "format" : "PROTOBUF_NOSR", + "properties" : { } + } + }, + "topicName" : "OUTPUT", + "timestampColumn" : null + }, + "queryId" : "CSAS_OUTPUT_0", + "runtimeId" : null + } + } ], + "configs" : { + "ksql.extension.dir" : "ext", + "ksql.streams.cache.max.bytes.buffering" : "0", + "metric.reporters" : "", + "ksql.query.status.running.threshold.seconds" : "300", + "ksql.connect.basic.auth.credentials.reload" : "false", + "ksql.output.topic.name.prefix" : "", + "ksql.query.pull.stream.enabled" : "true", + "ksql.query.push.v2.interpreter.enabled" : "true", + "ksql.queryanonymizer.logs_enabled" : "true", + "ksql.variable.substitution.enable" : "true", + "ksql.streams.shutdown.timeout.ms" : "300000", + "ksql.query.pull.max.allowed.offset.lag" : "9223372036854775807", + "ksql.query.pull.max.qps" : "2147483647", + "ksql.access.validator.enable" : "auto", + "ksql.streams.bootstrap.servers" : "localhost:0", + "ksql.query.pull.metrics.enabled" : "true", + "ksql.metrics.extension" : null, + "ksql.query.push.v2.alos.enabled" : "true", + "ksql.query.push.v2.max.hourly.bandwidth.megabytes" : "2147483647", + "ksql.query.pull.range.scan.enabled" : "true", + "ksql.transient.query.cleanup.service.initial.delay.seconds" : "600", + "ksql.hidden.topics" : "_confluent.*,__confluent.*,_schemas,__consumer_offsets,__transaction_state,connect-configs,connect-offsets,connect-status,connect-statuses", + "ksql.lambdas.enabled" : "true", + "ksql.source.table.materialization.enabled" : "true", + "ksql.query.pull.max.hourly.bandwidth.megabytes" : "2147483647", + "ksql.sink.window.change.log.additional.retention" : "1000000", + "ksql.query.persistent.active.limit" : "2147483647", + "ksql.persistence.wrap.single.values" : null, + "ksql.query.transient.max.bytes.buffering.total" : "-1", + "ksql.connect.basic.auth.credentials.source" : "NONE", + "ksql.schema.registry.url" : "schema_registry.url:0", + "ksql.properties.overrides.denylist" : "", + "ksql.service.id" : "some.ksql.service.id", + "ksql.query.push.v2.max.catchup.consumers" : "5", + "ksql.query.push.v2.enabled" : "false", + "ksql.transient.query.cleanup.service.enable" : "true", + "ksql.query.push.v2.metrics.enabled" : "true", + "ksql.rowpartition.rowoffset.enabled" : "true", + "ksql.streams.commit.interval.ms" : "2000", + "ksql.query.pull.table.scan.enabled" : "true", + "ksql.streams.auto.commit.interval.ms" : "0", + "ksql.streams.topology.optimization" : "all", + "ksql.endpoint.migrate.query" : "true", + "ksql.query.push.v2.registry.installed" : "false", + "ksql.streams.num.stream.threads" : "4", + "ksql.metrics.tags.custom" : "", + "ksql.query.push.v2.catchup.consumer.msg.window" : "50", + "ksql.runtime.feature.shared.enabled" : "false", + "ksql.udf.collect.metrics" : "false", + "ksql.new.query.planner.enabled" : "false", + "ksql.connect.request.headers.plugin" : null, + "ksql.security.extension.class" : null, + "ksql.transient.prefix" : "transient_", + "ksql.headers.columns.enabled" : "true", + "ksql.streams.default.deserialization.exception.handler" : "io.confluent.ksql.errors.LogMetricAndContinueExceptionHandler", + "ksql.connect.request.timeout.ms" : "5000", + "ksql.query.pull.enable.standby.reads" : "false", + "ksql.persistence.default.format.key" : "KAFKA", + "ksql.query.persistent.max.bytes.buffering.total" : "-1", + "ksql.query.error.max.queue.size" : "10", + "ksql.query.cleanup.shutdown.timeout.ms" : "30000", + "ksql.internal.topic.min.insync.replicas" : "1", + "ksql.internal.topic.replicas" : "1", + "ksql.insert.into.values.enabled" : "true", + "ksql.queryanonymizer.cluster_namespace" : null, + "ksql.create.or.replace.enabled" : "true", + "ksql.shared.runtimes.count" : "2", + "ksql.cast.strings.preserve.nulls" : "true", + "ksql.authorization.cache.max.entries" : "10000", + "ksql.pull.queries.enable" : "true", + "ksql.transient.query.cleanup.service.period.seconds" : "600", + "ksql.suppress.enabled" : "false", + "ksql.readonly.topics" : "_confluent.*,__confluent.*,_schemas,__consumer_offsets,__transaction_state,connect-configs,connect-offsets,connect-status,connect-statuses", + "ksql.connect.basic.auth.credentials.file" : "", + "ksql.authorization.cache.expiry.time.secs" : "30", + "ksql.query.retry.backoff.initial.ms" : "15000", + "ksql.query.pull.max.concurrent.requests" : "2147483647", + "ksql.streams.auto.offset.reset" : "earliest", + "ksql.connect.url" : "http://localhost:8083", + "ksql.query.push.v2.new.latest.delay.ms" : "5000", + "ksql.query.push.v2.latest.reset.age.ms" : "30000", + "ksql.streams.default.production.exception.handler" : "io.confluent.ksql.errors.ProductionExceptionHandlerUtil$LogAndFailProductionExceptionHandler", + "ksql.query.pull.interpreter.enabled" : "true", + "ksql.query.pull.limit.clause.enabled" : "true", + "ksql.query.pull.router.thread.pool.size" : "50", + "ksql.query.push.v2.continuation.tokens.enabled" : "false", + "ksql.query.retry.backoff.max.ms" : "900000", + "ksql.timestamp.throw.on.invalid" : "false", + "ksql.persistence.default.format.value" : null, + "ksql.udfs.enabled" : "true", + "ksql.udf.enable.security.manager" : "true", + "ksql.connect.worker.config" : "", + "ksql.nested.error.set.null" : "true", + "ksql.query.pull.thread.pool.size" : "50", + "ksql.persistent.prefix" : "query_", + "ksql.metastore.backup.location" : "", + "ksql.error.classifier.regex" : "", + "ksql.suppress.buffer.size.bytes" : "-1" + } +} \ No newline at end of file diff --git a/ksqldb-functional-tests/src/test/resources/historical_plans/joins_-_stream_stream_outer_join_-_PROTOBUF_-_PROTOBUF_NOSR/7.3.0_1651071349988/spec.json b/ksqldb-functional-tests/src/test/resources/historical_plans/joins_-_stream_stream_outer_join_-_PROTOBUF_-_PROTOBUF_NOSR/7.3.0_1651071349988/spec.json new file mode 100644 index 000000000000..886e2da79d2f --- /dev/null +++ b/ksqldb-functional-tests/src/test/resources/historical_plans/joins_-_stream_stream_outer_join_-_PROTOBUF_-_PROTOBUF_NOSR/7.3.0_1651071349988/spec.json @@ -0,0 +1,298 @@ +{ + "version" : "7.3.0", + "timestamp" : 1651071349988, + "path" : "query-validation-tests/joins.json", + "schemas" : { + "CSAS_OUTPUT_0.KafkaTopic_Right.Source" : { + "schema" : "`ID` BIGINT KEY, `F1` STRING, `F2` BIGINT", + "keyFormat" : { + "format" : "KAFKA" + }, + "valueFormat" : { + "format" : "PROTOBUF_NOSR" + } + }, + "CSAS_OUTPUT_0.Join.Left" : { + "schema" : "`T_ID` BIGINT KEY, `T_NAME` STRING, `T_VALUE` BIGINT, `T_ROWTIME` BIGINT, `T_ROWPARTITION` INTEGER, `T_ROWOFFSET` BIGINT, `T_ID` BIGINT", + "valueFormat" : { + "format" : "PROTOBUF_NOSR" + } + }, + "CSAS_OUTPUT_0.OUTPUT" : { + "schema" : "`ID` BIGINT KEY, `NAME` STRING, `VALUE` BIGINT, `F1` STRING, `F2` BIGINT", + "keyFormat" : { + "format" : "KAFKA" + }, + "valueFormat" : { + "format" : "PROTOBUF_NOSR" + } + }, + "CSAS_OUTPUT_0.KafkaTopic_Left.Source" : { + "schema" : "`ID` BIGINT KEY, `NAME` STRING, `VALUE` BIGINT", + "keyFormat" : { + "format" : "KAFKA" + }, + "valueFormat" : { + "format" : "PROTOBUF_NOSR" + } + }, + "CSAS_OUTPUT_0.Join" : { + "schema" : "`T_ID` BIGINT KEY, `T_NAME` STRING, `T_VALUE` BIGINT, `T_ROWTIME` BIGINT, `T_ROWPARTITION` INTEGER, `T_ROWOFFSET` BIGINT, `T_ID` BIGINT", + "keyFormat" : { + "format" : "KAFKA" + } + }, + "CSAS_OUTPUT_0.Join.Right" : { + "schema" : "`TT_ID` BIGINT KEY, `TT_F1` STRING, `TT_F2` BIGINT, `TT_ROWTIME` BIGINT, `TT_ROWPARTITION` INTEGER, `TT_ROWOFFSET` BIGINT, `TT_ID` BIGINT", + "valueFormat" : { + "format" : "PROTOBUF_NOSR" + } + } + }, + "testCase" : { + "name" : "stream stream outer join - PROTOBUF - PROTOBUF_NOSR", + "inputs" : [ { + "topic" : "left_topic", + "key" : 0, + "value" : { + "NAME" : "zero", + "VALUE" : 0 + }, + "timestamp" : 0 + }, { + "topic" : "right_topic", + "key" : 0, + "value" : { + "F1" : "blah", + "F2" : 50 + }, + "timestamp" : 10000 + }, { + "topic" : "left_topic", + "key" : 10, + "value" : { + "NAME" : "100", + "VALUE" : 5 + }, + "timestamp" : 11000 + }, { + "topic" : "left_topic", + "key" : 0, + "value" : { + "NAME" : "foo", + "VALUE" : 100 + }, + "timestamp" : 13000 + }, { + "topic" : "right_topic", + "key" : 0, + "value" : { + "F1" : "a", + "F2" : 10 + }, + "timestamp" : 15000 + }, { + "topic" : "left_topic", + "key" : 0, + "value" : { + "NAME" : "bar", + "VALUE" : 99 + }, + "timestamp" : 30000 + }, { + "topic" : "left_topic", + "key" : 90, + "value" : { + "NAME" : "ninety", + "VALUE" : 90 + }, + "timestamp" : 17000 + }, { + "topic" : "right_topic", + "key" : 100, + "value" : { + "F1" : "newblah", + "F2" : 150 + }, + "timestamp" : 20000 + } ], + "outputs" : [ { + "topic" : "OUTPUT", + "key" : 0, + "value" : { + "NAME" : "zero", + "VALUE" : 0, + "F1" : "", + "F2" : 0 + }, + "timestamp" : 0 + }, { + "topic" : "OUTPUT", + "key" : 0, + "value" : { + "NAME" : "zero", + "VALUE" : 0, + "F1" : "blah", + "F2" : 50 + }, + "timestamp" : 10000 + }, { + "topic" : "OUTPUT", + "key" : 10, + "value" : { + "NAME" : "100", + "VALUE" : 5, + "F1" : "", + "F2" : 0 + }, + "timestamp" : 11000 + }, { + "topic" : "OUTPUT", + "key" : 0, + "value" : { + "NAME" : "foo", + "VALUE" : 100, + "F1" : "blah", + "F2" : 50 + }, + "timestamp" : 13000 + }, { + "topic" : "OUTPUT", + "key" : 0, + "value" : { + "NAME" : "foo", + "VALUE" : 100, + "F1" : "a", + "F2" : 10 + }, + "timestamp" : 15000 + }, { + "topic" : "OUTPUT", + "key" : 0, + "value" : { + "NAME" : "bar", + "VALUE" : 99, + "F1" : "", + "F2" : 0 + }, + "timestamp" : 30000 + }, { + "topic" : "OUTPUT", + "key" : 90, + "value" : { + "NAME" : "ninety", + "VALUE" : 90, + "F1" : "", + "F2" : 0 + }, + "timestamp" : 17000 + }, { + "topic" : "OUTPUT", + "key" : 100, + "value" : { + "NAME" : "", + "VALUE" : 0, + "F1" : "newblah", + "F2" : 150 + }, + "timestamp" : 20000 + } ], + "topics" : [ { + "name" : "right_topic", + "replicas" : 1, + "numPartitions" : 4 + }, { + "name" : "OUTPUT", + "replicas" : 1, + "numPartitions" : 4 + }, { + "name" : "left_topic", + "replicas" : 1, + "numPartitions" : 4 + } ], + "statements" : [ "CREATE STREAM TEST (ID BIGINT KEY, NAME varchar, VALUE bigint) WITH (kafka_topic='left_topic', value_format='PROTOBUF_NOSR');", "CREATE STREAM TEST_STREAM (ID BIGINT KEY, F1 varchar, F2 bigint) WITH (kafka_topic='right_topic', value_format='PROTOBUF_NOSR');", "CREATE STREAM OUTPUT as SELECT ROWKEY as ID, name, value, f1, f2 FROM test t FULL OUTER join TEST_STREAM tt WITHIN 11 seconds on t.id = tt.id;" ], + "post" : { + "sources" : [ { + "name" : "OUTPUT", + "type" : "STREAM", + "schema" : "`ID` BIGINT KEY, `NAME` STRING, `VALUE` BIGINT, `F1` STRING, `F2` BIGINT", + "keyFormat" : { + "format" : "KAFKA" + }, + "valueFormat" : "PROTOBUF_NOSR", + "keyFeatures" : [ ], + "valueFeatures" : [ ], + "isSource" : false + }, { + "name" : "TEST", + "type" : "STREAM", + "schema" : "`ID` BIGINT KEY, `NAME` STRING, `VALUE` BIGINT", + "keyFormat" : { + "format" : "KAFKA" + }, + "valueFormat" : "PROTOBUF_NOSR", + "keyFeatures" : [ ], + "valueFeatures" : [ ], + "isSource" : false + }, { + "name" : "TEST_STREAM", + "type" : "STREAM", + "schema" : "`ID` BIGINT KEY, `F1` STRING, `F2` BIGINT", + "keyFormat" : { + "format" : "KAFKA" + }, + "valueFormat" : "PROTOBUF_NOSR", + "keyFeatures" : [ ], + "valueFeatures" : [ ], + "isSource" : false + } ], + "topics" : { + "topics" : [ { + "name" : "_confluent-ksql-some.ksql.service.idquery_CSAS_OUTPUT_0-KSTREAM-OUTERTHIS-0000000008-store-changelog", + "keyFormat" : { + "format" : "KAFKA" + }, + "valueFormat" : { + "format" : "PROTOBUF_NOSR" + } + }, { + "name" : "left_topic", + "keyFormat" : { + "format" : "KAFKA" + }, + "valueFormat" : { + "format" : "PROTOBUF_NOSR" + }, + "partitions" : 4 + }, { + "name" : "OUTPUT", + "keyFormat" : { + "format" : "KAFKA" + }, + "valueFormat" : { + "format" : "PROTOBUF_NOSR" + }, + "partitions" : 4 + }, { + "name" : "right_topic", + "keyFormat" : { + "format" : "KAFKA" + }, + "valueFormat" : { + "format" : "PROTOBUF_NOSR" + }, + "partitions" : 4 + }, { + "name" : "_confluent-ksql-some.ksql.service.idquery_CSAS_OUTPUT_0-KSTREAM-OUTEROTHER-0000000009-store-changelog", + "keyFormat" : { + "format" : "KAFKA" + }, + "valueFormat" : { + "format" : "PROTOBUF_NOSR" + } + } ], + "blackList" : ".*-repartition" + } + } + } +} \ No newline at end of file diff --git a/ksqldb-functional-tests/src/test/resources/historical_plans/joins_-_stream_stream_outer_join_-_PROTOBUF_-_PROTOBUF_NOSR/7.3.0_1651071349988/topology b/ksqldb-functional-tests/src/test/resources/historical_plans/joins_-_stream_stream_outer_join_-_PROTOBUF_-_PROTOBUF_NOSR/7.3.0_1651071349988/topology new file mode 100644 index 000000000000..7f807aeede02 --- /dev/null +++ b/ksqldb-functional-tests/src/test/resources/historical_plans/joins_-_stream_stream_outer_join_-_PROTOBUF_-_PROTOBUF_NOSR/7.3.0_1651071349988/topology @@ -0,0 +1,39 @@ +Topologies: + Sub-topology: 0 + Source: KSTREAM-SOURCE-0000000000 (topics: [left_topic]) + --> KSTREAM-TRANSFORMVALUES-0000000001 + Source: KSTREAM-SOURCE-0000000003 (topics: [right_topic]) + --> KSTREAM-TRANSFORMVALUES-0000000004 + Processor: KSTREAM-TRANSFORMVALUES-0000000001 (stores: []) + --> PrependAliasLeft + <-- KSTREAM-SOURCE-0000000000 + Processor: KSTREAM-TRANSFORMVALUES-0000000004 (stores: []) + --> PrependAliasRight + <-- KSTREAM-SOURCE-0000000003 + Processor: PrependAliasLeft (stores: []) + --> Join-this-windowed + <-- KSTREAM-TRANSFORMVALUES-0000000001 + Processor: PrependAliasRight (stores: []) + --> Join-other-windowed + <-- KSTREAM-TRANSFORMVALUES-0000000004 + Processor: Join-other-windowed (stores: [KSTREAM-OUTEROTHER-0000000009-store]) + --> Join-outer-other-join + <-- PrependAliasRight + Processor: Join-this-windowed (stores: [KSTREAM-OUTERTHIS-0000000008-store]) + --> Join-outer-this-join + <-- PrependAliasLeft + Processor: Join-outer-other-join (stores: [KSTREAM-OUTERTHIS-0000000008-store]) + --> Join-merge + <-- Join-other-windowed + Processor: Join-outer-this-join (stores: [KSTREAM-OUTEROTHER-0000000009-store]) + --> Join-merge + <-- Join-this-windowed + Processor: Join-merge (stores: []) + --> Project + <-- Join-outer-this-join, Join-outer-other-join + Processor: Project (stores: []) + --> KSTREAM-SINK-0000000012 + <-- Join-merge + Sink: KSTREAM-SINK-0000000012 (topic: OUTPUT) + <-- Project + diff --git a/ksqldb-functional-tests/src/test/resources/historical_plans/joins_-_stream_stream_right_join_-_PROTOBUF_-_PROTOBUF_NOSR/7.3.0_1651071291496/plan.json b/ksqldb-functional-tests/src/test/resources/historical_plans/joins_-_stream_stream_right_join_-_PROTOBUF_-_PROTOBUF_NOSR/7.3.0_1651071291496/plan.json new file mode 100644 index 000000000000..c56483b7763f --- /dev/null +++ b/ksqldb-functional-tests/src/test/resources/historical_plans/joins_-_stream_stream_right_join_-_PROTOBUF_-_PROTOBUF_NOSR/7.3.0_1651071291496/plan.json @@ -0,0 +1,296 @@ +{ + "plan" : [ { + "@type" : "ksqlPlanV1", + "statementText" : "CREATE STREAM TEST (ID BIGINT KEY, NAME STRING, VALUE BIGINT) WITH (KAFKA_TOPIC='left_topic', KEY_FORMAT='KAFKA', VALUE_FORMAT='PROTOBUF_NOSR');", + "ddlCommand" : { + "@type" : "createStreamV1", + "sourceName" : "TEST", + "schema" : "`ID` BIGINT KEY, `NAME` STRING, `VALUE` BIGINT", + "timestampColumn" : null, + "topicName" : "left_topic", + "formats" : { + "keyFormat" : { + "format" : "KAFKA", + "properties" : { } + }, + "valueFormat" : { + "format" : "PROTOBUF_NOSR", + "properties" : { } + } + }, + "windowInfo" : null, + "orReplace" : false, + "isSource" : false + }, + "queryPlan" : null + }, { + "@type" : "ksqlPlanV1", + "statementText" : "CREATE STREAM TEST_STREAM (ID BIGINT KEY, F1 STRING, F2 BIGINT) WITH (KAFKA_TOPIC='right_topic', KEY_FORMAT='KAFKA', VALUE_FORMAT='PROTOBUF_NOSR');", + "ddlCommand" : { + "@type" : "createStreamV1", + "sourceName" : "TEST_STREAM", + "schema" : "`ID` BIGINT KEY, `F1` STRING, `F2` BIGINT", + "timestampColumn" : null, + "topicName" : "right_topic", + "formats" : { + "keyFormat" : { + "format" : "KAFKA", + "properties" : { } + }, + "valueFormat" : { + "format" : "PROTOBUF_NOSR", + "properties" : { } + } + }, + "windowInfo" : null, + "orReplace" : false, + "isSource" : false + }, + "queryPlan" : null + }, { + "@type" : "ksqlPlanV1", + "statementText" : "CREATE STREAM OUTPUT AS SELECT\n T.ID T_ID,\n T.NAME NAME,\n T.VALUE VALUE,\n TT.F1 F1,\n TT.F2 F2\nFROM TEST T\nRIGHT OUTER JOIN TEST_STREAM TT WITHIN 11 SECONDS ON ((T.ID = TT.ID))\nEMIT CHANGES", + "ddlCommand" : { + "@type" : "createStreamV1", + "sourceName" : "OUTPUT", + "schema" : "`T_ID` BIGINT KEY, `NAME` STRING, `VALUE` BIGINT, `F1` STRING, `F2` BIGINT", + "timestampColumn" : null, + "topicName" : "OUTPUT", + "formats" : { + "keyFormat" : { + "format" : "KAFKA", + "properties" : { } + }, + "valueFormat" : { + "format" : "PROTOBUF_NOSR", + "properties" : { } + } + }, + "windowInfo" : null, + "orReplace" : false, + "isSource" : false + }, + "queryPlan" : { + "sources" : [ "TEST", "TEST_STREAM" ], + "sink" : "OUTPUT", + "physicalPlan" : { + "@type" : "streamSinkV1", + "properties" : { + "queryContext" : "OUTPUT" + }, + "source" : { + "@type" : "streamSelectV1", + "properties" : { + "queryContext" : "Project" + }, + "source" : { + "@type" : "streamStreamJoinV1", + "properties" : { + "queryContext" : "Join" + }, + "joinType" : "RIGHT", + "leftInternalFormats" : { + "keyFormat" : { + "format" : "KAFKA", + "properties" : { } + }, + "valueFormat" : { + "format" : "PROTOBUF_NOSR", + "properties" : { } + } + }, + "rightInternalFormats" : { + "keyFormat" : { + "format" : "KAFKA", + "properties" : { } + }, + "valueFormat" : { + "format" : "PROTOBUF_NOSR", + "properties" : { } + } + }, + "leftSource" : { + "@type" : "streamSelectV1", + "properties" : { + "queryContext" : "PrependAliasLeft" + }, + "source" : { + "@type" : "streamSourceV1", + "properties" : { + "queryContext" : "KafkaTopic_Left/Source" + }, + "topicName" : "left_topic", + "formats" : { + "keyFormat" : { + "format" : "KAFKA", + "properties" : { } + }, + "valueFormat" : { + "format" : "PROTOBUF_NOSR", + "properties" : { } + } + }, + "timestampColumn" : null, + "sourceSchema" : "`ID` BIGINT KEY, `NAME` STRING, `VALUE` BIGINT", + "pseudoColumnVersion" : 1 + }, + "keyColumnNames" : [ "T_ID" ], + "selectedKeys" : null, + "selectExpressions" : [ "NAME AS T_NAME", "VALUE AS T_VALUE", "ROWTIME AS T_ROWTIME", "ROWPARTITION AS T_ROWPARTITION", "ROWOFFSET AS T_ROWOFFSET", "ID AS T_ID" ] + }, + "rightSource" : { + "@type" : "streamSelectV1", + "properties" : { + "queryContext" : "PrependAliasRight" + }, + "source" : { + "@type" : "streamSourceV1", + "properties" : { + "queryContext" : "KafkaTopic_Right/Source" + }, + "topicName" : "right_topic", + "formats" : { + "keyFormat" : { + "format" : "KAFKA", + "properties" : { } + }, + "valueFormat" : { + "format" : "PROTOBUF_NOSR", + "properties" : { } + } + }, + "timestampColumn" : null, + "sourceSchema" : "`ID` BIGINT KEY, `F1` STRING, `F2` BIGINT", + "pseudoColumnVersion" : 1 + }, + "keyColumnNames" : [ "TT_ID" ], + "selectedKeys" : null, + "selectExpressions" : [ "F1 AS TT_F1", "F2 AS TT_F2", "ROWTIME AS TT_ROWTIME", "ROWPARTITION AS TT_ROWPARTITION", "ROWOFFSET AS TT_ROWOFFSET", "ID AS TT_ID" ] + }, + "beforeMillis" : 11.000000000, + "afterMillis" : 11.000000000, + "graceMillis" : null, + "keyColName" : "T_ID" + }, + "keyColumnNames" : [ "T_ID" ], + "selectedKeys" : null, + "selectExpressions" : [ "T_NAME AS NAME", "T_VALUE AS VALUE", "TT_F1 AS F1", "TT_F2 AS F2" ] + }, + "formats" : { + "keyFormat" : { + "format" : "KAFKA", + "properties" : { } + }, + "valueFormat" : { + "format" : "PROTOBUF_NOSR", + "properties" : { } + } + }, + "topicName" : "OUTPUT", + "timestampColumn" : null + }, + "queryId" : "CSAS_OUTPUT_0", + "runtimeId" : null + } + } ], + "configs" : { + "ksql.extension.dir" : "ext", + "ksql.streams.cache.max.bytes.buffering" : "0", + "metric.reporters" : "", + "ksql.query.status.running.threshold.seconds" : "300", + "ksql.connect.basic.auth.credentials.reload" : "false", + "ksql.output.topic.name.prefix" : "", + "ksql.query.pull.stream.enabled" : "true", + "ksql.query.push.v2.interpreter.enabled" : "true", + "ksql.queryanonymizer.logs_enabled" : "true", + "ksql.variable.substitution.enable" : "true", + "ksql.streams.shutdown.timeout.ms" : "300000", + "ksql.query.pull.max.allowed.offset.lag" : "9223372036854775807", + "ksql.query.pull.max.qps" : "2147483647", + "ksql.access.validator.enable" : "auto", + "ksql.streams.bootstrap.servers" : "localhost:0", + "ksql.query.pull.metrics.enabled" : "true", + "ksql.metrics.extension" : null, + "ksql.query.push.v2.alos.enabled" : "true", + "ksql.query.push.v2.max.hourly.bandwidth.megabytes" : "2147483647", + "ksql.query.pull.range.scan.enabled" : "true", + "ksql.transient.query.cleanup.service.initial.delay.seconds" : "600", + "ksql.hidden.topics" : "_confluent.*,__confluent.*,_schemas,__consumer_offsets,__transaction_state,connect-configs,connect-offsets,connect-status,connect-statuses", + "ksql.lambdas.enabled" : "true", + "ksql.source.table.materialization.enabled" : "true", + "ksql.query.pull.max.hourly.bandwidth.megabytes" : "2147483647", + "ksql.sink.window.change.log.additional.retention" : "1000000", + "ksql.query.persistent.active.limit" : "2147483647", + "ksql.persistence.wrap.single.values" : null, + "ksql.query.transient.max.bytes.buffering.total" : "-1", + "ksql.connect.basic.auth.credentials.source" : "NONE", + "ksql.schema.registry.url" : "schema_registry.url:0", + "ksql.properties.overrides.denylist" : "", + "ksql.service.id" : "some.ksql.service.id", + "ksql.query.push.v2.max.catchup.consumers" : "5", + "ksql.query.push.v2.enabled" : "false", + "ksql.transient.query.cleanup.service.enable" : "true", + "ksql.query.push.v2.metrics.enabled" : "true", + "ksql.rowpartition.rowoffset.enabled" : "true", + "ksql.streams.commit.interval.ms" : "2000", + "ksql.query.pull.table.scan.enabled" : "true", + "ksql.streams.auto.commit.interval.ms" : "0", + "ksql.streams.topology.optimization" : "all", + "ksql.endpoint.migrate.query" : "true", + "ksql.query.push.v2.registry.installed" : "false", + "ksql.streams.num.stream.threads" : "4", + "ksql.metrics.tags.custom" : "", + "ksql.query.push.v2.catchup.consumer.msg.window" : "50", + "ksql.runtime.feature.shared.enabled" : "false", + "ksql.udf.collect.metrics" : "false", + "ksql.new.query.planner.enabled" : "false", + "ksql.connect.request.headers.plugin" : null, + "ksql.security.extension.class" : null, + "ksql.transient.prefix" : "transient_", + "ksql.headers.columns.enabled" : "true", + "ksql.streams.default.deserialization.exception.handler" : "io.confluent.ksql.errors.LogMetricAndContinueExceptionHandler", + "ksql.connect.request.timeout.ms" : "5000", + "ksql.query.pull.enable.standby.reads" : "false", + "ksql.persistence.default.format.key" : "KAFKA", + "ksql.query.persistent.max.bytes.buffering.total" : "-1", + "ksql.query.error.max.queue.size" : "10", + "ksql.query.cleanup.shutdown.timeout.ms" : "30000", + "ksql.internal.topic.min.insync.replicas" : "1", + "ksql.internal.topic.replicas" : "1", + "ksql.insert.into.values.enabled" : "true", + "ksql.queryanonymizer.cluster_namespace" : null, + "ksql.create.or.replace.enabled" : "true", + "ksql.shared.runtimes.count" : "2", + "ksql.cast.strings.preserve.nulls" : "true", + "ksql.authorization.cache.max.entries" : "10000", + "ksql.pull.queries.enable" : "true", + "ksql.transient.query.cleanup.service.period.seconds" : "600", + "ksql.suppress.enabled" : "false", + "ksql.readonly.topics" : "_confluent.*,__confluent.*,_schemas,__consumer_offsets,__transaction_state,connect-configs,connect-offsets,connect-status,connect-statuses", + "ksql.connect.basic.auth.credentials.file" : "", + "ksql.authorization.cache.expiry.time.secs" : "30", + "ksql.query.retry.backoff.initial.ms" : "15000", + "ksql.query.pull.max.concurrent.requests" : "2147483647", + "ksql.streams.auto.offset.reset" : "earliest", + "ksql.connect.url" : "http://localhost:8083", + "ksql.query.push.v2.new.latest.delay.ms" : "5000", + "ksql.query.push.v2.latest.reset.age.ms" : "30000", + "ksql.streams.default.production.exception.handler" : "io.confluent.ksql.errors.ProductionExceptionHandlerUtil$LogAndFailProductionExceptionHandler", + "ksql.query.pull.interpreter.enabled" : "true", + "ksql.query.pull.limit.clause.enabled" : "true", + "ksql.query.pull.router.thread.pool.size" : "50", + "ksql.query.push.v2.continuation.tokens.enabled" : "false", + "ksql.query.retry.backoff.max.ms" : "900000", + "ksql.timestamp.throw.on.invalid" : "false", + "ksql.persistence.default.format.value" : null, + "ksql.udfs.enabled" : "true", + "ksql.udf.enable.security.manager" : "true", + "ksql.connect.worker.config" : "", + "ksql.nested.error.set.null" : "true", + "ksql.query.pull.thread.pool.size" : "50", + "ksql.persistent.prefix" : "query_", + "ksql.metastore.backup.location" : "", + "ksql.error.classifier.regex" : "", + "ksql.suppress.buffer.size.bytes" : "-1" + } +} \ No newline at end of file diff --git a/ksqldb-functional-tests/src/test/resources/historical_plans/joins_-_stream_stream_right_join_-_PROTOBUF_-_PROTOBUF_NOSR/7.3.0_1651071291496/spec.json b/ksqldb-functional-tests/src/test/resources/historical_plans/joins_-_stream_stream_right_join_-_PROTOBUF_-_PROTOBUF_NOSR/7.3.0_1651071291496/spec.json new file mode 100644 index 000000000000..a25784d4f9b4 --- /dev/null +++ b/ksqldb-functional-tests/src/test/resources/historical_plans/joins_-_stream_stream_right_join_-_PROTOBUF_-_PROTOBUF_NOSR/7.3.0_1651071291496/spec.json @@ -0,0 +1,258 @@ +{ + "version" : "7.3.0", + "timestamp" : 1651071291496, + "path" : "query-validation-tests/joins.json", + "schemas" : { + "CSAS_OUTPUT_0.KafkaTopic_Right.Source" : { + "schema" : "`ID` BIGINT KEY, `F1` STRING, `F2` BIGINT", + "keyFormat" : { + "format" : "KAFKA" + }, + "valueFormat" : { + "format" : "PROTOBUF_NOSR" + } + }, + "CSAS_OUTPUT_0.Join.Left" : { + "schema" : "`TT_ID` BIGINT KEY, `TT_F1` STRING, `TT_F2` BIGINT, `TT_ROWTIME` BIGINT, `TT_ROWPARTITION` INTEGER, `TT_ROWOFFSET` BIGINT, `TT_ID` BIGINT", + "valueFormat" : { + "format" : "PROTOBUF_NOSR" + } + }, + "CSAS_OUTPUT_0.OUTPUT" : { + "schema" : "`T_ID` BIGINT KEY, `NAME` STRING, `VALUE` BIGINT, `F1` STRING, `F2` BIGINT", + "keyFormat" : { + "format" : "KAFKA" + }, + "valueFormat" : { + "format" : "PROTOBUF_NOSR" + } + }, + "CSAS_OUTPUT_0.KafkaTopic_Left.Source" : { + "schema" : "`ID` BIGINT KEY, `NAME` STRING, `VALUE` BIGINT", + "keyFormat" : { + "format" : "KAFKA" + }, + "valueFormat" : { + "format" : "PROTOBUF_NOSR" + } + }, + "CSAS_OUTPUT_0.Join" : { + "schema" : "`TT_ID` BIGINT KEY, `TT_F1` STRING, `TT_F2` BIGINT, `TT_ROWTIME` BIGINT, `TT_ROWPARTITION` INTEGER, `TT_ROWOFFSET` BIGINT, `TT_ID` BIGINT", + "keyFormat" : { + "format" : "KAFKA" + } + }, + "CSAS_OUTPUT_0.Join.Right" : { + "schema" : "`T_ID` BIGINT KEY, `T_NAME` STRING, `T_VALUE` BIGINT, `T_ROWTIME` BIGINT, `T_ROWPARTITION` INTEGER, `T_ROWOFFSET` BIGINT, `T_ID` BIGINT", + "valueFormat" : { + "format" : "PROTOBUF_NOSR" + } + } + }, + "testCase" : { + "name" : "stream stream right join - PROTOBUF - PROTOBUF_NOSR", + "inputs" : [ { + "topic" : "left_topic", + "key" : 0, + "value" : { + "NAME" : "zero", + "VALUE" : 0 + }, + "timestamp" : 0 + }, { + "topic" : "right_topic", + "key" : 0, + "value" : { + "F1" : "blah", + "F2" : 50 + }, + "timestamp" : 10000 + }, { + "topic" : "left_topic", + "key" : 10, + "value" : { + "NAME" : "100", + "VALUE" : 5 + }, + "timestamp" : 11000 + }, { + "topic" : "left_topic", + "key" : 0, + "value" : { + "NAME" : "foo", + "VALUE" : 100 + }, + "timestamp" : 13000 + }, { + "topic" : "right_topic", + "key" : 0, + "value" : { + "F1" : "a", + "F2" : 10 + }, + "timestamp" : 15000 + }, { + "topic" : "right_topic", + "key" : 100, + "value" : { + "F1" : "newblah", + "F2" : 150 + }, + "timestamp" : 16000 + }, { + "topic" : "left_topic", + "key" : 90, + "value" : { + "NAME" : "ninety", + "VALUE" : 90 + }, + "timestamp" : 17000 + }, { + "topic" : "left_topic", + "key" : 0, + "value" : { + "NAME" : "bar", + "VALUE" : 99 + }, + "timestamp" : 30000 + } ], + "outputs" : [ { + "topic" : "OUTPUT", + "key" : 0, + "value" : { + "NAME" : "zero", + "VALUE" : 0, + "F1" : "blah", + "F2" : 50 + }, + "timestamp" : 10000 + }, { + "topic" : "OUTPUT", + "key" : 0, + "value" : { + "NAME" : "foo", + "VALUE" : 100, + "F1" : "blah", + "F2" : 50 + }, + "timestamp" : 13000 + }, { + "topic" : "OUTPUT", + "key" : 0, + "value" : { + "NAME" : "foo", + "VALUE" : 100, + "F1" : "a", + "F2" : 10 + }, + "timestamp" : 15000 + }, { + "topic" : "OUTPUT", + "key" : 100, + "value" : { + "NAME" : "", + "VALUE" : 0, + "F1" : "newblah", + "F2" : 150 + }, + "timestamp" : 16000 + } ], + "topics" : [ { + "name" : "right_topic", + "replicas" : 1, + "numPartitions" : 4 + }, { + "name" : "OUTPUT", + "replicas" : 1, + "numPartitions" : 4 + }, { + "name" : "left_topic", + "replicas" : 1, + "numPartitions" : 4 + } ], + "statements" : [ "CREATE STREAM TEST (id BIGINT KEY, name VARCHAR, value BIGINT) WITH (kafka_topic='left_topic', value_format='PROTOBUF_NOSR');", "CREATE STREAM TEST_STREAM (id BIGINT KEY, f1 VARCHAR, f2 BIGINT) WITH (kafka_topic='right_topic', value_format='PROTOBUF_NOSR');", "CREATE STREAM OUTPUT as SELECT t.id, name, value, f1, f2 FROM test t RIGHT JOIN TEST_STREAM tt WITHIN 11 seconds ON t.id = tt.id;" ], + "post" : { + "sources" : [ { + "name" : "OUTPUT", + "type" : "STREAM", + "schema" : "`T_ID` BIGINT KEY, `NAME` STRING, `VALUE` BIGINT, `F1` STRING, `F2` BIGINT", + "keyFormat" : { + "format" : "KAFKA" + }, + "valueFormat" : "PROTOBUF_NOSR", + "keyFeatures" : [ ], + "valueFeatures" : [ ], + "isSource" : false + }, { + "name" : "TEST", + "type" : "STREAM", + "schema" : "`ID` BIGINT KEY, `NAME` STRING, `VALUE` BIGINT", + "keyFormat" : { + "format" : "KAFKA" + }, + "valueFormat" : "PROTOBUF_NOSR", + "keyFeatures" : [ ], + "valueFeatures" : [ ], + "isSource" : false + }, { + "name" : "TEST_STREAM", + "type" : "STREAM", + "schema" : "`ID` BIGINT KEY, `F1` STRING, `F2` BIGINT", + "keyFormat" : { + "format" : "KAFKA" + }, + "valueFormat" : "PROTOBUF_NOSR", + "keyFeatures" : [ ], + "valueFeatures" : [ ], + "isSource" : false + } ], + "topics" : { + "topics" : [ { + "name" : "_confluent-ksql-some.ksql.service.idquery_CSAS_OUTPUT_0-KSTREAM-JOINTHIS-0000000008-store-changelog", + "keyFormat" : { + "format" : "KAFKA" + }, + "valueFormat" : { + "format" : "PROTOBUF_NOSR" + } + }, { + "name" : "left_topic", + "keyFormat" : { + "format" : "KAFKA" + }, + "valueFormat" : { + "format" : "PROTOBUF_NOSR" + }, + "partitions" : 4 + }, { + "name" : "OUTPUT", + "keyFormat" : { + "format" : "KAFKA" + }, + "valueFormat" : { + "format" : "PROTOBUF_NOSR" + }, + "partitions" : 4 + }, { + "name" : "right_topic", + "keyFormat" : { + "format" : "KAFKA" + }, + "valueFormat" : { + "format" : "PROTOBUF_NOSR" + }, + "partitions" : 4 + }, { + "name" : "_confluent-ksql-some.ksql.service.idquery_CSAS_OUTPUT_0-KSTREAM-OUTEROTHER-0000000009-store-changelog", + "keyFormat" : { + "format" : "KAFKA" + }, + "valueFormat" : { + "format" : "PROTOBUF_NOSR" + } + } ], + "blackList" : ".*-repartition" + } + } + } +} \ No newline at end of file diff --git a/ksqldb-functional-tests/src/test/resources/historical_plans/joins_-_stream_stream_right_join_-_PROTOBUF_-_PROTOBUF_NOSR/7.3.0_1651071291496/topology b/ksqldb-functional-tests/src/test/resources/historical_plans/joins_-_stream_stream_right_join_-_PROTOBUF_-_PROTOBUF_NOSR/7.3.0_1651071291496/topology new file mode 100644 index 000000000000..b2217f93221b --- /dev/null +++ b/ksqldb-functional-tests/src/test/resources/historical_plans/joins_-_stream_stream_right_join_-_PROTOBUF_-_PROTOBUF_NOSR/7.3.0_1651071291496/topology @@ -0,0 +1,39 @@ +Topologies: + Sub-topology: 0 + Source: KSTREAM-SOURCE-0000000000 (topics: [left_topic]) + --> KSTREAM-TRANSFORMVALUES-0000000001 + Source: KSTREAM-SOURCE-0000000003 (topics: [right_topic]) + --> KSTREAM-TRANSFORMVALUES-0000000004 + Processor: KSTREAM-TRANSFORMVALUES-0000000001 (stores: []) + --> PrependAliasLeft + <-- KSTREAM-SOURCE-0000000000 + Processor: KSTREAM-TRANSFORMVALUES-0000000004 (stores: []) + --> PrependAliasRight + <-- KSTREAM-SOURCE-0000000003 + Processor: PrependAliasLeft (stores: []) + --> Join-other-windowed + <-- KSTREAM-TRANSFORMVALUES-0000000001 + Processor: PrependAliasRight (stores: []) + --> Join-this-windowed + <-- KSTREAM-TRANSFORMVALUES-0000000004 + Processor: Join-other-windowed (stores: [KSTREAM-OUTEROTHER-0000000009-store]) + --> Join-outer-other-join + <-- PrependAliasLeft + Processor: Join-this-windowed (stores: [KSTREAM-JOINTHIS-0000000008-store]) + --> Join-this-join + <-- PrependAliasRight + Processor: Join-outer-other-join (stores: [KSTREAM-JOINTHIS-0000000008-store]) + --> Join-merge + <-- Join-other-windowed + Processor: Join-this-join (stores: [KSTREAM-OUTEROTHER-0000000009-store]) + --> Join-merge + <-- Join-this-windowed + Processor: Join-merge (stores: []) + --> Project + <-- Join-this-join, Join-outer-other-join + Processor: Project (stores: []) + --> KSTREAM-SINK-0000000012 + <-- Join-merge + Sink: KSTREAM-SINK-0000000012 (topic: OUTPUT) + <-- Project + diff --git a/ksqldb-functional-tests/src/test/resources/historical_plans/joins_-_stream_stream_right_join_-_rekey_-_PROTOBUF_NOSR/7.3.0_1651071300474/plan.json b/ksqldb-functional-tests/src/test/resources/historical_plans/joins_-_stream_stream_right_join_-_rekey_-_PROTOBUF_NOSR/7.3.0_1651071300474/plan.json new file mode 100644 index 000000000000..69f3be73d327 --- /dev/null +++ b/ksqldb-functional-tests/src/test/resources/historical_plans/joins_-_stream_stream_right_join_-_rekey_-_PROTOBUF_NOSR/7.3.0_1651071300474/plan.json @@ -0,0 +1,310 @@ +{ + "plan" : [ { + "@type" : "ksqlPlanV1", + "statementText" : "CREATE STREAM TEST (K STRING KEY, ID BIGINT, NAME STRING, VALUE BIGINT) WITH (KAFKA_TOPIC='left_topic', KEY_FORMAT='KAFKA', VALUE_FORMAT='PROTOBUF_NOSR');", + "ddlCommand" : { + "@type" : "createStreamV1", + "sourceName" : "TEST", + "schema" : "`K` STRING KEY, `ID` BIGINT, `NAME` STRING, `VALUE` BIGINT", + "timestampColumn" : null, + "topicName" : "left_topic", + "formats" : { + "keyFormat" : { + "format" : "KAFKA", + "properties" : { } + }, + "valueFormat" : { + "format" : "PROTOBUF_NOSR", + "properties" : { } + } + }, + "windowInfo" : null, + "orReplace" : false, + "isSource" : false + }, + "queryPlan" : null + }, { + "@type" : "ksqlPlanV1", + "statementText" : "CREATE STREAM TEST_STREAM (K STRING KEY, ID BIGINT, F1 STRING, F2 BIGINT) WITH (KAFKA_TOPIC='right_topic', KEY_FORMAT='KAFKA', VALUE_FORMAT='PROTOBUF_NOSR');", + "ddlCommand" : { + "@type" : "createStreamV1", + "sourceName" : "TEST_STREAM", + "schema" : "`K` STRING KEY, `ID` BIGINT, `F1` STRING, `F2` BIGINT", + "timestampColumn" : null, + "topicName" : "right_topic", + "formats" : { + "keyFormat" : { + "format" : "KAFKA", + "properties" : { } + }, + "valueFormat" : { + "format" : "PROTOBUF_NOSR", + "properties" : { } + } + }, + "windowInfo" : null, + "orReplace" : false, + "isSource" : false + }, + "queryPlan" : null + }, { + "@type" : "ksqlPlanV1", + "statementText" : "CREATE STREAM OUTPUT AS SELECT\n T.ID T_ID,\n T.NAME NAME,\n T.VALUE VALUE,\n TT.F1 F1,\n TT.F2 F2\nFROM TEST T\nRIGHT OUTER JOIN TEST_STREAM TT WITHIN 11 SECONDS ON ((T.ID = TT.ID))\nEMIT CHANGES", + "ddlCommand" : { + "@type" : "createStreamV1", + "sourceName" : "OUTPUT", + "schema" : "`T_ID` BIGINT KEY, `NAME` STRING, `VALUE` BIGINT, `F1` STRING, `F2` BIGINT", + "timestampColumn" : null, + "topicName" : "OUTPUT", + "formats" : { + "keyFormat" : { + "format" : "KAFKA", + "properties" : { } + }, + "valueFormat" : { + "format" : "PROTOBUF_NOSR", + "properties" : { } + } + }, + "windowInfo" : null, + "orReplace" : false, + "isSource" : false + }, + "queryPlan" : { + "sources" : [ "TEST", "TEST_STREAM" ], + "sink" : "OUTPUT", + "physicalPlan" : { + "@type" : "streamSinkV1", + "properties" : { + "queryContext" : "OUTPUT" + }, + "source" : { + "@type" : "streamSelectV1", + "properties" : { + "queryContext" : "Project" + }, + "source" : { + "@type" : "streamStreamJoinV1", + "properties" : { + "queryContext" : "Join" + }, + "joinType" : "RIGHT", + "leftInternalFormats" : { + "keyFormat" : { + "format" : "KAFKA", + "properties" : { } + }, + "valueFormat" : { + "format" : "PROTOBUF_NOSR", + "properties" : { } + } + }, + "rightInternalFormats" : { + "keyFormat" : { + "format" : "KAFKA", + "properties" : { } + }, + "valueFormat" : { + "format" : "PROTOBUF_NOSR", + "properties" : { } + } + }, + "leftSource" : { + "@type" : "streamSelectV1", + "properties" : { + "queryContext" : "PrependAliasLeft" + }, + "source" : { + "@type" : "streamSelectKeyV2", + "properties" : { + "queryContext" : "LeftSourceKeyed" + }, + "source" : { + "@type" : "streamSourceV1", + "properties" : { + "queryContext" : "KafkaTopic_Left/Source" + }, + "topicName" : "left_topic", + "formats" : { + "keyFormat" : { + "format" : "KAFKA", + "properties" : { } + }, + "valueFormat" : { + "format" : "PROTOBUF_NOSR", + "properties" : { } + } + }, + "timestampColumn" : null, + "sourceSchema" : "`K` STRING KEY, `ID` BIGINT, `NAME` STRING, `VALUE` BIGINT", + "pseudoColumnVersion" : 1 + }, + "keyExpression" : [ "ID" ] + }, + "keyColumnNames" : [ "T_ID" ], + "selectedKeys" : null, + "selectExpressions" : [ "ID AS T_ID", "NAME AS T_NAME", "VALUE AS T_VALUE", "ROWTIME AS T_ROWTIME", "ROWPARTITION AS T_ROWPARTITION", "ROWOFFSET AS T_ROWOFFSET", "K AS T_K" ] + }, + "rightSource" : { + "@type" : "streamSelectV1", + "properties" : { + "queryContext" : "PrependAliasRight" + }, + "source" : { + "@type" : "streamSelectKeyV2", + "properties" : { + "queryContext" : "RightSourceKeyed" + }, + "source" : { + "@type" : "streamSourceV1", + "properties" : { + "queryContext" : "KafkaTopic_Right/Source" + }, + "topicName" : "right_topic", + "formats" : { + "keyFormat" : { + "format" : "KAFKA", + "properties" : { } + }, + "valueFormat" : { + "format" : "PROTOBUF_NOSR", + "properties" : { } + } + }, + "timestampColumn" : null, + "sourceSchema" : "`K` STRING KEY, `ID` BIGINT, `F1` STRING, `F2` BIGINT", + "pseudoColumnVersion" : 1 + }, + "keyExpression" : [ "ID" ] + }, + "keyColumnNames" : [ "TT_ID" ], + "selectedKeys" : null, + "selectExpressions" : [ "ID AS TT_ID", "F1 AS TT_F1", "F2 AS TT_F2", "ROWTIME AS TT_ROWTIME", "ROWPARTITION AS TT_ROWPARTITION", "ROWOFFSET AS TT_ROWOFFSET", "K AS TT_K" ] + }, + "beforeMillis" : 11.000000000, + "afterMillis" : 11.000000000, + "graceMillis" : null, + "keyColName" : "T_ID" + }, + "keyColumnNames" : [ "T_ID" ], + "selectedKeys" : null, + "selectExpressions" : [ "T_NAME AS NAME", "T_VALUE AS VALUE", "TT_F1 AS F1", "TT_F2 AS F2" ] + }, + "formats" : { + "keyFormat" : { + "format" : "KAFKA", + "properties" : { } + }, + "valueFormat" : { + "format" : "PROTOBUF_NOSR", + "properties" : { } + } + }, + "topicName" : "OUTPUT", + "timestampColumn" : null + }, + "queryId" : "CSAS_OUTPUT_0", + "runtimeId" : null + } + } ], + "configs" : { + "ksql.extension.dir" : "ext", + "ksql.streams.cache.max.bytes.buffering" : "0", + "metric.reporters" : "", + "ksql.query.status.running.threshold.seconds" : "300", + "ksql.connect.basic.auth.credentials.reload" : "false", + "ksql.output.topic.name.prefix" : "", + "ksql.query.pull.stream.enabled" : "true", + "ksql.query.push.v2.interpreter.enabled" : "true", + "ksql.queryanonymizer.logs_enabled" : "true", + "ksql.variable.substitution.enable" : "true", + "ksql.streams.shutdown.timeout.ms" : "300000", + "ksql.query.pull.max.allowed.offset.lag" : "9223372036854775807", + "ksql.query.pull.max.qps" : "2147483647", + "ksql.access.validator.enable" : "auto", + "ksql.streams.bootstrap.servers" : "localhost:0", + "ksql.query.pull.metrics.enabled" : "true", + "ksql.metrics.extension" : null, + "ksql.query.push.v2.alos.enabled" : "true", + "ksql.query.push.v2.max.hourly.bandwidth.megabytes" : "2147483647", + "ksql.query.pull.range.scan.enabled" : "true", + "ksql.transient.query.cleanup.service.initial.delay.seconds" : "600", + "ksql.hidden.topics" : "_confluent.*,__confluent.*,_schemas,__consumer_offsets,__transaction_state,connect-configs,connect-offsets,connect-status,connect-statuses", + "ksql.lambdas.enabled" : "true", + "ksql.source.table.materialization.enabled" : "true", + "ksql.query.pull.max.hourly.bandwidth.megabytes" : "2147483647", + "ksql.sink.window.change.log.additional.retention" : "1000000", + "ksql.query.persistent.active.limit" : "2147483647", + "ksql.persistence.wrap.single.values" : null, + "ksql.query.transient.max.bytes.buffering.total" : "-1", + "ksql.connect.basic.auth.credentials.source" : "NONE", + "ksql.schema.registry.url" : "schema_registry.url:0", + "ksql.properties.overrides.denylist" : "", + "ksql.service.id" : "some.ksql.service.id", + "ksql.query.push.v2.max.catchup.consumers" : "5", + "ksql.query.push.v2.enabled" : "false", + "ksql.transient.query.cleanup.service.enable" : "true", + "ksql.query.push.v2.metrics.enabled" : "true", + "ksql.rowpartition.rowoffset.enabled" : "true", + "ksql.streams.commit.interval.ms" : "2000", + "ksql.query.pull.table.scan.enabled" : "true", + "ksql.streams.auto.commit.interval.ms" : "0", + "ksql.streams.topology.optimization" : "all", + "ksql.endpoint.migrate.query" : "true", + "ksql.query.push.v2.registry.installed" : "false", + "ksql.streams.num.stream.threads" : "4", + "ksql.metrics.tags.custom" : "", + "ksql.query.push.v2.catchup.consumer.msg.window" : "50", + "ksql.runtime.feature.shared.enabled" : "false", + "ksql.udf.collect.metrics" : "false", + "ksql.new.query.planner.enabled" : "false", + "ksql.connect.request.headers.plugin" : null, + "ksql.security.extension.class" : null, + "ksql.transient.prefix" : "transient_", + "ksql.headers.columns.enabled" : "true", + "ksql.streams.default.deserialization.exception.handler" : "io.confluent.ksql.errors.LogMetricAndContinueExceptionHandler", + "ksql.connect.request.timeout.ms" : "5000", + "ksql.query.pull.enable.standby.reads" : "false", + "ksql.persistence.default.format.key" : "KAFKA", + "ksql.query.persistent.max.bytes.buffering.total" : "-1", + "ksql.query.error.max.queue.size" : "10", + "ksql.query.cleanup.shutdown.timeout.ms" : "30000", + "ksql.internal.topic.min.insync.replicas" : "1", + "ksql.internal.topic.replicas" : "1", + "ksql.insert.into.values.enabled" : "true", + "ksql.queryanonymizer.cluster_namespace" : null, + "ksql.create.or.replace.enabled" : "true", + "ksql.shared.runtimes.count" : "2", + "ksql.cast.strings.preserve.nulls" : "true", + "ksql.authorization.cache.max.entries" : "10000", + "ksql.pull.queries.enable" : "true", + "ksql.transient.query.cleanup.service.period.seconds" : "600", + "ksql.suppress.enabled" : "false", + "ksql.readonly.topics" : "_confluent.*,__confluent.*,_schemas,__consumer_offsets,__transaction_state,connect-configs,connect-offsets,connect-status,connect-statuses", + "ksql.connect.basic.auth.credentials.file" : "", + "ksql.authorization.cache.expiry.time.secs" : "30", + "ksql.query.retry.backoff.initial.ms" : "15000", + "ksql.query.pull.max.concurrent.requests" : "2147483647", + "ksql.streams.auto.offset.reset" : "earliest", + "ksql.connect.url" : "http://localhost:8083", + "ksql.query.push.v2.new.latest.delay.ms" : "5000", + "ksql.query.push.v2.latest.reset.age.ms" : "30000", + "ksql.streams.default.production.exception.handler" : "io.confluent.ksql.errors.ProductionExceptionHandlerUtil$LogAndFailProductionExceptionHandler", + "ksql.query.pull.interpreter.enabled" : "true", + "ksql.query.pull.limit.clause.enabled" : "true", + "ksql.query.pull.router.thread.pool.size" : "50", + "ksql.query.push.v2.continuation.tokens.enabled" : "false", + "ksql.query.retry.backoff.max.ms" : "900000", + "ksql.timestamp.throw.on.invalid" : "false", + "ksql.persistence.default.format.value" : null, + "ksql.udfs.enabled" : "true", + "ksql.udf.enable.security.manager" : "true", + "ksql.connect.worker.config" : "", + "ksql.nested.error.set.null" : "true", + "ksql.query.pull.thread.pool.size" : "50", + "ksql.persistent.prefix" : "query_", + "ksql.metastore.backup.location" : "", + "ksql.error.classifier.regex" : "", + "ksql.suppress.buffer.size.bytes" : "-1" + } +} \ No newline at end of file diff --git a/ksqldb-functional-tests/src/test/resources/historical_plans/joins_-_stream_stream_right_join_-_rekey_-_PROTOBUF_NOSR/7.3.0_1651071300474/spec.json b/ksqldb-functional-tests/src/test/resources/historical_plans/joins_-_stream_stream_right_join_-_rekey_-_PROTOBUF_NOSR/7.3.0_1651071300474/spec.json new file mode 100644 index 000000000000..384232378303 --- /dev/null +++ b/ksqldb-functional-tests/src/test/resources/historical_plans/joins_-_stream_stream_right_join_-_rekey_-_PROTOBUF_NOSR/7.3.0_1651071300474/spec.json @@ -0,0 +1,281 @@ +{ + "version" : "7.3.0", + "timestamp" : 1651071300474, + "path" : "query-validation-tests/joins.json", + "schemas" : { + "CSAS_OUTPUT_0.KafkaTopic_Right.Source" : { + "schema" : "`K` STRING KEY, `ID` BIGINT, `F1` STRING, `F2` BIGINT", + "keyFormat" : { + "format" : "KAFKA" + }, + "valueFormat" : { + "format" : "PROTOBUF_NOSR" + } + }, + "CSAS_OUTPUT_0.Join.Left" : { + "schema" : "`TT_ID` BIGINT KEY, `TT_ID` BIGINT, `TT_F1` STRING, `TT_F2` BIGINT, `TT_ROWTIME` BIGINT, `TT_ROWPARTITION` INTEGER, `TT_ROWOFFSET` BIGINT, `TT_K` STRING", + "valueFormat" : { + "format" : "PROTOBUF_NOSR" + } + }, + "CSAS_OUTPUT_0.OUTPUT" : { + "schema" : "`T_ID` BIGINT KEY, `NAME` STRING, `VALUE` BIGINT, `F1` STRING, `F2` BIGINT", + "keyFormat" : { + "format" : "KAFKA" + }, + "valueFormat" : { + "format" : "PROTOBUF_NOSR" + } + }, + "CSAS_OUTPUT_0.KafkaTopic_Left.Source" : { + "schema" : "`K` STRING KEY, `ID` BIGINT, `NAME` STRING, `VALUE` BIGINT", + "keyFormat" : { + "format" : "KAFKA" + }, + "valueFormat" : { + "format" : "PROTOBUF_NOSR" + } + }, + "CSAS_OUTPUT_0.Join" : { + "schema" : "`TT_ID` BIGINT KEY, `TT_ID` BIGINT, `TT_F1` STRING, `TT_F2` BIGINT, `TT_ROWTIME` BIGINT, `TT_ROWPARTITION` INTEGER, `TT_ROWOFFSET` BIGINT, `TT_K` STRING", + "keyFormat" : { + "format" : "KAFKA" + } + }, + "CSAS_OUTPUT_0.Join.Right" : { + "schema" : "`T_ID` BIGINT KEY, `T_ID` BIGINT, `T_NAME` STRING, `T_VALUE` BIGINT, `T_ROWTIME` BIGINT, `T_ROWPARTITION` INTEGER, `T_ROWOFFSET` BIGINT, `T_K` STRING", + "valueFormat" : { + "format" : "PROTOBUF_NOSR" + } + } + }, + "testCase" : { + "name" : "stream stream right join - rekey - PROTOBUF_NOSR", + "inputs" : [ { + "topic" : "left_topic", + "key" : null, + "value" : { + "ID" : 0, + "NAME" : "zero", + "VALUE" : 0 + }, + "timestamp" : 0 + }, { + "topic" : "right_topic", + "key" : null, + "value" : { + "ID" : 0, + "F1" : "blah", + "F2" : 50 + }, + "timestamp" : 10000 + }, { + "topic" : "left_topic", + "key" : null, + "value" : { + "ID" : 10, + "NAME" : "100", + "VALUE" : 5 + }, + "timestamp" : 11000 + }, { + "topic" : "left_topic", + "key" : null, + "value" : { + "ID" : 0, + "NAME" : "foo", + "VALUE" : 100 + }, + "timestamp" : 13000 + }, { + "topic" : "right_topic", + "key" : null, + "value" : { + "ID" : 0, + "F1" : "a", + "F2" : 10 + }, + "timestamp" : 15000 + }, { + "topic" : "right_topic", + "key" : null, + "value" : { + "ID" : 100, + "F1" : "newblah", + "F2" : 150 + }, + "timestamp" : 16000 + }, { + "topic" : "left_topic", + "key" : null, + "value" : { + "ID" : 90, + "NAME" : "ninety", + "VALUE" : 90 + }, + "timestamp" : 17000 + }, { + "topic" : "left_topic", + "key" : null, + "value" : { + "ID" : 0, + "NAME" : "bar", + "VALUE" : 99 + }, + "timestamp" : 30000 + } ], + "outputs" : [ { + "topic" : "OUTPUT", + "key" : 0, + "value" : { + "NAME" : "zero", + "VALUE" : 0, + "F1" : "blah", + "F2" : 50 + }, + "timestamp" : 10000 + }, { + "topic" : "OUTPUT", + "key" : 0, + "value" : { + "NAME" : "foo", + "VALUE" : 100, + "F1" : "blah", + "F2" : 50 + }, + "timestamp" : 13000 + }, { + "topic" : "OUTPUT", + "key" : 0, + "value" : { + "NAME" : "foo", + "VALUE" : 100, + "F1" : "a", + "F2" : 10 + }, + "timestamp" : 15000 + }, { + "topic" : "OUTPUT", + "key" : 100, + "value" : { + "NAME" : "", + "VALUE" : 0, + "F1" : "newblah", + "F2" : 150 + }, + "timestamp" : 16000 + } ], + "topics" : [ { + "name" : "right_topic", + "replicas" : 1, + "numPartitions" : 4 + }, { + "name" : "OUTPUT", + "replicas" : 1, + "numPartitions" : 4 + }, { + "name" : "left_topic", + "replicas" : 1, + "numPartitions" : 4 + } ], + "statements" : [ "CREATE STREAM TEST (K STRING KEY, ID bigint, NAME varchar, VALUE bigint) WITH (kafka_topic='left_topic', value_format='PROTOBUF_NOSR');", "CREATE STREAM TEST_STREAM (K STRING KEY, ID bigint, F1 varchar, F2 bigint) WITH (kafka_topic='right_topic', value_format='PROTOBUF_NOSR');", "CREATE STREAM OUTPUT as SELECT t.id, name, value, f1, f2 FROM test t RIGHT JOIN test_stream tt WITHIN 11 seconds ON t.id = tt.id;" ], + "post" : { + "sources" : [ { + "name" : "OUTPUT", + "type" : "STREAM", + "schema" : "`T_ID` BIGINT KEY, `NAME` STRING, `VALUE` BIGINT, `F1` STRING, `F2` BIGINT", + "keyFormat" : { + "format" : "KAFKA" + }, + "valueFormat" : "PROTOBUF_NOSR", + "keyFeatures" : [ ], + "valueFeatures" : [ ], + "isSource" : false + }, { + "name" : "TEST", + "type" : "STREAM", + "schema" : "`K` STRING KEY, `ID` BIGINT, `NAME` STRING, `VALUE` BIGINT", + "keyFormat" : { + "format" : "KAFKA" + }, + "valueFormat" : "PROTOBUF_NOSR", + "keyFeatures" : [ ], + "valueFeatures" : [ ], + "isSource" : false + }, { + "name" : "TEST_STREAM", + "type" : "STREAM", + "schema" : "`K` STRING KEY, `ID` BIGINT, `F1` STRING, `F2` BIGINT", + "keyFormat" : { + "format" : "KAFKA" + }, + "valueFormat" : "PROTOBUF_NOSR", + "keyFeatures" : [ ], + "valueFeatures" : [ ], + "isSource" : false + } ], + "topics" : { + "topics" : [ { + "name" : "_confluent-ksql-some.ksql.service.idquery_CSAS_OUTPUT_0-Join-left-repartition", + "keyFormat" : { + "format" : "KAFKA" + }, + "valueFormat" : { + "format" : "PROTOBUF_NOSR" + } + }, { + "name" : "_confluent-ksql-some.ksql.service.idquery_CSAS_OUTPUT_0-Join-right-repartition", + "keyFormat" : { + "format" : "KAFKA" + }, + "valueFormat" : { + "format" : "PROTOBUF_NOSR" + } + }, { + "name" : "_confluent-ksql-some.ksql.service.idquery_CSAS_OUTPUT_0-KSTREAM-JOINTHIS-0000000016-store-changelog", + "keyFormat" : { + "format" : "KAFKA" + }, + "valueFormat" : { + "format" : "PROTOBUF_NOSR" + } + }, { + "name" : "left_topic", + "keyFormat" : { + "format" : "KAFKA" + }, + "valueFormat" : { + "format" : "PROTOBUF_NOSR" + }, + "partitions" : 4 + }, { + "name" : "_confluent-ksql-some.ksql.service.idquery_CSAS_OUTPUT_0-KSTREAM-OUTEROTHER-0000000017-store-changelog", + "keyFormat" : { + "format" : "KAFKA" + }, + "valueFormat" : { + "format" : "PROTOBUF_NOSR" + } + }, { + "name" : "OUTPUT", + "keyFormat" : { + "format" : "KAFKA" + }, + "valueFormat" : { + "format" : "PROTOBUF_NOSR" + }, + "partitions" : 4 + }, { + "name" : "right_topic", + "keyFormat" : { + "format" : "KAFKA" + }, + "valueFormat" : { + "format" : "PROTOBUF_NOSR" + }, + "partitions" : 4 + } ] + } + } + } +} \ No newline at end of file diff --git a/ksqldb-functional-tests/src/test/resources/historical_plans/joins_-_stream_stream_right_join_-_rekey_-_PROTOBUF_NOSR/7.3.0_1651071300474/topology b/ksqldb-functional-tests/src/test/resources/historical_plans/joins_-_stream_stream_right_join_-_rekey_-_PROTOBUF_NOSR/7.3.0_1651071300474/topology new file mode 100644 index 000000000000..6222d30b3022 --- /dev/null +++ b/ksqldb-functional-tests/src/test/resources/historical_plans/joins_-_stream_stream_right_join_-_rekey_-_PROTOBUF_NOSR/7.3.0_1651071300474/topology @@ -0,0 +1,63 @@ +Topologies: + Sub-topology: 0 + Source: KSTREAM-SOURCE-0000000000 (topics: [left_topic]) + --> KSTREAM-TRANSFORMVALUES-0000000001 + Processor: KSTREAM-TRANSFORMVALUES-0000000001 (stores: []) + --> LeftSourceKeyed-SelectKey + <-- KSTREAM-SOURCE-0000000000 + Processor: LeftSourceKeyed-SelectKey (stores: []) + --> PrependAliasLeft + <-- KSTREAM-TRANSFORMVALUES-0000000001 + Processor: PrependAliasLeft (stores: []) + --> Join-right-repartition-filter + <-- LeftSourceKeyed-SelectKey + Processor: Join-right-repartition-filter (stores: []) + --> Join-right-repartition-sink + <-- PrependAliasLeft + Sink: Join-right-repartition-sink (topic: Join-right-repartition) + <-- Join-right-repartition-filter + + Sub-topology: 1 + Source: KSTREAM-SOURCE-0000000004 (topics: [right_topic]) + --> KSTREAM-TRANSFORMVALUES-0000000005 + Processor: KSTREAM-TRANSFORMVALUES-0000000005 (stores: []) + --> RightSourceKeyed-SelectKey + <-- KSTREAM-SOURCE-0000000004 + Processor: RightSourceKeyed-SelectKey (stores: []) + --> PrependAliasRight + <-- KSTREAM-TRANSFORMVALUES-0000000005 + Processor: PrependAliasRight (stores: []) + --> Join-left-repartition-filter + <-- RightSourceKeyed-SelectKey + Processor: Join-left-repartition-filter (stores: []) + --> Join-left-repartition-sink + <-- PrependAliasRight + Sink: Join-left-repartition-sink (topic: Join-left-repartition) + <-- Join-left-repartition-filter + + Sub-topology: 2 + Source: Join-left-repartition-source (topics: [Join-left-repartition]) + --> Join-this-windowed + Source: Join-right-repartition-source (topics: [Join-right-repartition]) + --> Join-other-windowed + Processor: Join-other-windowed (stores: [KSTREAM-OUTEROTHER-0000000017-store]) + --> Join-outer-other-join + <-- Join-right-repartition-source + Processor: Join-this-windowed (stores: [KSTREAM-JOINTHIS-0000000016-store]) + --> Join-this-join + <-- Join-left-repartition-source + Processor: Join-outer-other-join (stores: [KSTREAM-JOINTHIS-0000000016-store]) + --> Join-merge + <-- Join-other-windowed + Processor: Join-this-join (stores: [KSTREAM-OUTEROTHER-0000000017-store]) + --> Join-merge + <-- Join-this-windowed + Processor: Join-merge (stores: []) + --> Project + <-- Join-this-join, Join-outer-other-join + Processor: Project (stores: []) + --> KSTREAM-SINK-0000000020 + <-- Join-merge + Sink: KSTREAM-SINK-0000000020 (topic: OUTPUT) + <-- Project + diff --git a/ksqldb-functional-tests/src/test/resources/historical_plans/joins_-_stream_stream_right_join_with_key_in_projection_-_rekey_-_PROTOBUF_NOSR/7.3.0_1651071295926/plan.json b/ksqldb-functional-tests/src/test/resources/historical_plans/joins_-_stream_stream_right_join_with_key_in_projection_-_rekey_-_PROTOBUF_NOSR/7.3.0_1651071295926/plan.json new file mode 100644 index 000000000000..af5728128059 --- /dev/null +++ b/ksqldb-functional-tests/src/test/resources/historical_plans/joins_-_stream_stream_right_join_with_key_in_projection_-_rekey_-_PROTOBUF_NOSR/7.3.0_1651071295926/plan.json @@ -0,0 +1,310 @@ +{ + "plan" : [ { + "@type" : "ksqlPlanV1", + "statementText" : "CREATE STREAM TEST (K STRING KEY, ID BIGINT, NAME STRING, VALUE BIGINT) WITH (KAFKA_TOPIC='left_topic', KEY_FORMAT='KAFKA', VALUE_FORMAT='PROTOBUF_NOSR');", + "ddlCommand" : { + "@type" : "createStreamV1", + "sourceName" : "TEST", + "schema" : "`K` STRING KEY, `ID` BIGINT, `NAME` STRING, `VALUE` BIGINT", + "timestampColumn" : null, + "topicName" : "left_topic", + "formats" : { + "keyFormat" : { + "format" : "KAFKA", + "properties" : { } + }, + "valueFormat" : { + "format" : "PROTOBUF_NOSR", + "properties" : { } + } + }, + "windowInfo" : null, + "orReplace" : false, + "isSource" : false + }, + "queryPlan" : null + }, { + "@type" : "ksqlPlanV1", + "statementText" : "CREATE STREAM TEST_STREAM (K STRING KEY, ID BIGINT, F1 STRING, F2 BIGINT) WITH (KAFKA_TOPIC='right_topic', KEY_FORMAT='KAFKA', VALUE_FORMAT='PROTOBUF_NOSR');", + "ddlCommand" : { + "@type" : "createStreamV1", + "sourceName" : "TEST_STREAM", + "schema" : "`K` STRING KEY, `ID` BIGINT, `F1` STRING, `F2` BIGINT", + "timestampColumn" : null, + "topicName" : "right_topic", + "formats" : { + "keyFormat" : { + "format" : "KAFKA", + "properties" : { } + }, + "valueFormat" : { + "format" : "PROTOBUF_NOSR", + "properties" : { } + } + }, + "windowInfo" : null, + "orReplace" : false, + "isSource" : false + }, + "queryPlan" : null + }, { + "@type" : "ksqlPlanV1", + "statementText" : "CREATE STREAM OUTPUT AS SELECT\n T.ID T_ID,\n T.K T_K,\n T.NAME NAME,\n T.VALUE VALUE,\n TT.F1 F1,\n TT.F2 F2\nFROM TEST T\nRIGHT OUTER JOIN TEST_STREAM TT WITHIN 11 SECONDS ON ((T.ID = TT.ID))\nEMIT CHANGES", + "ddlCommand" : { + "@type" : "createStreamV1", + "sourceName" : "OUTPUT", + "schema" : "`T_ID` BIGINT KEY, `T_K` STRING, `NAME` STRING, `VALUE` BIGINT, `F1` STRING, `F2` BIGINT", + "timestampColumn" : null, + "topicName" : "OUTPUT", + "formats" : { + "keyFormat" : { + "format" : "KAFKA", + "properties" : { } + }, + "valueFormat" : { + "format" : "PROTOBUF_NOSR", + "properties" : { } + } + }, + "windowInfo" : null, + "orReplace" : false, + "isSource" : false + }, + "queryPlan" : { + "sources" : [ "TEST", "TEST_STREAM" ], + "sink" : "OUTPUT", + "physicalPlan" : { + "@type" : "streamSinkV1", + "properties" : { + "queryContext" : "OUTPUT" + }, + "source" : { + "@type" : "streamSelectV1", + "properties" : { + "queryContext" : "Project" + }, + "source" : { + "@type" : "streamStreamJoinV1", + "properties" : { + "queryContext" : "Join" + }, + "joinType" : "RIGHT", + "leftInternalFormats" : { + "keyFormat" : { + "format" : "KAFKA", + "properties" : { } + }, + "valueFormat" : { + "format" : "PROTOBUF_NOSR", + "properties" : { } + } + }, + "rightInternalFormats" : { + "keyFormat" : { + "format" : "KAFKA", + "properties" : { } + }, + "valueFormat" : { + "format" : "PROTOBUF_NOSR", + "properties" : { } + } + }, + "leftSource" : { + "@type" : "streamSelectV1", + "properties" : { + "queryContext" : "PrependAliasLeft" + }, + "source" : { + "@type" : "streamSelectKeyV2", + "properties" : { + "queryContext" : "LeftSourceKeyed" + }, + "source" : { + "@type" : "streamSourceV1", + "properties" : { + "queryContext" : "KafkaTopic_Left/Source" + }, + "topicName" : "left_topic", + "formats" : { + "keyFormat" : { + "format" : "KAFKA", + "properties" : { } + }, + "valueFormat" : { + "format" : "PROTOBUF_NOSR", + "properties" : { } + } + }, + "timestampColumn" : null, + "sourceSchema" : "`K` STRING KEY, `ID` BIGINT, `NAME` STRING, `VALUE` BIGINT", + "pseudoColumnVersion" : 1 + }, + "keyExpression" : [ "ID" ] + }, + "keyColumnNames" : [ "T_ID" ], + "selectedKeys" : null, + "selectExpressions" : [ "ID AS T_ID", "NAME AS T_NAME", "VALUE AS T_VALUE", "ROWTIME AS T_ROWTIME", "ROWPARTITION AS T_ROWPARTITION", "ROWOFFSET AS T_ROWOFFSET", "K AS T_K" ] + }, + "rightSource" : { + "@type" : "streamSelectV1", + "properties" : { + "queryContext" : "PrependAliasRight" + }, + "source" : { + "@type" : "streamSelectKeyV2", + "properties" : { + "queryContext" : "RightSourceKeyed" + }, + "source" : { + "@type" : "streamSourceV1", + "properties" : { + "queryContext" : "KafkaTopic_Right/Source" + }, + "topicName" : "right_topic", + "formats" : { + "keyFormat" : { + "format" : "KAFKA", + "properties" : { } + }, + "valueFormat" : { + "format" : "PROTOBUF_NOSR", + "properties" : { } + } + }, + "timestampColumn" : null, + "sourceSchema" : "`K` STRING KEY, `ID` BIGINT, `F1` STRING, `F2` BIGINT", + "pseudoColumnVersion" : 1 + }, + "keyExpression" : [ "ID" ] + }, + "keyColumnNames" : [ "TT_ID" ], + "selectedKeys" : null, + "selectExpressions" : [ "ID AS TT_ID", "F1 AS TT_F1", "F2 AS TT_F2", "ROWTIME AS TT_ROWTIME", "ROWPARTITION AS TT_ROWPARTITION", "ROWOFFSET AS TT_ROWOFFSET", "K AS TT_K" ] + }, + "beforeMillis" : 11.000000000, + "afterMillis" : 11.000000000, + "graceMillis" : null, + "keyColName" : "T_ID" + }, + "keyColumnNames" : [ "T_ID" ], + "selectedKeys" : null, + "selectExpressions" : [ "T_K AS T_K", "T_NAME AS NAME", "T_VALUE AS VALUE", "TT_F1 AS F1", "TT_F2 AS F2" ] + }, + "formats" : { + "keyFormat" : { + "format" : "KAFKA", + "properties" : { } + }, + "valueFormat" : { + "format" : "PROTOBUF_NOSR", + "properties" : { } + } + }, + "topicName" : "OUTPUT", + "timestampColumn" : null + }, + "queryId" : "CSAS_OUTPUT_0", + "runtimeId" : null + } + } ], + "configs" : { + "ksql.extension.dir" : "ext", + "ksql.streams.cache.max.bytes.buffering" : "0", + "metric.reporters" : "", + "ksql.query.status.running.threshold.seconds" : "300", + "ksql.connect.basic.auth.credentials.reload" : "false", + "ksql.output.topic.name.prefix" : "", + "ksql.query.pull.stream.enabled" : "true", + "ksql.query.push.v2.interpreter.enabled" : "true", + "ksql.queryanonymizer.logs_enabled" : "true", + "ksql.variable.substitution.enable" : "true", + "ksql.streams.shutdown.timeout.ms" : "300000", + "ksql.query.pull.max.allowed.offset.lag" : "9223372036854775807", + "ksql.query.pull.max.qps" : "2147483647", + "ksql.access.validator.enable" : "auto", + "ksql.streams.bootstrap.servers" : "localhost:0", + "ksql.query.pull.metrics.enabled" : "true", + "ksql.metrics.extension" : null, + "ksql.query.push.v2.alos.enabled" : "true", + "ksql.query.push.v2.max.hourly.bandwidth.megabytes" : "2147483647", + "ksql.query.pull.range.scan.enabled" : "true", + "ksql.transient.query.cleanup.service.initial.delay.seconds" : "600", + "ksql.hidden.topics" : "_confluent.*,__confluent.*,_schemas,__consumer_offsets,__transaction_state,connect-configs,connect-offsets,connect-status,connect-statuses", + "ksql.lambdas.enabled" : "true", + "ksql.source.table.materialization.enabled" : "true", + "ksql.query.pull.max.hourly.bandwidth.megabytes" : "2147483647", + "ksql.sink.window.change.log.additional.retention" : "1000000", + "ksql.query.persistent.active.limit" : "2147483647", + "ksql.persistence.wrap.single.values" : null, + "ksql.query.transient.max.bytes.buffering.total" : "-1", + "ksql.connect.basic.auth.credentials.source" : "NONE", + "ksql.schema.registry.url" : "schema_registry.url:0", + "ksql.properties.overrides.denylist" : "", + "ksql.service.id" : "some.ksql.service.id", + "ksql.query.push.v2.max.catchup.consumers" : "5", + "ksql.query.push.v2.enabled" : "false", + "ksql.transient.query.cleanup.service.enable" : "true", + "ksql.query.push.v2.metrics.enabled" : "true", + "ksql.rowpartition.rowoffset.enabled" : "true", + "ksql.streams.commit.interval.ms" : "2000", + "ksql.query.pull.table.scan.enabled" : "true", + "ksql.streams.auto.commit.interval.ms" : "0", + "ksql.streams.topology.optimization" : "all", + "ksql.endpoint.migrate.query" : "true", + "ksql.query.push.v2.registry.installed" : "false", + "ksql.streams.num.stream.threads" : "4", + "ksql.metrics.tags.custom" : "", + "ksql.query.push.v2.catchup.consumer.msg.window" : "50", + "ksql.runtime.feature.shared.enabled" : "false", + "ksql.udf.collect.metrics" : "false", + "ksql.new.query.planner.enabled" : "false", + "ksql.connect.request.headers.plugin" : null, + "ksql.security.extension.class" : null, + "ksql.transient.prefix" : "transient_", + "ksql.headers.columns.enabled" : "true", + "ksql.streams.default.deserialization.exception.handler" : "io.confluent.ksql.errors.LogMetricAndContinueExceptionHandler", + "ksql.connect.request.timeout.ms" : "5000", + "ksql.query.pull.enable.standby.reads" : "false", + "ksql.persistence.default.format.key" : "KAFKA", + "ksql.query.persistent.max.bytes.buffering.total" : "-1", + "ksql.query.error.max.queue.size" : "10", + "ksql.query.cleanup.shutdown.timeout.ms" : "30000", + "ksql.internal.topic.min.insync.replicas" : "1", + "ksql.internal.topic.replicas" : "1", + "ksql.insert.into.values.enabled" : "true", + "ksql.queryanonymizer.cluster_namespace" : null, + "ksql.create.or.replace.enabled" : "true", + "ksql.shared.runtimes.count" : "2", + "ksql.cast.strings.preserve.nulls" : "true", + "ksql.authorization.cache.max.entries" : "10000", + "ksql.pull.queries.enable" : "true", + "ksql.transient.query.cleanup.service.period.seconds" : "600", + "ksql.suppress.enabled" : "false", + "ksql.readonly.topics" : "_confluent.*,__confluent.*,_schemas,__consumer_offsets,__transaction_state,connect-configs,connect-offsets,connect-status,connect-statuses", + "ksql.connect.basic.auth.credentials.file" : "", + "ksql.authorization.cache.expiry.time.secs" : "30", + "ksql.query.retry.backoff.initial.ms" : "15000", + "ksql.query.pull.max.concurrent.requests" : "2147483647", + "ksql.streams.auto.offset.reset" : "earliest", + "ksql.connect.url" : "http://localhost:8083", + "ksql.query.push.v2.new.latest.delay.ms" : "5000", + "ksql.query.push.v2.latest.reset.age.ms" : "30000", + "ksql.streams.default.production.exception.handler" : "io.confluent.ksql.errors.ProductionExceptionHandlerUtil$LogAndFailProductionExceptionHandler", + "ksql.query.pull.interpreter.enabled" : "true", + "ksql.query.pull.limit.clause.enabled" : "true", + "ksql.query.pull.router.thread.pool.size" : "50", + "ksql.query.push.v2.continuation.tokens.enabled" : "false", + "ksql.query.retry.backoff.max.ms" : "900000", + "ksql.timestamp.throw.on.invalid" : "false", + "ksql.persistence.default.format.value" : null, + "ksql.udfs.enabled" : "true", + "ksql.udf.enable.security.manager" : "true", + "ksql.connect.worker.config" : "", + "ksql.nested.error.set.null" : "true", + "ksql.query.pull.thread.pool.size" : "50", + "ksql.persistent.prefix" : "query_", + "ksql.metastore.backup.location" : "", + "ksql.error.classifier.regex" : "", + "ksql.suppress.buffer.size.bytes" : "-1" + } +} \ No newline at end of file diff --git a/ksqldb-functional-tests/src/test/resources/historical_plans/joins_-_stream_stream_right_join_with_key_in_projection_-_rekey_-_PROTOBUF_NOSR/7.3.0_1651071295926/spec.json b/ksqldb-functional-tests/src/test/resources/historical_plans/joins_-_stream_stream_right_join_with_key_in_projection_-_rekey_-_PROTOBUF_NOSR/7.3.0_1651071295926/spec.json new file mode 100644 index 000000000000..32d6d862a735 --- /dev/null +++ b/ksqldb-functional-tests/src/test/resources/historical_plans/joins_-_stream_stream_right_join_with_key_in_projection_-_rekey_-_PROTOBUF_NOSR/7.3.0_1651071295926/spec.json @@ -0,0 +1,285 @@ +{ + "version" : "7.3.0", + "timestamp" : 1651071295926, + "path" : "query-validation-tests/joins.json", + "schemas" : { + "CSAS_OUTPUT_0.KafkaTopic_Right.Source" : { + "schema" : "`K` STRING KEY, `ID` BIGINT, `F1` STRING, `F2` BIGINT", + "keyFormat" : { + "format" : "KAFKA" + }, + "valueFormat" : { + "format" : "PROTOBUF_NOSR" + } + }, + "CSAS_OUTPUT_0.Join.Left" : { + "schema" : "`TT_ID` BIGINT KEY, `TT_ID` BIGINT, `TT_F1` STRING, `TT_F2` BIGINT, `TT_ROWTIME` BIGINT, `TT_ROWPARTITION` INTEGER, `TT_ROWOFFSET` BIGINT, `TT_K` STRING", + "valueFormat" : { + "format" : "PROTOBUF_NOSR" + } + }, + "CSAS_OUTPUT_0.OUTPUT" : { + "schema" : "`T_ID` BIGINT KEY, `T_K` STRING, `NAME` STRING, `VALUE` BIGINT, `F1` STRING, `F2` BIGINT", + "keyFormat" : { + "format" : "KAFKA" + }, + "valueFormat" : { + "format" : "PROTOBUF_NOSR" + } + }, + "CSAS_OUTPUT_0.KafkaTopic_Left.Source" : { + "schema" : "`K` STRING KEY, `ID` BIGINT, `NAME` STRING, `VALUE` BIGINT", + "keyFormat" : { + "format" : "KAFKA" + }, + "valueFormat" : { + "format" : "PROTOBUF_NOSR" + } + }, + "CSAS_OUTPUT_0.Join" : { + "schema" : "`TT_ID` BIGINT KEY, `TT_ID` BIGINT, `TT_F1` STRING, `TT_F2` BIGINT, `TT_ROWTIME` BIGINT, `TT_ROWPARTITION` INTEGER, `TT_ROWOFFSET` BIGINT, `TT_K` STRING", + "keyFormat" : { + "format" : "KAFKA" + } + }, + "CSAS_OUTPUT_0.Join.Right" : { + "schema" : "`T_ID` BIGINT KEY, `T_ID` BIGINT, `T_NAME` STRING, `T_VALUE` BIGINT, `T_ROWTIME` BIGINT, `T_ROWPARTITION` INTEGER, `T_ROWOFFSET` BIGINT, `T_K` STRING", + "valueFormat" : { + "format" : "PROTOBUF_NOSR" + } + } + }, + "testCase" : { + "name" : "stream stream right join with key in projection - rekey - PROTOBUF_NOSR", + "inputs" : [ { + "topic" : "left_topic", + "key" : "foo", + "value" : { + "ID" : 0, + "NAME" : "zero", + "VALUE" : 0 + }, + "timestamp" : 0 + }, { + "topic" : "right_topic", + "key" : "foo", + "value" : { + "ID" : 0, + "F1" : "blah", + "F2" : 50 + }, + "timestamp" : 10000 + }, { + "topic" : "left_topic", + "key" : "foo", + "value" : { + "ID" : 10, + "NAME" : "100", + "VALUE" : 5 + }, + "timestamp" : 11000 + }, { + "topic" : "left_topic", + "key" : "foo", + "value" : { + "ID" : 0, + "NAME" : "foo", + "VALUE" : 100 + }, + "timestamp" : 13000 + }, { + "topic" : "right_topic", + "key" : "foo", + "value" : { + "ID" : 0, + "F1" : "a", + "F2" : 10 + }, + "timestamp" : 15000 + }, { + "topic" : "right_topic", + "key" : "foo", + "value" : { + "ID" : 100, + "F1" : "newblah", + "F2" : 150 + }, + "timestamp" : 16000 + }, { + "topic" : "left_topic", + "key" : "foo", + "value" : { + "ID" : 90, + "NAME" : "ninety", + "VALUE" : 90 + }, + "timestamp" : 17000 + }, { + "topic" : "left_topic", + "key" : "foo", + "value" : { + "ID" : 0, + "NAME" : "bar", + "VALUE" : 99 + }, + "timestamp" : 30000 + } ], + "outputs" : [ { + "topic" : "OUTPUT", + "key" : 0, + "value" : { + "T_K" : "foo", + "NAME" : "zero", + "VALUE" : 0, + "F1" : "blah", + "F2" : 50 + }, + "timestamp" : 10000 + }, { + "topic" : "OUTPUT", + "key" : 0, + "value" : { + "T_K" : "foo", + "NAME" : "foo", + "VALUE" : 100, + "F1" : "blah", + "F2" : 50 + }, + "timestamp" : 13000 + }, { + "topic" : "OUTPUT", + "key" : 0, + "value" : { + "T_K" : "foo", + "NAME" : "foo", + "VALUE" : 100, + "F1" : "a", + "F2" : 10 + }, + "timestamp" : 15000 + }, { + "topic" : "OUTPUT", + "key" : 100, + "value" : { + "T_K" : "", + "NAME" : "", + "VALUE" : 0, + "F1" : "newblah", + "F2" : 150 + }, + "timestamp" : 16000 + } ], + "topics" : [ { + "name" : "right_topic", + "replicas" : 1, + "numPartitions" : 4 + }, { + "name" : "OUTPUT", + "replicas" : 1, + "numPartitions" : 4 + }, { + "name" : "left_topic", + "replicas" : 1, + "numPartitions" : 4 + } ], + "statements" : [ "CREATE STREAM TEST (K STRING KEY, ID bigint, NAME varchar, VALUE bigint) WITH (kafka_topic='left_topic', value_format='PROTOBUF_NOSR');", "CREATE STREAM TEST_STREAM (K STRING KEY, ID bigint, F1 varchar, F2 bigint) WITH (kafka_topic='right_topic', value_format='PROTOBUF_NOSR');", "CREATE STREAM OUTPUT as SELECT t.id, t.k, name, value, f1, f2 FROM test t RIGHT JOIN test_stream tt WITHIN 11 seconds ON t.id = tt.id;" ], + "post" : { + "sources" : [ { + "name" : "OUTPUT", + "type" : "STREAM", + "schema" : "`T_ID` BIGINT KEY, `T_K` STRING, `NAME` STRING, `VALUE` BIGINT, `F1` STRING, `F2` BIGINT", + "keyFormat" : { + "format" : "KAFKA" + }, + "valueFormat" : "PROTOBUF_NOSR", + "keyFeatures" : [ ], + "valueFeatures" : [ ], + "isSource" : false + }, { + "name" : "TEST", + "type" : "STREAM", + "schema" : "`K` STRING KEY, `ID` BIGINT, `NAME` STRING, `VALUE` BIGINT", + "keyFormat" : { + "format" : "KAFKA" + }, + "valueFormat" : "PROTOBUF_NOSR", + "keyFeatures" : [ ], + "valueFeatures" : [ ], + "isSource" : false + }, { + "name" : "TEST_STREAM", + "type" : "STREAM", + "schema" : "`K` STRING KEY, `ID` BIGINT, `F1` STRING, `F2` BIGINT", + "keyFormat" : { + "format" : "KAFKA" + }, + "valueFormat" : "PROTOBUF_NOSR", + "keyFeatures" : [ ], + "valueFeatures" : [ ], + "isSource" : false + } ], + "topics" : { + "topics" : [ { + "name" : "_confluent-ksql-some.ksql.service.idquery_CSAS_OUTPUT_0-Join-left-repartition", + "keyFormat" : { + "format" : "KAFKA" + }, + "valueFormat" : { + "format" : "PROTOBUF_NOSR" + } + }, { + "name" : "_confluent-ksql-some.ksql.service.idquery_CSAS_OUTPUT_0-Join-right-repartition", + "keyFormat" : { + "format" : "KAFKA" + }, + "valueFormat" : { + "format" : "PROTOBUF_NOSR" + } + }, { + "name" : "_confluent-ksql-some.ksql.service.idquery_CSAS_OUTPUT_0-KSTREAM-JOINTHIS-0000000016-store-changelog", + "keyFormat" : { + "format" : "KAFKA" + }, + "valueFormat" : { + "format" : "PROTOBUF_NOSR" + } + }, { + "name" : "left_topic", + "keyFormat" : { + "format" : "KAFKA" + }, + "valueFormat" : { + "format" : "PROTOBUF_NOSR" + }, + "partitions" : 4 + }, { + "name" : "_confluent-ksql-some.ksql.service.idquery_CSAS_OUTPUT_0-KSTREAM-OUTEROTHER-0000000017-store-changelog", + "keyFormat" : { + "format" : "KAFKA" + }, + "valueFormat" : { + "format" : "PROTOBUF_NOSR" + } + }, { + "name" : "OUTPUT", + "keyFormat" : { + "format" : "KAFKA" + }, + "valueFormat" : { + "format" : "PROTOBUF_NOSR" + }, + "partitions" : 4 + }, { + "name" : "right_topic", + "keyFormat" : { + "format" : "KAFKA" + }, + "valueFormat" : { + "format" : "PROTOBUF_NOSR" + }, + "partitions" : 4 + } ] + } + } + } +} \ No newline at end of file diff --git a/ksqldb-functional-tests/src/test/resources/historical_plans/joins_-_stream_stream_right_join_with_key_in_projection_-_rekey_-_PROTOBUF_NOSR/7.3.0_1651071295926/topology b/ksqldb-functional-tests/src/test/resources/historical_plans/joins_-_stream_stream_right_join_with_key_in_projection_-_rekey_-_PROTOBUF_NOSR/7.3.0_1651071295926/topology new file mode 100644 index 000000000000..6222d30b3022 --- /dev/null +++ b/ksqldb-functional-tests/src/test/resources/historical_plans/joins_-_stream_stream_right_join_with_key_in_projection_-_rekey_-_PROTOBUF_NOSR/7.3.0_1651071295926/topology @@ -0,0 +1,63 @@ +Topologies: + Sub-topology: 0 + Source: KSTREAM-SOURCE-0000000000 (topics: [left_topic]) + --> KSTREAM-TRANSFORMVALUES-0000000001 + Processor: KSTREAM-TRANSFORMVALUES-0000000001 (stores: []) + --> LeftSourceKeyed-SelectKey + <-- KSTREAM-SOURCE-0000000000 + Processor: LeftSourceKeyed-SelectKey (stores: []) + --> PrependAliasLeft + <-- KSTREAM-TRANSFORMVALUES-0000000001 + Processor: PrependAliasLeft (stores: []) + --> Join-right-repartition-filter + <-- LeftSourceKeyed-SelectKey + Processor: Join-right-repartition-filter (stores: []) + --> Join-right-repartition-sink + <-- PrependAliasLeft + Sink: Join-right-repartition-sink (topic: Join-right-repartition) + <-- Join-right-repartition-filter + + Sub-topology: 1 + Source: KSTREAM-SOURCE-0000000004 (topics: [right_topic]) + --> KSTREAM-TRANSFORMVALUES-0000000005 + Processor: KSTREAM-TRANSFORMVALUES-0000000005 (stores: []) + --> RightSourceKeyed-SelectKey + <-- KSTREAM-SOURCE-0000000004 + Processor: RightSourceKeyed-SelectKey (stores: []) + --> PrependAliasRight + <-- KSTREAM-TRANSFORMVALUES-0000000005 + Processor: PrependAliasRight (stores: []) + --> Join-left-repartition-filter + <-- RightSourceKeyed-SelectKey + Processor: Join-left-repartition-filter (stores: []) + --> Join-left-repartition-sink + <-- PrependAliasRight + Sink: Join-left-repartition-sink (topic: Join-left-repartition) + <-- Join-left-repartition-filter + + Sub-topology: 2 + Source: Join-left-repartition-source (topics: [Join-left-repartition]) + --> Join-this-windowed + Source: Join-right-repartition-source (topics: [Join-right-repartition]) + --> Join-other-windowed + Processor: Join-other-windowed (stores: [KSTREAM-OUTEROTHER-0000000017-store]) + --> Join-outer-other-join + <-- Join-right-repartition-source + Processor: Join-this-windowed (stores: [KSTREAM-JOINTHIS-0000000016-store]) + --> Join-this-join + <-- Join-left-repartition-source + Processor: Join-outer-other-join (stores: [KSTREAM-JOINTHIS-0000000016-store]) + --> Join-merge + <-- Join-other-windowed + Processor: Join-this-join (stores: [KSTREAM-OUTEROTHER-0000000017-store]) + --> Join-merge + <-- Join-this-windowed + Processor: Join-merge (stores: []) + --> Project + <-- Join-this-join, Join-outer-other-join + Processor: Project (stores: []) + --> KSTREAM-SINK-0000000020 + <-- Join-merge + Sink: KSTREAM-SINK-0000000020 (topic: OUTPUT) + <-- Project + diff --git a/ksqldb-functional-tests/src/test/resources/historical_plans/joins_-_stream_table_inner_join_-_PROTOBUF_NOSR/7.3.0_1651071395690/plan.json b/ksqldb-functional-tests/src/test/resources/historical_plans/joins_-_stream_table_inner_join_-_PROTOBUF_NOSR/7.3.0_1651071395690/plan.json new file mode 100644 index 000000000000..846b4639558d --- /dev/null +++ b/ksqldb-functional-tests/src/test/resources/historical_plans/joins_-_stream_table_inner_join_-_PROTOBUF_NOSR/7.3.0_1651071395690/plan.json @@ -0,0 +1,302 @@ +{ + "plan" : [ { + "@type" : "ksqlPlanV1", + "statementText" : "CREATE STREAM TEST (ID BIGINT KEY, NAME STRING, VALUE BIGINT) WITH (KAFKA_TOPIC='test_topic', KEY_FORMAT='KAFKA', VALUE_FORMAT='PROTOBUF_NOSR');", + "ddlCommand" : { + "@type" : "createStreamV1", + "sourceName" : "TEST", + "schema" : "`ID` BIGINT KEY, `NAME` STRING, `VALUE` BIGINT", + "timestampColumn" : null, + "topicName" : "test_topic", + "formats" : { + "keyFormat" : { + "format" : "KAFKA", + "properties" : { } + }, + "valueFormat" : { + "format" : "PROTOBUF_NOSR", + "properties" : { } + } + }, + "windowInfo" : null, + "orReplace" : false, + "isSource" : false + }, + "queryPlan" : null + }, { + "@type" : "ksqlPlanV1", + "statementText" : "CREATE TABLE TEST_TABLE (ID BIGINT PRIMARY KEY, F1 STRING, F2 BIGINT) WITH (KAFKA_TOPIC='test_table', KEY_FORMAT='KAFKA', VALUE_FORMAT='PROTOBUF_NOSR');", + "ddlCommand" : { + "@type" : "createTableV1", + "sourceName" : "TEST_TABLE", + "schema" : "`ID` BIGINT KEY, `F1` STRING, `F2` BIGINT", + "timestampColumn" : null, + "topicName" : "test_table", + "formats" : { + "keyFormat" : { + "format" : "KAFKA", + "properties" : { } + }, + "valueFormat" : { + "format" : "PROTOBUF_NOSR", + "properties" : { } + } + }, + "windowInfo" : null, + "orReplace" : false, + "isSource" : false + }, + "queryPlan" : null + }, { + "@type" : "ksqlPlanV1", + "statementText" : "CREATE STREAM INNER_JOIN AS SELECT\n T.ID T_ID,\n T.NAME NAME,\n T.VALUE VALUE,\n TT.F1 F1,\n TT.F2 F2\nFROM TEST T\nINNER JOIN TEST_TABLE TT ON ((T.ID = TT.ID))\nEMIT CHANGES", + "ddlCommand" : { + "@type" : "createStreamV1", + "sourceName" : "INNER_JOIN", + "schema" : "`T_ID` BIGINT KEY, `NAME` STRING, `VALUE` BIGINT, `F1` STRING, `F2` BIGINT", + "timestampColumn" : null, + "topicName" : "INNER_JOIN", + "formats" : { + "keyFormat" : { + "format" : "KAFKA", + "properties" : { } + }, + "valueFormat" : { + "format" : "PROTOBUF_NOSR", + "properties" : { } + } + }, + "windowInfo" : null, + "orReplace" : false, + "isSource" : false + }, + "queryPlan" : { + "sources" : [ "TEST", "TEST_TABLE" ], + "sink" : "INNER_JOIN", + "physicalPlan" : { + "@type" : "streamSinkV1", + "properties" : { + "queryContext" : "INNER_JOIN" + }, + "source" : { + "@type" : "streamSelectV1", + "properties" : { + "queryContext" : "Project" + }, + "source" : { + "@type" : "streamTableJoinV1", + "properties" : { + "queryContext" : "Join" + }, + "joinType" : "INNER", + "internalFormats" : { + "keyFormat" : { + "format" : "KAFKA", + "properties" : { } + }, + "valueFormat" : { + "format" : "PROTOBUF_NOSR", + "properties" : { } + } + }, + "leftSource" : { + "@type" : "streamSelectV1", + "properties" : { + "queryContext" : "PrependAliasLeft" + }, + "source" : { + "@type" : "streamSourceV1", + "properties" : { + "queryContext" : "KafkaTopic_Left/Source" + }, + "topicName" : "test_topic", + "formats" : { + "keyFormat" : { + "format" : "KAFKA", + "properties" : { } + }, + "valueFormat" : { + "format" : "PROTOBUF_NOSR", + "properties" : { } + } + }, + "timestampColumn" : null, + "sourceSchema" : "`ID` BIGINT KEY, `NAME` STRING, `VALUE` BIGINT", + "pseudoColumnVersion" : 1 + }, + "keyColumnNames" : [ "T_ID" ], + "selectedKeys" : null, + "selectExpressions" : [ "NAME AS T_NAME", "VALUE AS T_VALUE", "ROWTIME AS T_ROWTIME", "ROWPARTITION AS T_ROWPARTITION", "ROWOFFSET AS T_ROWOFFSET", "ID AS T_ID" ] + }, + "rightSource" : { + "@type" : "tableSelectV1", + "properties" : { + "queryContext" : "PrependAliasRight" + }, + "source" : { + "@type" : "tableSourceV2", + "properties" : { + "queryContext" : "KafkaTopic_Right/Source" + }, + "topicName" : "test_table", + "formats" : { + "keyFormat" : { + "format" : "KAFKA", + "properties" : { } + }, + "valueFormat" : { + "format" : "PROTOBUF_NOSR", + "properties" : { } + } + }, + "timestampColumn" : null, + "sourceSchema" : "`ID` BIGINT KEY, `F1` STRING, `F2` BIGINT", + "pseudoColumnVersion" : 1, + "stateStoreFormats" : { + "keyFormat" : { + "format" : "KAFKA", + "properties" : { } + }, + "valueFormat" : { + "format" : "PROTOBUF_NOSR", + "properties" : { } + } + } + }, + "keyColumnNames" : [ "TT_ID" ], + "selectExpressions" : [ "F1 AS TT_F1", "F2 AS TT_F2", "ROWTIME AS TT_ROWTIME", "ROWPARTITION AS TT_ROWPARTITION", "ROWOFFSET AS TT_ROWOFFSET", "ID AS TT_ID" ], + "internalFormats" : { + "keyFormat" : { + "format" : "KAFKA", + "properties" : { } + }, + "valueFormat" : { + "format" : "PROTOBUF_NOSR", + "properties" : { } + } + } + }, + "keyColName" : "T_ID" + }, + "keyColumnNames" : [ "T_ID" ], + "selectedKeys" : null, + "selectExpressions" : [ "T_NAME AS NAME", "T_VALUE AS VALUE", "TT_F1 AS F1", "TT_F2 AS F2" ] + }, + "formats" : { + "keyFormat" : { + "format" : "KAFKA", + "properties" : { } + }, + "valueFormat" : { + "format" : "PROTOBUF_NOSR", + "properties" : { } + } + }, + "topicName" : "INNER_JOIN", + "timestampColumn" : null + }, + "queryId" : "CSAS_INNER_JOIN_0", + "runtimeId" : null + } + } ], + "configs" : { + "ksql.extension.dir" : "ext", + "ksql.streams.cache.max.bytes.buffering" : "0", + "metric.reporters" : "", + "ksql.query.status.running.threshold.seconds" : "300", + "ksql.connect.basic.auth.credentials.reload" : "false", + "ksql.output.topic.name.prefix" : "", + "ksql.query.pull.stream.enabled" : "true", + "ksql.query.push.v2.interpreter.enabled" : "true", + "ksql.queryanonymizer.logs_enabled" : "true", + "ksql.variable.substitution.enable" : "true", + "ksql.streams.shutdown.timeout.ms" : "300000", + "ksql.query.pull.max.allowed.offset.lag" : "9223372036854775807", + "ksql.query.pull.max.qps" : "2147483647", + "ksql.access.validator.enable" : "auto", + "ksql.streams.bootstrap.servers" : "localhost:0", + "ksql.query.pull.metrics.enabled" : "true", + "ksql.metrics.extension" : null, + "ksql.query.push.v2.alos.enabled" : "true", + "ksql.query.push.v2.max.hourly.bandwidth.megabytes" : "2147483647", + "ksql.query.pull.range.scan.enabled" : "true", + "ksql.transient.query.cleanup.service.initial.delay.seconds" : "600", + "ksql.hidden.topics" : "_confluent.*,__confluent.*,_schemas,__consumer_offsets,__transaction_state,connect-configs,connect-offsets,connect-status,connect-statuses", + "ksql.lambdas.enabled" : "true", + "ksql.source.table.materialization.enabled" : "true", + "ksql.query.pull.max.hourly.bandwidth.megabytes" : "2147483647", + "ksql.sink.window.change.log.additional.retention" : "1000000", + "ksql.query.persistent.active.limit" : "2147483647", + "ksql.persistence.wrap.single.values" : null, + "ksql.query.transient.max.bytes.buffering.total" : "-1", + "ksql.connect.basic.auth.credentials.source" : "NONE", + "ksql.schema.registry.url" : "schema_registry.url:0", + "ksql.properties.overrides.denylist" : "", + "ksql.service.id" : "some.ksql.service.id", + "ksql.query.push.v2.max.catchup.consumers" : "5", + "ksql.query.push.v2.enabled" : "false", + "ksql.transient.query.cleanup.service.enable" : "true", + "ksql.query.push.v2.metrics.enabled" : "true", + "ksql.rowpartition.rowoffset.enabled" : "true", + "ksql.streams.commit.interval.ms" : "2000", + "ksql.query.pull.table.scan.enabled" : "true", + "ksql.streams.auto.commit.interval.ms" : "0", + "ksql.streams.topology.optimization" : "all", + "ksql.endpoint.migrate.query" : "true", + "ksql.query.push.v2.registry.installed" : "false", + "ksql.streams.num.stream.threads" : "4", + "ksql.metrics.tags.custom" : "", + "ksql.query.push.v2.catchup.consumer.msg.window" : "50", + "ksql.runtime.feature.shared.enabled" : "false", + "ksql.udf.collect.metrics" : "false", + "ksql.new.query.planner.enabled" : "false", + "ksql.connect.request.headers.plugin" : null, + "ksql.security.extension.class" : null, + "ksql.transient.prefix" : "transient_", + "ksql.headers.columns.enabled" : "true", + "ksql.streams.default.deserialization.exception.handler" : "io.confluent.ksql.errors.LogMetricAndContinueExceptionHandler", + "ksql.connect.request.timeout.ms" : "5000", + "ksql.query.pull.enable.standby.reads" : "false", + "ksql.persistence.default.format.key" : "KAFKA", + "ksql.query.persistent.max.bytes.buffering.total" : "-1", + "ksql.query.error.max.queue.size" : "10", + "ksql.query.cleanup.shutdown.timeout.ms" : "30000", + "ksql.internal.topic.min.insync.replicas" : "1", + "ksql.internal.topic.replicas" : "1", + "ksql.insert.into.values.enabled" : "true", + "ksql.queryanonymizer.cluster_namespace" : null, + "ksql.create.or.replace.enabled" : "true", + "ksql.shared.runtimes.count" : "2", + "ksql.cast.strings.preserve.nulls" : "true", + "ksql.authorization.cache.max.entries" : "10000", + "ksql.pull.queries.enable" : "true", + "ksql.transient.query.cleanup.service.period.seconds" : "600", + "ksql.suppress.enabled" : "false", + "ksql.readonly.topics" : "_confluent.*,__confluent.*,_schemas,__consumer_offsets,__transaction_state,connect-configs,connect-offsets,connect-status,connect-statuses", + "ksql.connect.basic.auth.credentials.file" : "", + "ksql.authorization.cache.expiry.time.secs" : "30", + "ksql.query.retry.backoff.initial.ms" : "15000", + "ksql.query.pull.max.concurrent.requests" : "2147483647", + "ksql.streams.auto.offset.reset" : "earliest", + "ksql.connect.url" : "http://localhost:8083", + "ksql.query.push.v2.new.latest.delay.ms" : "5000", + "ksql.query.push.v2.latest.reset.age.ms" : "30000", + "ksql.streams.default.production.exception.handler" : "io.confluent.ksql.errors.ProductionExceptionHandlerUtil$LogAndFailProductionExceptionHandler", + "ksql.query.pull.interpreter.enabled" : "true", + "ksql.query.pull.limit.clause.enabled" : "true", + "ksql.query.pull.router.thread.pool.size" : "50", + "ksql.query.push.v2.continuation.tokens.enabled" : "false", + "ksql.query.retry.backoff.max.ms" : "900000", + "ksql.timestamp.throw.on.invalid" : "false", + "ksql.persistence.default.format.value" : null, + "ksql.udfs.enabled" : "true", + "ksql.udf.enable.security.manager" : "true", + "ksql.connect.worker.config" : "", + "ksql.nested.error.set.null" : "true", + "ksql.query.pull.thread.pool.size" : "50", + "ksql.persistent.prefix" : "query_", + "ksql.metastore.backup.location" : "", + "ksql.error.classifier.regex" : "", + "ksql.suppress.buffer.size.bytes" : "-1" + } +} \ No newline at end of file diff --git a/ksqldb-functional-tests/src/test/resources/historical_plans/joins_-_stream_table_inner_join_-_PROTOBUF_NOSR/7.3.0_1651071395690/spec.json b/ksqldb-functional-tests/src/test/resources/historical_plans/joins_-_stream_table_inner_join_-_PROTOBUF_NOSR/7.3.0_1651071395690/spec.json new file mode 100644 index 000000000000..fb07a5894244 --- /dev/null +++ b/ksqldb-functional-tests/src/test/resources/historical_plans/joins_-_stream_table_inner_join_-_PROTOBUF_NOSR/7.3.0_1651071395690/spec.json @@ -0,0 +1,244 @@ +{ + "version" : "7.3.0", + "timestamp" : 1651071395690, + "path" : "query-validation-tests/joins.json", + "schemas" : { + "CSAS_INNER_JOIN_0.INNER_JOIN" : { + "schema" : "`T_ID` BIGINT KEY, `NAME` STRING, `VALUE` BIGINT, `F1` STRING, `F2` BIGINT", + "keyFormat" : { + "format" : "KAFKA" + }, + "valueFormat" : { + "format" : "PROTOBUF_NOSR" + } + }, + "CSAS_INNER_JOIN_0.Join.Left" : { + "schema" : "`T_ID` BIGINT KEY, `T_NAME` STRING, `T_VALUE` BIGINT, `T_ROWTIME` BIGINT, `T_ROWPARTITION` INTEGER, `T_ROWOFFSET` BIGINT, `T_ID` BIGINT", + "valueFormat" : { + "format" : "PROTOBUF_NOSR" + } + }, + "CSAS_INNER_JOIN_0.KafkaTopic_Right.Source" : { + "schema" : "`ID` BIGINT KEY, `F1` STRING, `F2` BIGINT", + "keyFormat" : { + "format" : "KAFKA" + }, + "valueFormat" : { + "format" : "PROTOBUF_NOSR" + } + }, + "CSAS_INNER_JOIN_0.Join" : { + "schema" : "`T_ID` BIGINT KEY, `T_NAME` STRING, `T_VALUE` BIGINT, `T_ROWTIME` BIGINT, `T_ROWPARTITION` INTEGER, `T_ROWOFFSET` BIGINT, `T_ID` BIGINT", + "keyFormat" : { + "format" : "KAFKA" + } + }, + "CSAS_INNER_JOIN_0.KafkaTopic_Left.Source" : { + "schema" : "`ID` BIGINT KEY, `NAME` STRING, `VALUE` BIGINT", + "keyFormat" : { + "format" : "KAFKA" + }, + "valueFormat" : { + "format" : "PROTOBUF_NOSR" + } + }, + "CSAS_INNER_JOIN_0.PrependAliasRight" : { + "schema" : "`TT_ID` BIGINT KEY, `TT_F1` STRING, `TT_F2` BIGINT, `TT_ROWTIME` BIGINT, `TT_ROWPARTITION` INTEGER, `TT_ROWOFFSET` BIGINT, `TT_ID` BIGINT", + "keyFormat" : { + "format" : "KAFKA" + }, + "valueFormat" : { + "format" : "PROTOBUF_NOSR" + } + }, + "CSAS_INNER_JOIN_0.KafkaTopic_Right.Source.Materialized" : { + "schema" : "`ID` BIGINT KEY, `F1` STRING, `F2` BIGINT, `ROWPARTITION` INTEGER, `ROWOFFSET` BIGINT", + "keyFormat" : { + "format" : "KAFKA" + }, + "valueFormat" : { + "format" : "PROTOBUF_NOSR" + } + } + }, + "testCase" : { + "name" : "stream table inner join - PROTOBUF_NOSR", + "inputs" : [ { + "topic" : "test_table", + "key" : 0, + "value" : { + "F1" : "zero", + "F2" : 0 + }, + "timestamp" : 0 + }, { + "topic" : "test_table", + "key" : 10, + "value" : { + "F1" : "100", + "F2" : 5 + }, + "timestamp" : 10000 + }, { + "topic" : "test_topic", + "key" : 0, + "value" : { + "NAME" : "blah", + "VALUE" : 50 + }, + "timestamp" : 10000 + }, { + "topic" : "test_topic", + "key" : 0, + "value" : { + "NAME" : "foo", + "VALUE" : 100 + }, + "timestamp" : 10000 + }, { + "topic" : "test_table", + "key" : 0, + "value" : { + "F1" : "a", + "F2" : 10 + }, + "timestamp" : 15000 + }, { + "topic" : "test_topic", + "key" : 0, + "value" : { + "NAME" : "bar", + "VALUE" : 99 + }, + "timestamp" : 15000 + }, { + "topic" : "test_topic", + "key" : 90, + "value" : { + "NAME" : "ninety", + "VALUE" : 90 + }, + "timestamp" : 15000 + } ], + "outputs" : [ { + "topic" : "INNER_JOIN", + "key" : 0, + "value" : { + "NAME" : "blah", + "VALUE" : 50, + "F1" : "zero", + "F2" : 0 + }, + "timestamp" : 10000 + }, { + "topic" : "INNER_JOIN", + "key" : 0, + "value" : { + "NAME" : "foo", + "VALUE" : 100, + "F1" : "zero", + "F2" : 0 + }, + "timestamp" : 10000 + }, { + "topic" : "INNER_JOIN", + "key" : 0, + "value" : { + "NAME" : "bar", + "VALUE" : 99, + "F1" : "a", + "F2" : 10 + }, + "timestamp" : 15000 + } ], + "topics" : [ { + "name" : "test_topic", + "replicas" : 1, + "numPartitions" : 4 + }, { + "name" : "INNER_JOIN", + "replicas" : 1, + "numPartitions" : 4 + }, { + "name" : "test_table", + "replicas" : 1, + "numPartitions" : 4 + } ], + "statements" : [ "CREATE STREAM TEST (ID BIGINT KEY, NAME varchar, VALUE bigint) WITH (kafka_topic='test_topic', value_format='PROTOBUF_NOSR');", "CREATE TABLE TEST_TABLE (id BIGINT PRIMARY KEY, f1 VARCHAR, f2 BIGINT) WITH (kafka_topic='test_table', value_format='PROTOBUF_NOSR');", "CREATE STREAM INNER_JOIN as SELECT t.id, name, value, f1, f2 FROM test t join test_table tt on t.id = tt.id;" ], + "post" : { + "sources" : [ { + "name" : "INNER_JOIN", + "type" : "STREAM", + "schema" : "`T_ID` BIGINT KEY, `NAME` STRING, `VALUE` BIGINT, `F1` STRING, `F2` BIGINT", + "keyFormat" : { + "format" : "KAFKA" + }, + "valueFormat" : "PROTOBUF_NOSR", + "keyFeatures" : [ ], + "valueFeatures" : [ ], + "isSource" : false + }, { + "name" : "TEST", + "type" : "STREAM", + "schema" : "`ID` BIGINT KEY, `NAME` STRING, `VALUE` BIGINT", + "keyFormat" : { + "format" : "KAFKA" + }, + "valueFormat" : "PROTOBUF_NOSR", + "keyFeatures" : [ ], + "valueFeatures" : [ ], + "isSource" : false + }, { + "name" : "TEST_TABLE", + "type" : "TABLE", + "schema" : "`ID` BIGINT KEY, `F1` STRING, `F2` BIGINT", + "keyFormat" : { + "format" : "KAFKA" + }, + "valueFormat" : "PROTOBUF_NOSR", + "keyFeatures" : [ ], + "valueFeatures" : [ ], + "isSource" : false + } ], + "topics" : { + "topics" : [ { + "name" : "_confluent-ksql-some.ksql.service.idquery_CSAS_INNER_JOIN_0-KafkaTopic_Right-Reduce-changelog", + "keyFormat" : { + "format" : "KAFKA" + }, + "valueFormat" : { + "format" : "PROTOBUF_NOSR" + } + }, { + "name" : "INNER_JOIN", + "keyFormat" : { + "format" : "KAFKA" + }, + "valueFormat" : { + "format" : "PROTOBUF_NOSR" + }, + "partitions" : 4 + }, { + "name" : "test_topic", + "keyFormat" : { + "format" : "KAFKA" + }, + "valueFormat" : { + "format" : "PROTOBUF_NOSR" + }, + "partitions" : 4 + }, { + "name" : "test_table", + "keyFormat" : { + "format" : "KAFKA" + }, + "valueFormat" : { + "format" : "PROTOBUF_NOSR" + }, + "partitions" : 4 + } ], + "blackList" : ".*-repartition" + } + } + } +} \ No newline at end of file diff --git a/ksqldb-functional-tests/src/test/resources/historical_plans/joins_-_stream_table_inner_join_-_PROTOBUF_NOSR/7.3.0_1651071395690/topology b/ksqldb-functional-tests/src/test/resources/historical_plans/joins_-_stream_table_inner_join_-_PROTOBUF_NOSR/7.3.0_1651071395690/topology new file mode 100644 index 000000000000..88e4f3b1bc78 --- /dev/null +++ b/ksqldb-functional-tests/src/test/resources/historical_plans/joins_-_stream_table_inner_join_-_PROTOBUF_NOSR/7.3.0_1651071395690/topology @@ -0,0 +1,33 @@ +Topologies: + Sub-topology: 0 + Source: KSTREAM-SOURCE-0000000006 (topics: [test_topic]) + --> KSTREAM-TRANSFORMVALUES-0000000007 + Source: KSTREAM-SOURCE-0000000001 (topics: [test_table]) + --> KTABLE-SOURCE-0000000002 + Processor: KSTREAM-TRANSFORMVALUES-0000000007 (stores: []) + --> PrependAliasLeft + <-- KSTREAM-SOURCE-0000000006 + Processor: KTABLE-SOURCE-0000000002 (stores: []) + --> KTABLE-TRANSFORMVALUES-0000000003 + <-- KSTREAM-SOURCE-0000000001 + Processor: PrependAliasLeft (stores: []) + --> Join + <-- KSTREAM-TRANSFORMVALUES-0000000007 + Processor: Join (stores: [KafkaTopic_Right-Reduce]) + --> Project + <-- PrependAliasLeft + Processor: KTABLE-TRANSFORMVALUES-0000000003 (stores: [KafkaTopic_Right-Reduce]) + --> KTABLE-TRANSFORMVALUES-0000000004 + <-- KTABLE-SOURCE-0000000002 + Processor: KTABLE-TRANSFORMVALUES-0000000004 (stores: []) + --> PrependAliasRight + <-- KTABLE-TRANSFORMVALUES-0000000003 + Processor: Project (stores: []) + --> KSTREAM-SINK-0000000011 + <-- Join + Sink: KSTREAM-SINK-0000000011 (topic: INNER_JOIN) + <-- Project + Processor: PrependAliasRight (stores: []) + --> none + <-- KTABLE-TRANSFORMVALUES-0000000004 + diff --git a/ksqldb-functional-tests/src/test/resources/historical_plans/joins_-_stream_table_left_join_-_PROTOBUF_-_PROTOBUF/7.3.0_1651071391893/plan.json b/ksqldb-functional-tests/src/test/resources/historical_plans/joins_-_stream_table_left_join_-_PROTOBUF_-_PROTOBUF/7.3.0_1651071391893/plan.json new file mode 100644 index 000000000000..3c1fe369541f --- /dev/null +++ b/ksqldb-functional-tests/src/test/resources/historical_plans/joins_-_stream_table_left_join_-_PROTOBUF_-_PROTOBUF/7.3.0_1651071391893/plan.json @@ -0,0 +1,320 @@ +{ + "plan" : [ { + "@type" : "ksqlPlanV1", + "statementText" : "CREATE STREAM TEST (ID BIGINT KEY, NAME STRING, VALUE BIGINT) WITH (KAFKA_TOPIC='test_topic', KEY_FORMAT='KAFKA', VALUE_FORMAT='PROTOBUF');", + "ddlCommand" : { + "@type" : "createStreamV1", + "sourceName" : "TEST", + "schema" : "`ID` BIGINT KEY, `NAME` STRING, `VALUE` BIGINT", + "timestampColumn" : null, + "topicName" : "test_topic", + "formats" : { + "keyFormat" : { + "format" : "KAFKA", + "properties" : { } + }, + "valueFormat" : { + "format" : "PROTOBUF", + "properties" : { + "unwrapPrimitives" : "true" + } + } + }, + "windowInfo" : null, + "orReplace" : false, + "isSource" : false + }, + "queryPlan" : null + }, { + "@type" : "ksqlPlanV1", + "statementText" : "CREATE TABLE TEST_TABLE (ID BIGINT PRIMARY KEY, F1 STRING, F2 BIGINT) WITH (KAFKA_TOPIC='test_table', KEY_FORMAT='KAFKA', VALUE_FORMAT='PROTOBUF');", + "ddlCommand" : { + "@type" : "createTableV1", + "sourceName" : "TEST_TABLE", + "schema" : "`ID` BIGINT KEY, `F1` STRING, `F2` BIGINT", + "timestampColumn" : null, + "topicName" : "test_table", + "formats" : { + "keyFormat" : { + "format" : "KAFKA", + "properties" : { } + }, + "valueFormat" : { + "format" : "PROTOBUF", + "properties" : { + "unwrapPrimitives" : "true" + } + } + }, + "windowInfo" : null, + "orReplace" : false, + "isSource" : false + }, + "queryPlan" : null + }, { + "@type" : "ksqlPlanV1", + "statementText" : "CREATE STREAM LEFT_JOIN AS SELECT\n T.ID T_ID,\n T.NAME NAME,\n T.VALUE VALUE,\n TT.F1 F1,\n TT.F2 F2\nFROM TEST T\nLEFT OUTER JOIN TEST_TABLE TT ON ((T.ID = TT.ID))\nEMIT CHANGES", + "ddlCommand" : { + "@type" : "createStreamV1", + "sourceName" : "LEFT_JOIN", + "schema" : "`T_ID` BIGINT KEY, `NAME` STRING, `VALUE` BIGINT, `F1` STRING, `F2` BIGINT", + "timestampColumn" : null, + "topicName" : "LEFT_JOIN", + "formats" : { + "keyFormat" : { + "format" : "KAFKA", + "properties" : { } + }, + "valueFormat" : { + "format" : "PROTOBUF", + "properties" : { + "unwrapPrimitives" : "true" + } + } + }, + "windowInfo" : null, + "orReplace" : false, + "isSource" : false + }, + "queryPlan" : { + "sources" : [ "TEST", "TEST_TABLE" ], + "sink" : "LEFT_JOIN", + "physicalPlan" : { + "@type" : "streamSinkV1", + "properties" : { + "queryContext" : "LEFT_JOIN" + }, + "source" : { + "@type" : "streamSelectV1", + "properties" : { + "queryContext" : "Project" + }, + "source" : { + "@type" : "streamTableJoinV1", + "properties" : { + "queryContext" : "Join" + }, + "joinType" : "LEFT", + "internalFormats" : { + "keyFormat" : { + "format" : "KAFKA", + "properties" : { } + }, + "valueFormat" : { + "format" : "PROTOBUF", + "properties" : { + "unwrapPrimitives" : "true" + } + } + }, + "leftSource" : { + "@type" : "streamSelectV1", + "properties" : { + "queryContext" : "PrependAliasLeft" + }, + "source" : { + "@type" : "streamSourceV1", + "properties" : { + "queryContext" : "KafkaTopic_Left/Source" + }, + "topicName" : "test_topic", + "formats" : { + "keyFormat" : { + "format" : "KAFKA", + "properties" : { } + }, + "valueFormat" : { + "format" : "PROTOBUF", + "properties" : { + "unwrapPrimitives" : "true" + } + } + }, + "timestampColumn" : null, + "sourceSchema" : "`ID` BIGINT KEY, `NAME` STRING, `VALUE` BIGINT", + "pseudoColumnVersion" : 1 + }, + "keyColumnNames" : [ "T_ID" ], + "selectedKeys" : null, + "selectExpressions" : [ "NAME AS T_NAME", "VALUE AS T_VALUE", "ROWTIME AS T_ROWTIME", "ROWPARTITION AS T_ROWPARTITION", "ROWOFFSET AS T_ROWOFFSET", "ID AS T_ID" ] + }, + "rightSource" : { + "@type" : "tableSelectV1", + "properties" : { + "queryContext" : "PrependAliasRight" + }, + "source" : { + "@type" : "tableSourceV2", + "properties" : { + "queryContext" : "KafkaTopic_Right/Source" + }, + "topicName" : "test_table", + "formats" : { + "keyFormat" : { + "format" : "KAFKA", + "properties" : { } + }, + "valueFormat" : { + "format" : "PROTOBUF", + "properties" : { + "unwrapPrimitives" : "true" + } + } + }, + "timestampColumn" : null, + "sourceSchema" : "`ID` BIGINT KEY, `F1` STRING, `F2` BIGINT", + "pseudoColumnVersion" : 1, + "stateStoreFormats" : { + "keyFormat" : { + "format" : "KAFKA", + "properties" : { } + }, + "valueFormat" : { + "format" : "PROTOBUF", + "properties" : { + "unwrapPrimitives" : "true" + } + } + } + }, + "keyColumnNames" : [ "TT_ID" ], + "selectExpressions" : [ "F1 AS TT_F1", "F2 AS TT_F2", "ROWTIME AS TT_ROWTIME", "ROWPARTITION AS TT_ROWPARTITION", "ROWOFFSET AS TT_ROWOFFSET", "ID AS TT_ID" ], + "internalFormats" : { + "keyFormat" : { + "format" : "KAFKA", + "properties" : { } + }, + "valueFormat" : { + "format" : "PROTOBUF", + "properties" : { + "unwrapPrimitives" : "true" + } + } + } + }, + "keyColName" : "T_ID" + }, + "keyColumnNames" : [ "T_ID" ], + "selectedKeys" : null, + "selectExpressions" : [ "T_NAME AS NAME", "T_VALUE AS VALUE", "TT_F1 AS F1", "TT_F2 AS F2" ] + }, + "formats" : { + "keyFormat" : { + "format" : "KAFKA", + "properties" : { } + }, + "valueFormat" : { + "format" : "PROTOBUF", + "properties" : { + "unwrapPrimitives" : "true" + } + } + }, + "topicName" : "LEFT_JOIN", + "timestampColumn" : null + }, + "queryId" : "CSAS_LEFT_JOIN_0", + "runtimeId" : null + } + } ], + "configs" : { + "ksql.extension.dir" : "ext", + "ksql.streams.cache.max.bytes.buffering" : "0", + "metric.reporters" : "", + "ksql.query.status.running.threshold.seconds" : "300", + "ksql.connect.basic.auth.credentials.reload" : "false", + "ksql.output.topic.name.prefix" : "", + "ksql.query.pull.stream.enabled" : "true", + "ksql.query.push.v2.interpreter.enabled" : "true", + "ksql.queryanonymizer.logs_enabled" : "true", + "ksql.variable.substitution.enable" : "true", + "ksql.streams.shutdown.timeout.ms" : "300000", + "ksql.query.pull.max.allowed.offset.lag" : "9223372036854775807", + "ksql.query.pull.max.qps" : "2147483647", + "ksql.access.validator.enable" : "auto", + "ksql.streams.bootstrap.servers" : "localhost:0", + "ksql.query.pull.metrics.enabled" : "true", + "ksql.metrics.extension" : null, + "ksql.query.push.v2.alos.enabled" : "true", + "ksql.query.push.v2.max.hourly.bandwidth.megabytes" : "2147483647", + "ksql.query.pull.range.scan.enabled" : "true", + "ksql.transient.query.cleanup.service.initial.delay.seconds" : "600", + "ksql.hidden.topics" : "_confluent.*,__confluent.*,_schemas,__consumer_offsets,__transaction_state,connect-configs,connect-offsets,connect-status,connect-statuses", + "ksql.lambdas.enabled" : "true", + "ksql.source.table.materialization.enabled" : "true", + "ksql.query.pull.max.hourly.bandwidth.megabytes" : "2147483647", + "ksql.sink.window.change.log.additional.retention" : "1000000", + "ksql.query.persistent.active.limit" : "2147483647", + "ksql.persistence.wrap.single.values" : null, + "ksql.query.transient.max.bytes.buffering.total" : "-1", + "ksql.connect.basic.auth.credentials.source" : "NONE", + "ksql.schema.registry.url" : "schema_registry.url:0", + "ksql.properties.overrides.denylist" : "", + "ksql.service.id" : "some.ksql.service.id", + "ksql.query.push.v2.max.catchup.consumers" : "5", + "ksql.query.push.v2.enabled" : "false", + "ksql.transient.query.cleanup.service.enable" : "true", + "ksql.query.push.v2.metrics.enabled" : "true", + "ksql.rowpartition.rowoffset.enabled" : "true", + "ksql.streams.commit.interval.ms" : "2000", + "ksql.query.pull.table.scan.enabled" : "true", + "ksql.streams.auto.commit.interval.ms" : "0", + "ksql.streams.topology.optimization" : "all", + "ksql.endpoint.migrate.query" : "true", + "ksql.query.push.v2.registry.installed" : "false", + "ksql.streams.num.stream.threads" : "4", + "ksql.metrics.tags.custom" : "", + "ksql.query.push.v2.catchup.consumer.msg.window" : "50", + "ksql.runtime.feature.shared.enabled" : "false", + "ksql.udf.collect.metrics" : "false", + "ksql.new.query.planner.enabled" : "false", + "ksql.connect.request.headers.plugin" : null, + "ksql.security.extension.class" : null, + "ksql.transient.prefix" : "transient_", + "ksql.headers.columns.enabled" : "true", + "ksql.streams.default.deserialization.exception.handler" : "io.confluent.ksql.errors.LogMetricAndContinueExceptionHandler", + "ksql.connect.request.timeout.ms" : "5000", + "ksql.query.pull.enable.standby.reads" : "false", + "ksql.persistence.default.format.key" : "KAFKA", + "ksql.query.persistent.max.bytes.buffering.total" : "-1", + "ksql.query.error.max.queue.size" : "10", + "ksql.query.cleanup.shutdown.timeout.ms" : "30000", + "ksql.internal.topic.min.insync.replicas" : "1", + "ksql.internal.topic.replicas" : "1", + "ksql.insert.into.values.enabled" : "true", + "ksql.queryanonymizer.cluster_namespace" : null, + "ksql.create.or.replace.enabled" : "true", + "ksql.shared.runtimes.count" : "2", + "ksql.cast.strings.preserve.nulls" : "true", + "ksql.authorization.cache.max.entries" : "10000", + "ksql.pull.queries.enable" : "true", + "ksql.transient.query.cleanup.service.period.seconds" : "600", + "ksql.suppress.enabled" : "false", + "ksql.readonly.topics" : "_confluent.*,__confluent.*,_schemas,__consumer_offsets,__transaction_state,connect-configs,connect-offsets,connect-status,connect-statuses", + "ksql.connect.basic.auth.credentials.file" : "", + "ksql.authorization.cache.expiry.time.secs" : "30", + "ksql.query.retry.backoff.initial.ms" : "15000", + "ksql.query.pull.max.concurrent.requests" : "2147483647", + "ksql.streams.auto.offset.reset" : "earliest", + "ksql.connect.url" : "http://localhost:8083", + "ksql.query.push.v2.new.latest.delay.ms" : "5000", + "ksql.query.push.v2.latest.reset.age.ms" : "30000", + "ksql.streams.default.production.exception.handler" : "io.confluent.ksql.errors.ProductionExceptionHandlerUtil$LogAndFailProductionExceptionHandler", + "ksql.query.pull.interpreter.enabled" : "true", + "ksql.query.pull.limit.clause.enabled" : "true", + "ksql.query.pull.router.thread.pool.size" : "50", + "ksql.query.push.v2.continuation.tokens.enabled" : "false", + "ksql.query.retry.backoff.max.ms" : "900000", + "ksql.timestamp.throw.on.invalid" : "false", + "ksql.persistence.default.format.value" : null, + "ksql.udfs.enabled" : "true", + "ksql.udf.enable.security.manager" : "true", + "ksql.connect.worker.config" : "", + "ksql.nested.error.set.null" : "true", + "ksql.query.pull.thread.pool.size" : "50", + "ksql.persistent.prefix" : "query_", + "ksql.metastore.backup.location" : "", + "ksql.error.classifier.regex" : "", + "ksql.suppress.buffer.size.bytes" : "-1" + } +} \ No newline at end of file diff --git a/ksqldb-functional-tests/src/test/resources/historical_plans/joins_-_stream_table_left_join_-_PROTOBUF_-_PROTOBUF/7.3.0_1651071391893/spec.json b/ksqldb-functional-tests/src/test/resources/historical_plans/joins_-_stream_table_left_join_-_PROTOBUF_-_PROTOBUF/7.3.0_1651071391893/spec.json new file mode 100644 index 000000000000..81e6bc2631aa --- /dev/null +++ b/ksqldb-functional-tests/src/test/resources/historical_plans/joins_-_stream_table_left_join_-_PROTOBUF_-_PROTOBUF/7.3.0_1651071391893/spec.json @@ -0,0 +1,292 @@ +{ + "version" : "7.3.0", + "timestamp" : 1651071391893, + "path" : "query-validation-tests/joins.json", + "schemas" : { + "CSAS_LEFT_JOIN_0.Join" : { + "schema" : "`T_ID` BIGINT KEY, `T_NAME` STRING, `T_VALUE` BIGINT, `T_ROWTIME` BIGINT, `T_ROWPARTITION` INTEGER, `T_ROWOFFSET` BIGINT, `T_ID` BIGINT", + "keyFormat" : { + "format" : "KAFKA" + } + }, + "CSAS_LEFT_JOIN_0.LEFT_JOIN" : { + "schema" : "`T_ID` BIGINT KEY, `NAME` STRING, `VALUE` BIGINT, `F1` STRING, `F2` BIGINT", + "keyFormat" : { + "format" : "KAFKA" + }, + "valueFormat" : { + "format" : "PROTOBUF", + "properties" : { + "unwrapPrimitives" : "true" + } + } + }, + "CSAS_LEFT_JOIN_0.KafkaTopic_Left.Source" : { + "schema" : "`ID` BIGINT KEY, `NAME` STRING, `VALUE` BIGINT", + "keyFormat" : { + "format" : "KAFKA" + }, + "valueFormat" : { + "format" : "PROTOBUF", + "properties" : { + "unwrapPrimitives" : "true" + } + } + }, + "CSAS_LEFT_JOIN_0.KafkaTopic_Right.Source.Materialized" : { + "schema" : "`ID` BIGINT KEY, `F1` STRING, `F2` BIGINT, `ROWPARTITION` INTEGER, `ROWOFFSET` BIGINT", + "keyFormat" : { + "format" : "KAFKA" + }, + "valueFormat" : { + "format" : "PROTOBUF", + "properties" : { + "unwrapPrimitives" : "true" + } + } + }, + "CSAS_LEFT_JOIN_0.PrependAliasRight" : { + "schema" : "`TT_ID` BIGINT KEY, `TT_F1` STRING, `TT_F2` BIGINT, `TT_ROWTIME` BIGINT, `TT_ROWPARTITION` INTEGER, `TT_ROWOFFSET` BIGINT, `TT_ID` BIGINT", + "keyFormat" : { + "format" : "KAFKA" + }, + "valueFormat" : { + "format" : "PROTOBUF", + "properties" : { + "unwrapPrimitives" : "true" + } + } + }, + "CSAS_LEFT_JOIN_0.Join.Left" : { + "schema" : "`T_ID` BIGINT KEY, `T_NAME` STRING, `T_VALUE` BIGINT, `T_ROWTIME` BIGINT, `T_ROWPARTITION` INTEGER, `T_ROWOFFSET` BIGINT, `T_ID` BIGINT", + "valueFormat" : { + "format" : "PROTOBUF", + "properties" : { + "unwrapPrimitives" : "true" + } + } + }, + "CSAS_LEFT_JOIN_0.KafkaTopic_Right.Source" : { + "schema" : "`ID` BIGINT KEY, `F1` STRING, `F2` BIGINT", + "keyFormat" : { + "format" : "KAFKA" + }, + "valueFormat" : { + "format" : "PROTOBUF", + "properties" : { + "unwrapPrimitives" : "true" + } + } + } + }, + "testCase" : { + "name" : "stream table left join - PROTOBUF - PROTOBUF", + "inputs" : [ { + "topic" : "test_table", + "key" : 0, + "value" : { + "F1" : "zero", + "F2" : 0 + }, + "timestamp" : 0 + }, { + "topic" : "test_table", + "key" : 10, + "value" : { + "F1" : "100", + "F2" : 5 + }, + "timestamp" : 10000 + }, { + "topic" : "test_topic", + "key" : 0, + "value" : { + "NAME" : "blah", + "VALUE" : 50 + }, + "timestamp" : 10000 + }, { + "topic" : "test_topic", + "key" : 0, + "value" : { + "NAME" : "foo", + "VALUE" : 100 + }, + "timestamp" : 10000 + }, { + "topic" : "test_table", + "key" : 0, + "value" : { + "F1" : "a", + "F2" : 10 + }, + "timestamp" : 15000 + }, { + "topic" : "test_topic", + "key" : 0, + "value" : { + "NAME" : "bar", + "VALUE" : 99 + }, + "timestamp" : 15000 + }, { + "topic" : "test_topic", + "key" : 90, + "value" : { + "NAME" : "ninety", + "VALUE" : 90 + }, + "timestamp" : 15000 + } ], + "outputs" : [ { + "topic" : "LEFT_JOIN", + "key" : 0, + "value" : { + "NAME" : "blah", + "VALUE" : 50, + "F1" : "zero", + "F2" : 0 + }, + "timestamp" : 10000 + }, { + "topic" : "LEFT_JOIN", + "key" : 0, + "value" : { + "NAME" : "foo", + "VALUE" : 100, + "F1" : "zero", + "F2" : 0 + }, + "timestamp" : 10000 + }, { + "topic" : "LEFT_JOIN", + "key" : 0, + "value" : { + "NAME" : "bar", + "VALUE" : 99, + "F1" : "a", + "F2" : 10 + }, + "timestamp" : 15000 + }, { + "topic" : "LEFT_JOIN", + "key" : 90, + "value" : { + "NAME" : "ninety", + "VALUE" : 90, + "F1" : "", + "F2" : 0 + }, + "timestamp" : 15000 + } ], + "topics" : [ { + "name" : "test_topic", + "valueSchema" : "syntax = \"proto3\";\n\nmessage ConnectDefault1 {\n string NAME = 1;\n int64 VALUE = 2;\n}\n", + "valueFormat" : "PROTOBUF", + "replicas" : 1, + "numPartitions" : 4 + }, { + "name" : "LEFT_JOIN", + "replicas" : 1, + "numPartitions" : 4 + }, { + "name" : "test_table", + "valueSchema" : "syntax = \"proto3\";\n\nmessage ConnectDefault1 {\n string F1 = 1;\n int64 F2 = 2;\n}\n", + "valueFormat" : "PROTOBUF", + "replicas" : 1, + "numPartitions" : 4 + } ], + "statements" : [ "CREATE STREAM TEST (ID BIGINT KEY, NAME varchar, VALUE bigint) WITH (kafka_topic='test_topic', value_format='PROTOBUF');", "CREATE TABLE TEST_TABLE (id BIGINT PRIMARY KEY, f1 VARCHAR, f2 BIGINT) WITH (kafka_topic='test_table', value_format='PROTOBUF');", "CREATE STREAM LEFT_JOIN as SELECT t.id, name, value, f1, f2 FROM test t left join test_table tt on t.id = tt.id;" ], + "post" : { + "sources" : [ { + "name" : "LEFT_JOIN", + "type" : "STREAM", + "schema" : "`T_ID` BIGINT KEY, `NAME` STRING, `VALUE` BIGINT, `F1` STRING, `F2` BIGINT", + "keyFormat" : { + "format" : "KAFKA" + }, + "valueFormat" : "PROTOBUF", + "keyFeatures" : [ ], + "valueFeatures" : [ ], + "isSource" : false + }, { + "name" : "TEST", + "type" : "STREAM", + "schema" : "`ID` BIGINT KEY, `NAME` STRING, `VALUE` BIGINT", + "keyFormat" : { + "format" : "KAFKA" + }, + "valueFormat" : "PROTOBUF", + "keyFeatures" : [ ], + "valueFeatures" : [ ], + "isSource" : false + }, { + "name" : "TEST_TABLE", + "type" : "TABLE", + "schema" : "`ID` BIGINT KEY, `F1` STRING, `F2` BIGINT", + "keyFormat" : { + "format" : "KAFKA" + }, + "valueFormat" : "PROTOBUF", + "keyFeatures" : [ ], + "valueFeatures" : [ ], + "isSource" : false + } ], + "topics" : { + "topics" : [ { + "name" : "_confluent-ksql-some.ksql.service.idquery_CSAS_LEFT_JOIN_0-KafkaTopic_Right-Reduce-changelog", + "keyFormat" : { + "format" : "KAFKA" + }, + "valueFormat" : { + "format" : "PROTOBUF", + "properties" : { + "unwrapPrimitives" : "true" + } + }, + "valueSchema" : "syntax = \"proto3\";\n\nmessage ConnectDefault1 {\n string F1 = 1;\n int64 F2 = 2;\n int32 ROWPARTITION = 3;\n int64 ROWOFFSET = 4;\n}\n" + }, { + "name" : "test_topic", + "keyFormat" : { + "format" : "KAFKA" + }, + "valueFormat" : { + "format" : "PROTOBUF", + "properties" : { + "unwrapPrimitives" : "true" + } + }, + "partitions" : 4, + "valueSchema" : "syntax = \"proto3\";\n\nmessage ConnectDefault1 {\n string NAME = 1;\n int64 VALUE = 2;\n}\n" + }, { + "name" : "LEFT_JOIN", + "keyFormat" : { + "format" : "KAFKA" + }, + "valueFormat" : { + "format" : "PROTOBUF", + "properties" : { + "unwrapPrimitives" : "true" + } + }, + "partitions" : 4, + "valueSchema" : "syntax = \"proto3\";\n\nmessage ConnectDefault1 {\n string NAME = 1;\n int64 VALUE = 2;\n string F1 = 3;\n int64 F2 = 4;\n}\n" + }, { + "name" : "test_table", + "keyFormat" : { + "format" : "KAFKA" + }, + "valueFormat" : { + "format" : "PROTOBUF", + "properties" : { + "unwrapPrimitives" : "true" + } + }, + "partitions" : 4, + "valueSchema" : "syntax = \"proto3\";\n\nmessage ConnectDefault1 {\n string F1 = 1;\n int64 F2 = 2;\n}\n" + } ], + "blackList" : ".*-repartition" + } + } + } +} \ No newline at end of file diff --git a/ksqldb-functional-tests/src/test/resources/historical_plans/joins_-_stream_table_left_join_-_PROTOBUF_-_PROTOBUF/7.3.0_1651071391893/topology b/ksqldb-functional-tests/src/test/resources/historical_plans/joins_-_stream_table_left_join_-_PROTOBUF_-_PROTOBUF/7.3.0_1651071391893/topology new file mode 100644 index 000000000000..0849f4319e4f --- /dev/null +++ b/ksqldb-functional-tests/src/test/resources/historical_plans/joins_-_stream_table_left_join_-_PROTOBUF_-_PROTOBUF/7.3.0_1651071391893/topology @@ -0,0 +1,33 @@ +Topologies: + Sub-topology: 0 + Source: KSTREAM-SOURCE-0000000006 (topics: [test_topic]) + --> KSTREAM-TRANSFORMVALUES-0000000007 + Source: KSTREAM-SOURCE-0000000001 (topics: [test_table]) + --> KTABLE-SOURCE-0000000002 + Processor: KSTREAM-TRANSFORMVALUES-0000000007 (stores: []) + --> PrependAliasLeft + <-- KSTREAM-SOURCE-0000000006 + Processor: KTABLE-SOURCE-0000000002 (stores: []) + --> KTABLE-TRANSFORMVALUES-0000000003 + <-- KSTREAM-SOURCE-0000000001 + Processor: PrependAliasLeft (stores: []) + --> Join + <-- KSTREAM-TRANSFORMVALUES-0000000007 + Processor: Join (stores: [KafkaTopic_Right-Reduce]) + --> Project + <-- PrependAliasLeft + Processor: KTABLE-TRANSFORMVALUES-0000000003 (stores: [KafkaTopic_Right-Reduce]) + --> KTABLE-TRANSFORMVALUES-0000000004 + <-- KTABLE-SOURCE-0000000002 + Processor: KTABLE-TRANSFORMVALUES-0000000004 (stores: []) + --> PrependAliasRight + <-- KTABLE-TRANSFORMVALUES-0000000003 + Processor: Project (stores: []) + --> KSTREAM-SINK-0000000011 + <-- Join + Sink: KSTREAM-SINK-0000000011 (topic: LEFT_JOIN) + <-- Project + Processor: PrependAliasRight (stores: []) + --> none + <-- KTABLE-TRANSFORMVALUES-0000000004 + diff --git a/ksqldb-functional-tests/src/test/resources/historical_plans/joins_-_stream_table_left_join_-_PROTOBUF_-_PROTOBUF_NOSR/7.3.0_1651071392646/plan.json b/ksqldb-functional-tests/src/test/resources/historical_plans/joins_-_stream_table_left_join_-_PROTOBUF_-_PROTOBUF_NOSR/7.3.0_1651071392646/plan.json new file mode 100644 index 000000000000..153e9c5bb21a --- /dev/null +++ b/ksqldb-functional-tests/src/test/resources/historical_plans/joins_-_stream_table_left_join_-_PROTOBUF_-_PROTOBUF_NOSR/7.3.0_1651071392646/plan.json @@ -0,0 +1,302 @@ +{ + "plan" : [ { + "@type" : "ksqlPlanV1", + "statementText" : "CREATE STREAM TEST (ID BIGINT KEY, NAME STRING, VALUE BIGINT) WITH (KAFKA_TOPIC='test_topic', KEY_FORMAT='KAFKA', VALUE_FORMAT='PROTOBUF_NOSR');", + "ddlCommand" : { + "@type" : "createStreamV1", + "sourceName" : "TEST", + "schema" : "`ID` BIGINT KEY, `NAME` STRING, `VALUE` BIGINT", + "timestampColumn" : null, + "topicName" : "test_topic", + "formats" : { + "keyFormat" : { + "format" : "KAFKA", + "properties" : { } + }, + "valueFormat" : { + "format" : "PROTOBUF_NOSR", + "properties" : { } + } + }, + "windowInfo" : null, + "orReplace" : false, + "isSource" : false + }, + "queryPlan" : null + }, { + "@type" : "ksqlPlanV1", + "statementText" : "CREATE TABLE TEST_TABLE (ID BIGINT PRIMARY KEY, F1 STRING, F2 BIGINT) WITH (KAFKA_TOPIC='test_table', KEY_FORMAT='KAFKA', VALUE_FORMAT='PROTOBUF_NOSR');", + "ddlCommand" : { + "@type" : "createTableV1", + "sourceName" : "TEST_TABLE", + "schema" : "`ID` BIGINT KEY, `F1` STRING, `F2` BIGINT", + "timestampColumn" : null, + "topicName" : "test_table", + "formats" : { + "keyFormat" : { + "format" : "KAFKA", + "properties" : { } + }, + "valueFormat" : { + "format" : "PROTOBUF_NOSR", + "properties" : { } + } + }, + "windowInfo" : null, + "orReplace" : false, + "isSource" : false + }, + "queryPlan" : null + }, { + "@type" : "ksqlPlanV1", + "statementText" : "CREATE STREAM LEFT_JOIN AS SELECT\n T.ID T_ID,\n T.NAME NAME,\n T.VALUE VALUE,\n TT.F1 F1,\n TT.F2 F2\nFROM TEST T\nLEFT OUTER JOIN TEST_TABLE TT ON ((T.ID = TT.ID))\nEMIT CHANGES", + "ddlCommand" : { + "@type" : "createStreamV1", + "sourceName" : "LEFT_JOIN", + "schema" : "`T_ID` BIGINT KEY, `NAME` STRING, `VALUE` BIGINT, `F1` STRING, `F2` BIGINT", + "timestampColumn" : null, + "topicName" : "LEFT_JOIN", + "formats" : { + "keyFormat" : { + "format" : "KAFKA", + "properties" : { } + }, + "valueFormat" : { + "format" : "PROTOBUF_NOSR", + "properties" : { } + } + }, + "windowInfo" : null, + "orReplace" : false, + "isSource" : false + }, + "queryPlan" : { + "sources" : [ "TEST", "TEST_TABLE" ], + "sink" : "LEFT_JOIN", + "physicalPlan" : { + "@type" : "streamSinkV1", + "properties" : { + "queryContext" : "LEFT_JOIN" + }, + "source" : { + "@type" : "streamSelectV1", + "properties" : { + "queryContext" : "Project" + }, + "source" : { + "@type" : "streamTableJoinV1", + "properties" : { + "queryContext" : "Join" + }, + "joinType" : "LEFT", + "internalFormats" : { + "keyFormat" : { + "format" : "KAFKA", + "properties" : { } + }, + "valueFormat" : { + "format" : "PROTOBUF_NOSR", + "properties" : { } + } + }, + "leftSource" : { + "@type" : "streamSelectV1", + "properties" : { + "queryContext" : "PrependAliasLeft" + }, + "source" : { + "@type" : "streamSourceV1", + "properties" : { + "queryContext" : "KafkaTopic_Left/Source" + }, + "topicName" : "test_topic", + "formats" : { + "keyFormat" : { + "format" : "KAFKA", + "properties" : { } + }, + "valueFormat" : { + "format" : "PROTOBUF_NOSR", + "properties" : { } + } + }, + "timestampColumn" : null, + "sourceSchema" : "`ID` BIGINT KEY, `NAME` STRING, `VALUE` BIGINT", + "pseudoColumnVersion" : 1 + }, + "keyColumnNames" : [ "T_ID" ], + "selectedKeys" : null, + "selectExpressions" : [ "NAME AS T_NAME", "VALUE AS T_VALUE", "ROWTIME AS T_ROWTIME", "ROWPARTITION AS T_ROWPARTITION", "ROWOFFSET AS T_ROWOFFSET", "ID AS T_ID" ] + }, + "rightSource" : { + "@type" : "tableSelectV1", + "properties" : { + "queryContext" : "PrependAliasRight" + }, + "source" : { + "@type" : "tableSourceV2", + "properties" : { + "queryContext" : "KafkaTopic_Right/Source" + }, + "topicName" : "test_table", + "formats" : { + "keyFormat" : { + "format" : "KAFKA", + "properties" : { } + }, + "valueFormat" : { + "format" : "PROTOBUF_NOSR", + "properties" : { } + } + }, + "timestampColumn" : null, + "sourceSchema" : "`ID` BIGINT KEY, `F1` STRING, `F2` BIGINT", + "pseudoColumnVersion" : 1, + "stateStoreFormats" : { + "keyFormat" : { + "format" : "KAFKA", + "properties" : { } + }, + "valueFormat" : { + "format" : "PROTOBUF_NOSR", + "properties" : { } + } + } + }, + "keyColumnNames" : [ "TT_ID" ], + "selectExpressions" : [ "F1 AS TT_F1", "F2 AS TT_F2", "ROWTIME AS TT_ROWTIME", "ROWPARTITION AS TT_ROWPARTITION", "ROWOFFSET AS TT_ROWOFFSET", "ID AS TT_ID" ], + "internalFormats" : { + "keyFormat" : { + "format" : "KAFKA", + "properties" : { } + }, + "valueFormat" : { + "format" : "PROTOBUF_NOSR", + "properties" : { } + } + } + }, + "keyColName" : "T_ID" + }, + "keyColumnNames" : [ "T_ID" ], + "selectedKeys" : null, + "selectExpressions" : [ "T_NAME AS NAME", "T_VALUE AS VALUE", "TT_F1 AS F1", "TT_F2 AS F2" ] + }, + "formats" : { + "keyFormat" : { + "format" : "KAFKA", + "properties" : { } + }, + "valueFormat" : { + "format" : "PROTOBUF_NOSR", + "properties" : { } + } + }, + "topicName" : "LEFT_JOIN", + "timestampColumn" : null + }, + "queryId" : "CSAS_LEFT_JOIN_0", + "runtimeId" : null + } + } ], + "configs" : { + "ksql.extension.dir" : "ext", + "ksql.streams.cache.max.bytes.buffering" : "0", + "metric.reporters" : "", + "ksql.query.status.running.threshold.seconds" : "300", + "ksql.connect.basic.auth.credentials.reload" : "false", + "ksql.output.topic.name.prefix" : "", + "ksql.query.pull.stream.enabled" : "true", + "ksql.query.push.v2.interpreter.enabled" : "true", + "ksql.queryanonymizer.logs_enabled" : "true", + "ksql.variable.substitution.enable" : "true", + "ksql.streams.shutdown.timeout.ms" : "300000", + "ksql.query.pull.max.allowed.offset.lag" : "9223372036854775807", + "ksql.query.pull.max.qps" : "2147483647", + "ksql.access.validator.enable" : "auto", + "ksql.streams.bootstrap.servers" : "localhost:0", + "ksql.query.pull.metrics.enabled" : "true", + "ksql.metrics.extension" : null, + "ksql.query.push.v2.alos.enabled" : "true", + "ksql.query.push.v2.max.hourly.bandwidth.megabytes" : "2147483647", + "ksql.query.pull.range.scan.enabled" : "true", + "ksql.transient.query.cleanup.service.initial.delay.seconds" : "600", + "ksql.hidden.topics" : "_confluent.*,__confluent.*,_schemas,__consumer_offsets,__transaction_state,connect-configs,connect-offsets,connect-status,connect-statuses", + "ksql.lambdas.enabled" : "true", + "ksql.source.table.materialization.enabled" : "true", + "ksql.query.pull.max.hourly.bandwidth.megabytes" : "2147483647", + "ksql.sink.window.change.log.additional.retention" : "1000000", + "ksql.query.persistent.active.limit" : "2147483647", + "ksql.persistence.wrap.single.values" : null, + "ksql.query.transient.max.bytes.buffering.total" : "-1", + "ksql.connect.basic.auth.credentials.source" : "NONE", + "ksql.schema.registry.url" : "schema_registry.url:0", + "ksql.properties.overrides.denylist" : "", + "ksql.service.id" : "some.ksql.service.id", + "ksql.query.push.v2.max.catchup.consumers" : "5", + "ksql.query.push.v2.enabled" : "false", + "ksql.transient.query.cleanup.service.enable" : "true", + "ksql.query.push.v2.metrics.enabled" : "true", + "ksql.rowpartition.rowoffset.enabled" : "true", + "ksql.streams.commit.interval.ms" : "2000", + "ksql.query.pull.table.scan.enabled" : "true", + "ksql.streams.auto.commit.interval.ms" : "0", + "ksql.streams.topology.optimization" : "all", + "ksql.endpoint.migrate.query" : "true", + "ksql.query.push.v2.registry.installed" : "false", + "ksql.streams.num.stream.threads" : "4", + "ksql.metrics.tags.custom" : "", + "ksql.query.push.v2.catchup.consumer.msg.window" : "50", + "ksql.runtime.feature.shared.enabled" : "false", + "ksql.udf.collect.metrics" : "false", + "ksql.new.query.planner.enabled" : "false", + "ksql.connect.request.headers.plugin" : null, + "ksql.security.extension.class" : null, + "ksql.transient.prefix" : "transient_", + "ksql.headers.columns.enabled" : "true", + "ksql.streams.default.deserialization.exception.handler" : "io.confluent.ksql.errors.LogMetricAndContinueExceptionHandler", + "ksql.connect.request.timeout.ms" : "5000", + "ksql.query.pull.enable.standby.reads" : "false", + "ksql.persistence.default.format.key" : "KAFKA", + "ksql.query.persistent.max.bytes.buffering.total" : "-1", + "ksql.query.error.max.queue.size" : "10", + "ksql.query.cleanup.shutdown.timeout.ms" : "30000", + "ksql.internal.topic.min.insync.replicas" : "1", + "ksql.internal.topic.replicas" : "1", + "ksql.insert.into.values.enabled" : "true", + "ksql.queryanonymizer.cluster_namespace" : null, + "ksql.create.or.replace.enabled" : "true", + "ksql.shared.runtimes.count" : "2", + "ksql.cast.strings.preserve.nulls" : "true", + "ksql.authorization.cache.max.entries" : "10000", + "ksql.pull.queries.enable" : "true", + "ksql.transient.query.cleanup.service.period.seconds" : "600", + "ksql.suppress.enabled" : "false", + "ksql.readonly.topics" : "_confluent.*,__confluent.*,_schemas,__consumer_offsets,__transaction_state,connect-configs,connect-offsets,connect-status,connect-statuses", + "ksql.connect.basic.auth.credentials.file" : "", + "ksql.authorization.cache.expiry.time.secs" : "30", + "ksql.query.retry.backoff.initial.ms" : "15000", + "ksql.query.pull.max.concurrent.requests" : "2147483647", + "ksql.streams.auto.offset.reset" : "earliest", + "ksql.connect.url" : "http://localhost:8083", + "ksql.query.push.v2.new.latest.delay.ms" : "5000", + "ksql.query.push.v2.latest.reset.age.ms" : "30000", + "ksql.streams.default.production.exception.handler" : "io.confluent.ksql.errors.ProductionExceptionHandlerUtil$LogAndFailProductionExceptionHandler", + "ksql.query.pull.interpreter.enabled" : "true", + "ksql.query.pull.limit.clause.enabled" : "true", + "ksql.query.pull.router.thread.pool.size" : "50", + "ksql.query.push.v2.continuation.tokens.enabled" : "false", + "ksql.query.retry.backoff.max.ms" : "900000", + "ksql.timestamp.throw.on.invalid" : "false", + "ksql.persistence.default.format.value" : null, + "ksql.udfs.enabled" : "true", + "ksql.udf.enable.security.manager" : "true", + "ksql.connect.worker.config" : "", + "ksql.nested.error.set.null" : "true", + "ksql.query.pull.thread.pool.size" : "50", + "ksql.persistent.prefix" : "query_", + "ksql.metastore.backup.location" : "", + "ksql.error.classifier.regex" : "", + "ksql.suppress.buffer.size.bytes" : "-1" + } +} \ No newline at end of file diff --git a/ksqldb-functional-tests/src/test/resources/historical_plans/joins_-_stream_table_left_join_-_PROTOBUF_-_PROTOBUF_NOSR/7.3.0_1651071392646/spec.json b/ksqldb-functional-tests/src/test/resources/historical_plans/joins_-_stream_table_left_join_-_PROTOBUF_-_PROTOBUF_NOSR/7.3.0_1651071392646/spec.json new file mode 100644 index 000000000000..a77ebaecb414 --- /dev/null +++ b/ksqldb-functional-tests/src/test/resources/historical_plans/joins_-_stream_table_left_join_-_PROTOBUF_-_PROTOBUF_NOSR/7.3.0_1651071392646/spec.json @@ -0,0 +1,254 @@ +{ + "version" : "7.3.0", + "timestamp" : 1651071392646, + "path" : "query-validation-tests/joins.json", + "schemas" : { + "CSAS_LEFT_JOIN_0.Join" : { + "schema" : "`T_ID` BIGINT KEY, `T_NAME` STRING, `T_VALUE` BIGINT, `T_ROWTIME` BIGINT, `T_ROWPARTITION` INTEGER, `T_ROWOFFSET` BIGINT, `T_ID` BIGINT", + "keyFormat" : { + "format" : "KAFKA" + } + }, + "CSAS_LEFT_JOIN_0.LEFT_JOIN" : { + "schema" : "`T_ID` BIGINT KEY, `NAME` STRING, `VALUE` BIGINT, `F1` STRING, `F2` BIGINT", + "keyFormat" : { + "format" : "KAFKA" + }, + "valueFormat" : { + "format" : "PROTOBUF_NOSR" + } + }, + "CSAS_LEFT_JOIN_0.KafkaTopic_Left.Source" : { + "schema" : "`ID` BIGINT KEY, `NAME` STRING, `VALUE` BIGINT", + "keyFormat" : { + "format" : "KAFKA" + }, + "valueFormat" : { + "format" : "PROTOBUF_NOSR" + } + }, + "CSAS_LEFT_JOIN_0.KafkaTopic_Right.Source.Materialized" : { + "schema" : "`ID` BIGINT KEY, `F1` STRING, `F2` BIGINT, `ROWPARTITION` INTEGER, `ROWOFFSET` BIGINT", + "keyFormat" : { + "format" : "KAFKA" + }, + "valueFormat" : { + "format" : "PROTOBUF_NOSR" + } + }, + "CSAS_LEFT_JOIN_0.PrependAliasRight" : { + "schema" : "`TT_ID` BIGINT KEY, `TT_F1` STRING, `TT_F2` BIGINT, `TT_ROWTIME` BIGINT, `TT_ROWPARTITION` INTEGER, `TT_ROWOFFSET` BIGINT, `TT_ID` BIGINT", + "keyFormat" : { + "format" : "KAFKA" + }, + "valueFormat" : { + "format" : "PROTOBUF_NOSR" + } + }, + "CSAS_LEFT_JOIN_0.Join.Left" : { + "schema" : "`T_ID` BIGINT KEY, `T_NAME` STRING, `T_VALUE` BIGINT, `T_ROWTIME` BIGINT, `T_ROWPARTITION` INTEGER, `T_ROWOFFSET` BIGINT, `T_ID` BIGINT", + "valueFormat" : { + "format" : "PROTOBUF_NOSR" + } + }, + "CSAS_LEFT_JOIN_0.KafkaTopic_Right.Source" : { + "schema" : "`ID` BIGINT KEY, `F1` STRING, `F2` BIGINT", + "keyFormat" : { + "format" : "KAFKA" + }, + "valueFormat" : { + "format" : "PROTOBUF_NOSR" + } + } + }, + "testCase" : { + "name" : "stream table left join - PROTOBUF - PROTOBUF_NOSR", + "inputs" : [ { + "topic" : "test_table", + "key" : 0, + "value" : { + "F1" : "zero", + "F2" : 0 + }, + "timestamp" : 0 + }, { + "topic" : "test_table", + "key" : 10, + "value" : { + "F1" : "100", + "F2" : 5 + }, + "timestamp" : 10000 + }, { + "topic" : "test_topic", + "key" : 0, + "value" : { + "NAME" : "blah", + "VALUE" : 50 + }, + "timestamp" : 10000 + }, { + "topic" : "test_topic", + "key" : 0, + "value" : { + "NAME" : "foo", + "VALUE" : 100 + }, + "timestamp" : 10000 + }, { + "topic" : "test_table", + "key" : 0, + "value" : { + "F1" : "a", + "F2" : 10 + }, + "timestamp" : 15000 + }, { + "topic" : "test_topic", + "key" : 0, + "value" : { + "NAME" : "bar", + "VALUE" : 99 + }, + "timestamp" : 15000 + }, { + "topic" : "test_topic", + "key" : 90, + "value" : { + "NAME" : "ninety", + "VALUE" : 90 + }, + "timestamp" : 15000 + } ], + "outputs" : [ { + "topic" : "LEFT_JOIN", + "key" : 0, + "value" : { + "NAME" : "blah", + "VALUE" : 50, + "F1" : "zero", + "F2" : 0 + }, + "timestamp" : 10000 + }, { + "topic" : "LEFT_JOIN", + "key" : 0, + "value" : { + "NAME" : "foo", + "VALUE" : 100, + "F1" : "zero", + "F2" : 0 + }, + "timestamp" : 10000 + }, { + "topic" : "LEFT_JOIN", + "key" : 0, + "value" : { + "NAME" : "bar", + "VALUE" : 99, + "F1" : "a", + "F2" : 10 + }, + "timestamp" : 15000 + }, { + "topic" : "LEFT_JOIN", + "key" : 90, + "value" : { + "NAME" : "ninety", + "VALUE" : 90, + "F1" : "", + "F2" : 0 + }, + "timestamp" : 15000 + } ], + "topics" : [ { + "name" : "test_topic", + "replicas" : 1, + "numPartitions" : 4 + }, { + "name" : "LEFT_JOIN", + "replicas" : 1, + "numPartitions" : 4 + }, { + "name" : "test_table", + "replicas" : 1, + "numPartitions" : 4 + } ], + "statements" : [ "CREATE STREAM TEST (ID BIGINT KEY, NAME varchar, VALUE bigint) WITH (kafka_topic='test_topic', value_format='PROTOBUF_NOSR');", "CREATE TABLE TEST_TABLE (id BIGINT PRIMARY KEY, f1 VARCHAR, f2 BIGINT) WITH (kafka_topic='test_table', value_format='PROTOBUF_NOSR');", "CREATE STREAM LEFT_JOIN as SELECT t.id, name, value, f1, f2 FROM test t left join test_table tt on t.id = tt.id;" ], + "post" : { + "sources" : [ { + "name" : "LEFT_JOIN", + "type" : "STREAM", + "schema" : "`T_ID` BIGINT KEY, `NAME` STRING, `VALUE` BIGINT, `F1` STRING, `F2` BIGINT", + "keyFormat" : { + "format" : "KAFKA" + }, + "valueFormat" : "PROTOBUF_NOSR", + "keyFeatures" : [ ], + "valueFeatures" : [ ], + "isSource" : false + }, { + "name" : "TEST", + "type" : "STREAM", + "schema" : "`ID` BIGINT KEY, `NAME` STRING, `VALUE` BIGINT", + "keyFormat" : { + "format" : "KAFKA" + }, + "valueFormat" : "PROTOBUF_NOSR", + "keyFeatures" : [ ], + "valueFeatures" : [ ], + "isSource" : false + }, { + "name" : "TEST_TABLE", + "type" : "TABLE", + "schema" : "`ID` BIGINT KEY, `F1` STRING, `F2` BIGINT", + "keyFormat" : { + "format" : "KAFKA" + }, + "valueFormat" : "PROTOBUF_NOSR", + "keyFeatures" : [ ], + "valueFeatures" : [ ], + "isSource" : false + } ], + "topics" : { + "topics" : [ { + "name" : "_confluent-ksql-some.ksql.service.idquery_CSAS_LEFT_JOIN_0-KafkaTopic_Right-Reduce-changelog", + "keyFormat" : { + "format" : "KAFKA" + }, + "valueFormat" : { + "format" : "PROTOBUF_NOSR" + } + }, { + "name" : "test_topic", + "keyFormat" : { + "format" : "KAFKA" + }, + "valueFormat" : { + "format" : "PROTOBUF_NOSR" + }, + "partitions" : 4 + }, { + "name" : "LEFT_JOIN", + "keyFormat" : { + "format" : "KAFKA" + }, + "valueFormat" : { + "format" : "PROTOBUF_NOSR" + }, + "partitions" : 4 + }, { + "name" : "test_table", + "keyFormat" : { + "format" : "KAFKA" + }, + "valueFormat" : { + "format" : "PROTOBUF_NOSR" + }, + "partitions" : 4 + } ], + "blackList" : ".*-repartition" + } + } + } +} \ No newline at end of file diff --git a/ksqldb-functional-tests/src/test/resources/historical_plans/joins_-_stream_table_left_join_-_PROTOBUF_-_PROTOBUF_NOSR/7.3.0_1651071392646/topology b/ksqldb-functional-tests/src/test/resources/historical_plans/joins_-_stream_table_left_join_-_PROTOBUF_-_PROTOBUF_NOSR/7.3.0_1651071392646/topology new file mode 100644 index 000000000000..0849f4319e4f --- /dev/null +++ b/ksqldb-functional-tests/src/test/resources/historical_plans/joins_-_stream_table_left_join_-_PROTOBUF_-_PROTOBUF_NOSR/7.3.0_1651071392646/topology @@ -0,0 +1,33 @@ +Topologies: + Sub-topology: 0 + Source: KSTREAM-SOURCE-0000000006 (topics: [test_topic]) + --> KSTREAM-TRANSFORMVALUES-0000000007 + Source: KSTREAM-SOURCE-0000000001 (topics: [test_table]) + --> KTABLE-SOURCE-0000000002 + Processor: KSTREAM-TRANSFORMVALUES-0000000007 (stores: []) + --> PrependAliasLeft + <-- KSTREAM-SOURCE-0000000006 + Processor: KTABLE-SOURCE-0000000002 (stores: []) + --> KTABLE-TRANSFORMVALUES-0000000003 + <-- KSTREAM-SOURCE-0000000001 + Processor: PrependAliasLeft (stores: []) + --> Join + <-- KSTREAM-TRANSFORMVALUES-0000000007 + Processor: Join (stores: [KafkaTopic_Right-Reduce]) + --> Project + <-- PrependAliasLeft + Processor: KTABLE-TRANSFORMVALUES-0000000003 (stores: [KafkaTopic_Right-Reduce]) + --> KTABLE-TRANSFORMVALUES-0000000004 + <-- KTABLE-SOURCE-0000000002 + Processor: KTABLE-TRANSFORMVALUES-0000000004 (stores: []) + --> PrependAliasRight + <-- KTABLE-TRANSFORMVALUES-0000000003 + Processor: Project (stores: []) + --> KSTREAM-SINK-0000000011 + <-- Join + Sink: KSTREAM-SINK-0000000011 (topic: LEFT_JOIN) + <-- Project + Processor: PrependAliasRight (stores: []) + --> none + <-- KTABLE-TRANSFORMVALUES-0000000004 + diff --git a/ksqldb-functional-tests/src/test/resources/historical_plans/joins_-_table_table_inner_join_-_PROTOBUF_NOSR/7.3.0_1651071378390/plan.json b/ksqldb-functional-tests/src/test/resources/historical_plans/joins_-_table_table_inner_join_-_PROTOBUF_NOSR/7.3.0_1651071378390/plan.json new file mode 100644 index 000000000000..eb2098e9eb6f --- /dev/null +++ b/ksqldb-functional-tests/src/test/resources/historical_plans/joins_-_table_table_inner_join_-_PROTOBUF_NOSR/7.3.0_1651071378390/plan.json @@ -0,0 +1,320 @@ +{ + "plan" : [ { + "@type" : "ksqlPlanV1", + "statementText" : "CREATE TABLE TEST (ID BIGINT PRIMARY KEY, NAME STRING, VALUE BIGINT) WITH (KAFKA_TOPIC='left_topic', KEY_FORMAT='KAFKA', VALUE_FORMAT='PROTOBUF_NOSR');", + "ddlCommand" : { + "@type" : "createTableV1", + "sourceName" : "TEST", + "schema" : "`ID` BIGINT KEY, `NAME` STRING, `VALUE` BIGINT", + "timestampColumn" : null, + "topicName" : "left_topic", + "formats" : { + "keyFormat" : { + "format" : "KAFKA", + "properties" : { } + }, + "valueFormat" : { + "format" : "PROTOBUF_NOSR", + "properties" : { } + } + }, + "windowInfo" : null, + "orReplace" : false, + "isSource" : false + }, + "queryPlan" : null + }, { + "@type" : "ksqlPlanV1", + "statementText" : "CREATE TABLE TEST_TABLE (ID BIGINT PRIMARY KEY, F1 STRING, F2 BIGINT) WITH (KAFKA_TOPIC='right_topic', KEY_FORMAT='KAFKA', VALUE_FORMAT='PROTOBUF_NOSR');", + "ddlCommand" : { + "@type" : "createTableV1", + "sourceName" : "TEST_TABLE", + "schema" : "`ID` BIGINT KEY, `F1` STRING, `F2` BIGINT", + "timestampColumn" : null, + "topicName" : "right_topic", + "formats" : { + "keyFormat" : { + "format" : "KAFKA", + "properties" : { } + }, + "valueFormat" : { + "format" : "PROTOBUF_NOSR", + "properties" : { } + } + }, + "windowInfo" : null, + "orReplace" : false, + "isSource" : false + }, + "queryPlan" : null + }, { + "@type" : "ksqlPlanV1", + "statementText" : "CREATE TABLE INNER_JOIN AS SELECT\n T.ID T_ID,\n T.NAME NAME,\n T.VALUE VALUE,\n TT.F1 F1,\n TT.F2 F2\nFROM TEST T\nINNER JOIN TEST_TABLE TT ON ((T.ID = TT.ID))\nEMIT CHANGES", + "ddlCommand" : { + "@type" : "createTableV1", + "sourceName" : "INNER_JOIN", + "schema" : "`T_ID` BIGINT KEY, `NAME` STRING, `VALUE` BIGINT, `F1` STRING, `F2` BIGINT", + "timestampColumn" : null, + "topicName" : "INNER_JOIN", + "formats" : { + "keyFormat" : { + "format" : "KAFKA", + "properties" : { } + }, + "valueFormat" : { + "format" : "PROTOBUF_NOSR", + "properties" : { } + } + }, + "windowInfo" : null, + "orReplace" : false, + "isSource" : false + }, + "queryPlan" : { + "sources" : [ "TEST", "TEST_TABLE" ], + "sink" : "INNER_JOIN", + "physicalPlan" : { + "@type" : "tableSinkV1", + "properties" : { + "queryContext" : "INNER_JOIN" + }, + "source" : { + "@type" : "tableSelectV1", + "properties" : { + "queryContext" : "Project" + }, + "source" : { + "@type" : "tableTableJoinV1", + "properties" : { + "queryContext" : "Join" + }, + "joinType" : "INNER", + "leftSource" : { + "@type" : "tableSelectV1", + "properties" : { + "queryContext" : "PrependAliasLeft" + }, + "source" : { + "@type" : "tableSourceV2", + "properties" : { + "queryContext" : "KafkaTopic_Left/Source" + }, + "topicName" : "left_topic", + "formats" : { + "keyFormat" : { + "format" : "KAFKA", + "properties" : { } + }, + "valueFormat" : { + "format" : "PROTOBUF_NOSR", + "properties" : { } + } + }, + "timestampColumn" : null, + "sourceSchema" : "`ID` BIGINT KEY, `NAME` STRING, `VALUE` BIGINT", + "pseudoColumnVersion" : 1, + "stateStoreFormats" : { + "keyFormat" : { + "format" : "KAFKA", + "properties" : { } + }, + "valueFormat" : { + "format" : "PROTOBUF_NOSR", + "properties" : { } + } + } + }, + "keyColumnNames" : [ "T_ID" ], + "selectExpressions" : [ "NAME AS T_NAME", "VALUE AS T_VALUE", "ROWTIME AS T_ROWTIME", "ROWPARTITION AS T_ROWPARTITION", "ROWOFFSET AS T_ROWOFFSET", "ID AS T_ID" ], + "internalFormats" : { + "keyFormat" : { + "format" : "KAFKA", + "properties" : { } + }, + "valueFormat" : { + "format" : "PROTOBUF_NOSR", + "properties" : { } + } + } + }, + "rightSource" : { + "@type" : "tableSelectV1", + "properties" : { + "queryContext" : "PrependAliasRight" + }, + "source" : { + "@type" : "tableSourceV2", + "properties" : { + "queryContext" : "KafkaTopic_Right/Source" + }, + "topicName" : "right_topic", + "formats" : { + "keyFormat" : { + "format" : "KAFKA", + "properties" : { } + }, + "valueFormat" : { + "format" : "PROTOBUF_NOSR", + "properties" : { } + } + }, + "timestampColumn" : null, + "sourceSchema" : "`ID` BIGINT KEY, `F1` STRING, `F2` BIGINT", + "pseudoColumnVersion" : 1, + "stateStoreFormats" : { + "keyFormat" : { + "format" : "KAFKA", + "properties" : { } + }, + "valueFormat" : { + "format" : "PROTOBUF_NOSR", + "properties" : { } + } + } + }, + "keyColumnNames" : [ "TT_ID" ], + "selectExpressions" : [ "F1 AS TT_F1", "F2 AS TT_F2", "ROWTIME AS TT_ROWTIME", "ROWPARTITION AS TT_ROWPARTITION", "ROWOFFSET AS TT_ROWOFFSET", "ID AS TT_ID" ], + "internalFormats" : { + "keyFormat" : { + "format" : "KAFKA", + "properties" : { } + }, + "valueFormat" : { + "format" : "PROTOBUF_NOSR", + "properties" : { } + } + } + }, + "keyColName" : "T_ID" + }, + "keyColumnNames" : [ "T_ID" ], + "selectExpressions" : [ "T_NAME AS NAME", "T_VALUE AS VALUE", "TT_F1 AS F1", "TT_F2 AS F2" ], + "internalFormats" : { + "keyFormat" : { + "format" : "KAFKA", + "properties" : { } + }, + "valueFormat" : { + "format" : "PROTOBUF_NOSR", + "properties" : { } + } + } + }, + "formats" : { + "keyFormat" : { + "format" : "KAFKA", + "properties" : { } + }, + "valueFormat" : { + "format" : "PROTOBUF_NOSR", + "properties" : { } + } + }, + "topicName" : "INNER_JOIN", + "timestampColumn" : null + }, + "queryId" : "CTAS_INNER_JOIN_0", + "runtimeId" : null + } + } ], + "configs" : { + "ksql.extension.dir" : "ext", + "ksql.streams.cache.max.bytes.buffering" : "0", + "metric.reporters" : "", + "ksql.query.status.running.threshold.seconds" : "300", + "ksql.connect.basic.auth.credentials.reload" : "false", + "ksql.output.topic.name.prefix" : "", + "ksql.query.pull.stream.enabled" : "true", + "ksql.query.push.v2.interpreter.enabled" : "true", + "ksql.queryanonymizer.logs_enabled" : "true", + "ksql.variable.substitution.enable" : "true", + "ksql.streams.shutdown.timeout.ms" : "300000", + "ksql.query.pull.max.allowed.offset.lag" : "9223372036854775807", + "ksql.query.pull.max.qps" : "2147483647", + "ksql.access.validator.enable" : "auto", + "ksql.streams.bootstrap.servers" : "localhost:0", + "ksql.query.pull.metrics.enabled" : "true", + "ksql.metrics.extension" : null, + "ksql.query.push.v2.alos.enabled" : "true", + "ksql.query.push.v2.max.hourly.bandwidth.megabytes" : "2147483647", + "ksql.query.pull.range.scan.enabled" : "true", + "ksql.transient.query.cleanup.service.initial.delay.seconds" : "600", + "ksql.hidden.topics" : "_confluent.*,__confluent.*,_schemas,__consumer_offsets,__transaction_state,connect-configs,connect-offsets,connect-status,connect-statuses", + "ksql.lambdas.enabled" : "true", + "ksql.source.table.materialization.enabled" : "true", + "ksql.query.pull.max.hourly.bandwidth.megabytes" : "2147483647", + "ksql.sink.window.change.log.additional.retention" : "1000000", + "ksql.query.persistent.active.limit" : "2147483647", + "ksql.persistence.wrap.single.values" : null, + "ksql.query.transient.max.bytes.buffering.total" : "-1", + "ksql.connect.basic.auth.credentials.source" : "NONE", + "ksql.schema.registry.url" : "schema_registry.url:0", + "ksql.properties.overrides.denylist" : "", + "ksql.service.id" : "some.ksql.service.id", + "ksql.query.push.v2.max.catchup.consumers" : "5", + "ksql.query.push.v2.enabled" : "false", + "ksql.transient.query.cleanup.service.enable" : "true", + "ksql.query.push.v2.metrics.enabled" : "true", + "ksql.rowpartition.rowoffset.enabled" : "true", + "ksql.streams.commit.interval.ms" : "2000", + "ksql.query.pull.table.scan.enabled" : "true", + "ksql.streams.auto.commit.interval.ms" : "0", + "ksql.streams.topology.optimization" : "all", + "ksql.endpoint.migrate.query" : "true", + "ksql.query.push.v2.registry.installed" : "false", + "ksql.streams.num.stream.threads" : "4", + "ksql.metrics.tags.custom" : "", + "ksql.query.push.v2.catchup.consumer.msg.window" : "50", + "ksql.runtime.feature.shared.enabled" : "false", + "ksql.udf.collect.metrics" : "false", + "ksql.new.query.planner.enabled" : "false", + "ksql.connect.request.headers.plugin" : null, + "ksql.security.extension.class" : null, + "ksql.transient.prefix" : "transient_", + "ksql.headers.columns.enabled" : "true", + "ksql.streams.default.deserialization.exception.handler" : "io.confluent.ksql.errors.LogMetricAndContinueExceptionHandler", + "ksql.connect.request.timeout.ms" : "5000", + "ksql.query.pull.enable.standby.reads" : "false", + "ksql.persistence.default.format.key" : "KAFKA", + "ksql.query.persistent.max.bytes.buffering.total" : "-1", + "ksql.query.error.max.queue.size" : "10", + "ksql.query.cleanup.shutdown.timeout.ms" : "30000", + "ksql.internal.topic.min.insync.replicas" : "1", + "ksql.internal.topic.replicas" : "1", + "ksql.insert.into.values.enabled" : "true", + "ksql.queryanonymizer.cluster_namespace" : null, + "ksql.create.or.replace.enabled" : "true", + "ksql.shared.runtimes.count" : "2", + "ksql.cast.strings.preserve.nulls" : "true", + "ksql.authorization.cache.max.entries" : "10000", + "ksql.pull.queries.enable" : "true", + "ksql.transient.query.cleanup.service.period.seconds" : "600", + "ksql.suppress.enabled" : "false", + "ksql.readonly.topics" : "_confluent.*,__confluent.*,_schemas,__consumer_offsets,__transaction_state,connect-configs,connect-offsets,connect-status,connect-statuses", + "ksql.connect.basic.auth.credentials.file" : "", + "ksql.authorization.cache.expiry.time.secs" : "30", + "ksql.query.retry.backoff.initial.ms" : "15000", + "ksql.query.pull.max.concurrent.requests" : "2147483647", + "ksql.streams.auto.offset.reset" : "earliest", + "ksql.connect.url" : "http://localhost:8083", + "ksql.query.push.v2.new.latest.delay.ms" : "5000", + "ksql.query.push.v2.latest.reset.age.ms" : "30000", + "ksql.streams.default.production.exception.handler" : "io.confluent.ksql.errors.ProductionExceptionHandlerUtil$LogAndFailProductionExceptionHandler", + "ksql.query.pull.interpreter.enabled" : "true", + "ksql.query.pull.limit.clause.enabled" : "true", + "ksql.query.pull.router.thread.pool.size" : "50", + "ksql.query.push.v2.continuation.tokens.enabled" : "false", + "ksql.query.retry.backoff.max.ms" : "900000", + "ksql.timestamp.throw.on.invalid" : "false", + "ksql.persistence.default.format.value" : null, + "ksql.udfs.enabled" : "true", + "ksql.udf.enable.security.manager" : "true", + "ksql.connect.worker.config" : "", + "ksql.nested.error.set.null" : "true", + "ksql.query.pull.thread.pool.size" : "50", + "ksql.persistent.prefix" : "query_", + "ksql.metastore.backup.location" : "", + "ksql.error.classifier.regex" : "", + "ksql.suppress.buffer.size.bytes" : "-1" + } +} \ No newline at end of file diff --git a/ksqldb-functional-tests/src/test/resources/historical_plans/joins_-_table_table_inner_join_-_PROTOBUF_NOSR/7.3.0_1651071378390/spec.json b/ksqldb-functional-tests/src/test/resources/historical_plans/joins_-_table_table_inner_join_-_PROTOBUF_NOSR/7.3.0_1651071378390/spec.json new file mode 100644 index 000000000000..680e409f4132 --- /dev/null +++ b/ksqldb-functional-tests/src/test/resources/historical_plans/joins_-_table_table_inner_join_-_PROTOBUF_NOSR/7.3.0_1651071378390/spec.json @@ -0,0 +1,293 @@ +{ + "version" : "7.3.0", + "timestamp" : 1651071378390, + "path" : "query-validation-tests/joins.json", + "schemas" : { + "CTAS_INNER_JOIN_0.INNER_JOIN" : { + "schema" : "`T_ID` BIGINT KEY, `NAME` STRING, `VALUE` BIGINT, `F1` STRING, `F2` BIGINT", + "keyFormat" : { + "format" : "KAFKA" + }, + "valueFormat" : { + "format" : "PROTOBUF_NOSR" + } + }, + "CTAS_INNER_JOIN_0.KafkaTopic_Right.Source" : { + "schema" : "`ID` BIGINT KEY, `F1` STRING, `F2` BIGINT", + "keyFormat" : { + "format" : "KAFKA" + }, + "valueFormat" : { + "format" : "PROTOBUF_NOSR" + } + }, + "CTAS_INNER_JOIN_0.KafkaTopic_Right.Source.Materialized" : { + "schema" : "`ID` BIGINT KEY, `F1` STRING, `F2` BIGINT, `ROWPARTITION` INTEGER, `ROWOFFSET` BIGINT", + "keyFormat" : { + "format" : "KAFKA" + }, + "valueFormat" : { + "format" : "PROTOBUF_NOSR" + } + }, + "CTAS_INNER_JOIN_0.KafkaTopic_Left.Source" : { + "schema" : "`ID` BIGINT KEY, `NAME` STRING, `VALUE` BIGINT", + "keyFormat" : { + "format" : "KAFKA" + }, + "valueFormat" : { + "format" : "PROTOBUF_NOSR" + } + }, + "CTAS_INNER_JOIN_0.PrependAliasRight" : { + "schema" : "`TT_ID` BIGINT KEY, `TT_F1` STRING, `TT_F2` BIGINT, `TT_ROWTIME` BIGINT, `TT_ROWPARTITION` INTEGER, `TT_ROWOFFSET` BIGINT, `TT_ID` BIGINT", + "keyFormat" : { + "format" : "KAFKA" + }, + "valueFormat" : { + "format" : "PROTOBUF_NOSR" + } + }, + "CTAS_INNER_JOIN_0.PrependAliasLeft" : { + "schema" : "`T_ID` BIGINT KEY, `T_NAME` STRING, `T_VALUE` BIGINT, `T_ROWTIME` BIGINT, `T_ROWPARTITION` INTEGER, `T_ROWOFFSET` BIGINT, `T_ID` BIGINT", + "keyFormat" : { + "format" : "KAFKA" + }, + "valueFormat" : { + "format" : "PROTOBUF_NOSR" + } + }, + "CTAS_INNER_JOIN_0.Project" : { + "schema" : "`T_ID` BIGINT KEY, `NAME` STRING, `VALUE` BIGINT, `F1` STRING, `F2` BIGINT", + "keyFormat" : { + "format" : "KAFKA" + }, + "valueFormat" : { + "format" : "PROTOBUF_NOSR" + } + }, + "CTAS_INNER_JOIN_0.KafkaTopic_Left.Source.Materialized" : { + "schema" : "`ID` BIGINT KEY, `NAME` STRING, `VALUE` BIGINT, `ROWPARTITION` INTEGER, `ROWOFFSET` BIGINT", + "keyFormat" : { + "format" : "KAFKA" + }, + "valueFormat" : { + "format" : "PROTOBUF_NOSR" + } + } + }, + "testCase" : { + "name" : "table table inner join - PROTOBUF_NOSR", + "inputs" : [ { + "topic" : "left_topic", + "key" : 0, + "value" : { + "NAME" : "zero", + "VALUE" : 0 + }, + "timestamp" : 0 + }, { + "topic" : "right_topic", + "key" : 0, + "value" : { + "F1" : "blah", + "F2" : 50 + }, + "timestamp" : 10000 + }, { + "topic" : "left_topic", + "key" : 10, + "value" : { + "NAME" : "100", + "VALUE" : 5 + }, + "timestamp" : 11000 + }, { + "topic" : "left_topic", + "key" : 0, + "value" : { + "NAME" : "foo", + "VALUE" : 100 + }, + "timestamp" : 13000 + }, { + "topic" : "right_topic", + "key" : 0, + "value" : { + "F1" : "a", + "F2" : 10 + }, + "timestamp" : 15000 + }, { + "topic" : "right_topic", + "key" : 15, + "value" : { + "F1" : "c", + "F2" : 20 + }, + "timestamp" : 15500 + }, { + "topic" : "left_topic", + "key" : 0, + "value" : { + "NAME" : "bar", + "VALUE" : 99 + }, + "timestamp" : 16000 + }, { + "topic" : "left_topic", + "key" : 90, + "value" : { + "NAME" : "ninety", + "VALUE" : 90 + }, + "timestamp" : 17000 + } ], + "outputs" : [ { + "topic" : "INNER_JOIN", + "key" : 0, + "value" : { + "NAME" : "zero", + "VALUE" : 0, + "F1" : "blah", + "F2" : 50 + }, + "timestamp" : 10000 + }, { + "topic" : "INNER_JOIN", + "key" : 0, + "value" : { + "NAME" : "foo", + "VALUE" : 100, + "F1" : "blah", + "F2" : 50 + }, + "timestamp" : 13000 + }, { + "topic" : "INNER_JOIN", + "key" : 0, + "value" : { + "NAME" : "foo", + "VALUE" : 100, + "F1" : "a", + "F2" : 10 + }, + "timestamp" : 15000 + }, { + "topic" : "INNER_JOIN", + "key" : 0, + "value" : { + "NAME" : "bar", + "VALUE" : 99, + "F1" : "a", + "F2" : 10 + }, + "timestamp" : 16000 + } ], + "topics" : [ { + "name" : "right_topic", + "replicas" : 1, + "numPartitions" : 4 + }, { + "name" : "INNER_JOIN", + "replicas" : 1, + "numPartitions" : 4 + }, { + "name" : "left_topic", + "replicas" : 1, + "numPartitions" : 4 + } ], + "statements" : [ "CREATE TABLE TEST (id BIGINT PRIMARY KEY, name VARCHAR, value BIGINT) WITH (kafka_topic='left_topic', value_format='PROTOBUF_NOSR');", "CREATE TABLE TEST_TABLE (id BIGINT PRIMARY KEY, f1 VARCHAR, f2 BIGINT) WITH (kafka_topic='right_topic', value_format='PROTOBUF_NOSR');", "CREATE TABLE INNER_JOIN as SELECT t.id, name, value, f1, f2 FROM test t join TEST_TABLE tt on t.id = tt.id;" ], + "post" : { + "sources" : [ { + "name" : "INNER_JOIN", + "type" : "TABLE", + "schema" : "`T_ID` BIGINT KEY, `NAME` STRING, `VALUE` BIGINT, `F1` STRING, `F2` BIGINT", + "keyFormat" : { + "format" : "KAFKA" + }, + "valueFormat" : "PROTOBUF_NOSR", + "keyFeatures" : [ ], + "valueFeatures" : [ ], + "isSource" : false + }, { + "name" : "TEST", + "type" : "TABLE", + "schema" : "`ID` BIGINT KEY, `NAME` STRING, `VALUE` BIGINT", + "keyFormat" : { + "format" : "KAFKA" + }, + "valueFormat" : "PROTOBUF_NOSR", + "keyFeatures" : [ ], + "valueFeatures" : [ ], + "isSource" : false + }, { + "name" : "TEST_TABLE", + "type" : "TABLE", + "schema" : "`ID` BIGINT KEY, `F1` STRING, `F2` BIGINT", + "keyFormat" : { + "format" : "KAFKA" + }, + "valueFormat" : "PROTOBUF_NOSR", + "keyFeatures" : [ ], + "valueFeatures" : [ ], + "isSource" : false + } ], + "topics" : { + "topics" : [ { + "name" : "_confluent-ksql-some.ksql.service.idquery_CTAS_INNER_JOIN_0-KafkaTopic_Right-Reduce-changelog", + "keyFormat" : { + "format" : "KAFKA" + }, + "valueFormat" : { + "format" : "PROTOBUF_NOSR" + } + }, { + "name" : "_confluent-ksql-some.ksql.service.idquery_CTAS_INNER_JOIN_0-KafkaTopic_Left-Reduce-changelog", + "keyFormat" : { + "format" : "KAFKA" + }, + "valueFormat" : { + "format" : "PROTOBUF_NOSR" + } + }, { + "name" : "INNER_JOIN", + "keyFormat" : { + "format" : "KAFKA" + }, + "valueFormat" : { + "format" : "PROTOBUF_NOSR" + }, + "partitions" : 4 + }, { + "name" : "_confluent-ksql-some.ksql.service.idquery_CTAS_INNER_JOIN_0-Project-Last-changelog", + "keyFormat" : { + "format" : "KAFKA" + }, + "valueFormat" : { + "format" : "PROTOBUF_NOSR" + } + }, { + "name" : "left_topic", + "keyFormat" : { + "format" : "KAFKA" + }, + "valueFormat" : { + "format" : "PROTOBUF_NOSR" + }, + "partitions" : 4 + }, { + "name" : "right_topic", + "keyFormat" : { + "format" : "KAFKA" + }, + "valueFormat" : { + "format" : "PROTOBUF_NOSR" + }, + "partitions" : 4 + } ], + "blackList" : ".*-repartition" + } + } + } +} \ No newline at end of file diff --git a/ksqldb-functional-tests/src/test/resources/historical_plans/joins_-_table_table_inner_join_-_PROTOBUF_NOSR/7.3.0_1651071378390/topology b/ksqldb-functional-tests/src/test/resources/historical_plans/joins_-_table_table_inner_join_-_PROTOBUF_NOSR/7.3.0_1651071378390/topology new file mode 100644 index 000000000000..d41019eca5c9 --- /dev/null +++ b/ksqldb-functional-tests/src/test/resources/historical_plans/joins_-_table_table_inner_join_-_PROTOBUF_NOSR/7.3.0_1651071378390/topology @@ -0,0 +1,48 @@ +Topologies: + Sub-topology: 0 + Source: KSTREAM-SOURCE-0000000001 (topics: [left_topic]) + --> KTABLE-SOURCE-0000000002 + Source: KSTREAM-SOURCE-0000000007 (topics: [right_topic]) + --> KTABLE-SOURCE-0000000008 + Processor: KTABLE-SOURCE-0000000002 (stores: []) + --> KTABLE-TRANSFORMVALUES-0000000003 + <-- KSTREAM-SOURCE-0000000001 + Processor: KTABLE-SOURCE-0000000008 (stores: []) + --> KTABLE-TRANSFORMVALUES-0000000009 + <-- KSTREAM-SOURCE-0000000007 + Processor: KTABLE-TRANSFORMVALUES-0000000003 (stores: [KafkaTopic_Left-Reduce]) + --> KTABLE-TRANSFORMVALUES-0000000004 + <-- KTABLE-SOURCE-0000000002 + Processor: KTABLE-TRANSFORMVALUES-0000000009 (stores: [KafkaTopic_Right-Reduce]) + --> KTABLE-TRANSFORMVALUES-0000000010 + <-- KTABLE-SOURCE-0000000008 + Processor: KTABLE-TRANSFORMVALUES-0000000004 (stores: []) + --> PrependAliasLeft + <-- KTABLE-TRANSFORMVALUES-0000000003 + Processor: KTABLE-TRANSFORMVALUES-0000000010 (stores: []) + --> PrependAliasRight + <-- KTABLE-TRANSFORMVALUES-0000000009 + Processor: PrependAliasLeft (stores: []) + --> KTABLE-JOINTHIS-0000000013 + <-- KTABLE-TRANSFORMVALUES-0000000004 + Processor: PrependAliasRight (stores: []) + --> KTABLE-JOINOTHER-0000000014 + <-- KTABLE-TRANSFORMVALUES-0000000010 + Processor: KTABLE-JOINOTHER-0000000014 (stores: [KafkaTopic_Left-Reduce]) + --> KTABLE-MERGE-0000000012 + <-- PrependAliasRight + Processor: KTABLE-JOINTHIS-0000000013 (stores: [KafkaTopic_Right-Reduce]) + --> KTABLE-MERGE-0000000012 + <-- PrependAliasLeft + Processor: KTABLE-MERGE-0000000012 (stores: []) + --> KTABLE-TRANSFORMVALUES-0000000015 + <-- KTABLE-JOINTHIS-0000000013, KTABLE-JOINOTHER-0000000014 + Processor: KTABLE-TRANSFORMVALUES-0000000015 (stores: [Project-Last]) + --> KTABLE-TOSTREAM-0000000016 + <-- KTABLE-MERGE-0000000012 + Processor: KTABLE-TOSTREAM-0000000016 (stores: []) + --> KSTREAM-SINK-0000000017 + <-- KTABLE-TRANSFORMVALUES-0000000015 + Sink: KSTREAM-SINK-0000000017 (topic: INNER_JOIN) + <-- KTABLE-TOSTREAM-0000000016 + diff --git a/ksqldb-functional-tests/src/test/resources/historical_plans/joins_-_table_table_left_join_-_PROTOBUF_-_PROTOBUF/7.3.0_1651071355879/plan.json b/ksqldb-functional-tests/src/test/resources/historical_plans/joins_-_table_table_left_join_-_PROTOBUF_-_PROTOBUF/7.3.0_1651071355879/plan.json new file mode 100644 index 000000000000..3ca7fef743ab --- /dev/null +++ b/ksqldb-functional-tests/src/test/resources/historical_plans/joins_-_table_table_left_join_-_PROTOBUF_-_PROTOBUF/7.3.0_1651071355879/plan.json @@ -0,0 +1,342 @@ +{ + "plan" : [ { + "@type" : "ksqlPlanV1", + "statementText" : "CREATE TABLE TEST (ID BIGINT PRIMARY KEY, NAME STRING, VALUE BIGINT) WITH (KAFKA_TOPIC='left_topic', KEY_FORMAT='KAFKA', VALUE_FORMAT='PROTOBUF');", + "ddlCommand" : { + "@type" : "createTableV1", + "sourceName" : "TEST", + "schema" : "`ID` BIGINT KEY, `NAME` STRING, `VALUE` BIGINT", + "timestampColumn" : null, + "topicName" : "left_topic", + "formats" : { + "keyFormat" : { + "format" : "KAFKA", + "properties" : { } + }, + "valueFormat" : { + "format" : "PROTOBUF", + "properties" : { + "unwrapPrimitives" : "true" + } + } + }, + "windowInfo" : null, + "orReplace" : false, + "isSource" : false + }, + "queryPlan" : null + }, { + "@type" : "ksqlPlanV1", + "statementText" : "CREATE TABLE TEST_TABLE (ID BIGINT PRIMARY KEY, F1 STRING, F2 BIGINT) WITH (KAFKA_TOPIC='right_topic', KEY_FORMAT='KAFKA', VALUE_FORMAT='PROTOBUF');", + "ddlCommand" : { + "@type" : "createTableV1", + "sourceName" : "TEST_TABLE", + "schema" : "`ID` BIGINT KEY, `F1` STRING, `F2` BIGINT", + "timestampColumn" : null, + "topicName" : "right_topic", + "formats" : { + "keyFormat" : { + "format" : "KAFKA", + "properties" : { } + }, + "valueFormat" : { + "format" : "PROTOBUF", + "properties" : { + "unwrapPrimitives" : "true" + } + } + }, + "windowInfo" : null, + "orReplace" : false, + "isSource" : false + }, + "queryPlan" : null + }, { + "@type" : "ksqlPlanV1", + "statementText" : "CREATE TABLE OUTPUT AS SELECT\n T.ID T_ID,\n T.NAME NAME,\n T.VALUE VALUE,\n TT.F1 F1,\n TT.F2 F2\nFROM TEST T\nLEFT OUTER JOIN TEST_TABLE TT ON ((T.ID = TT.ID))\nEMIT CHANGES", + "ddlCommand" : { + "@type" : "createTableV1", + "sourceName" : "OUTPUT", + "schema" : "`T_ID` BIGINT KEY, `NAME` STRING, `VALUE` BIGINT, `F1` STRING, `F2` BIGINT", + "timestampColumn" : null, + "topicName" : "OUTPUT", + "formats" : { + "keyFormat" : { + "format" : "KAFKA", + "properties" : { } + }, + "valueFormat" : { + "format" : "PROTOBUF", + "properties" : { + "unwrapPrimitives" : "true" + } + } + }, + "windowInfo" : null, + "orReplace" : false, + "isSource" : false + }, + "queryPlan" : { + "sources" : [ "TEST", "TEST_TABLE" ], + "sink" : "OUTPUT", + "physicalPlan" : { + "@type" : "tableSinkV1", + "properties" : { + "queryContext" : "OUTPUT" + }, + "source" : { + "@type" : "tableSelectV1", + "properties" : { + "queryContext" : "Project" + }, + "source" : { + "@type" : "tableTableJoinV1", + "properties" : { + "queryContext" : "Join" + }, + "joinType" : "LEFT", + "leftSource" : { + "@type" : "tableSelectV1", + "properties" : { + "queryContext" : "PrependAliasLeft" + }, + "source" : { + "@type" : "tableSourceV2", + "properties" : { + "queryContext" : "KafkaTopic_Left/Source" + }, + "topicName" : "left_topic", + "formats" : { + "keyFormat" : { + "format" : "KAFKA", + "properties" : { } + }, + "valueFormat" : { + "format" : "PROTOBUF", + "properties" : { + "unwrapPrimitives" : "true" + } + } + }, + "timestampColumn" : null, + "sourceSchema" : "`ID` BIGINT KEY, `NAME` STRING, `VALUE` BIGINT", + "pseudoColumnVersion" : 1, + "stateStoreFormats" : { + "keyFormat" : { + "format" : "KAFKA", + "properties" : { } + }, + "valueFormat" : { + "format" : "PROTOBUF", + "properties" : { + "unwrapPrimitives" : "true" + } + } + } + }, + "keyColumnNames" : [ "T_ID" ], + "selectExpressions" : [ "NAME AS T_NAME", "VALUE AS T_VALUE", "ROWTIME AS T_ROWTIME", "ROWPARTITION AS T_ROWPARTITION", "ROWOFFSET AS T_ROWOFFSET", "ID AS T_ID" ], + "internalFormats" : { + "keyFormat" : { + "format" : "KAFKA", + "properties" : { } + }, + "valueFormat" : { + "format" : "PROTOBUF", + "properties" : { + "unwrapPrimitives" : "true" + } + } + } + }, + "rightSource" : { + "@type" : "tableSelectV1", + "properties" : { + "queryContext" : "PrependAliasRight" + }, + "source" : { + "@type" : "tableSourceV2", + "properties" : { + "queryContext" : "KafkaTopic_Right/Source" + }, + "topicName" : "right_topic", + "formats" : { + "keyFormat" : { + "format" : "KAFKA", + "properties" : { } + }, + "valueFormat" : { + "format" : "PROTOBUF", + "properties" : { + "unwrapPrimitives" : "true" + } + } + }, + "timestampColumn" : null, + "sourceSchema" : "`ID` BIGINT KEY, `F1` STRING, `F2` BIGINT", + "pseudoColumnVersion" : 1, + "stateStoreFormats" : { + "keyFormat" : { + "format" : "KAFKA", + "properties" : { } + }, + "valueFormat" : { + "format" : "PROTOBUF", + "properties" : { + "unwrapPrimitives" : "true" + } + } + } + }, + "keyColumnNames" : [ "TT_ID" ], + "selectExpressions" : [ "F1 AS TT_F1", "F2 AS TT_F2", "ROWTIME AS TT_ROWTIME", "ROWPARTITION AS TT_ROWPARTITION", "ROWOFFSET AS TT_ROWOFFSET", "ID AS TT_ID" ], + "internalFormats" : { + "keyFormat" : { + "format" : "KAFKA", + "properties" : { } + }, + "valueFormat" : { + "format" : "PROTOBUF", + "properties" : { + "unwrapPrimitives" : "true" + } + } + } + }, + "keyColName" : "T_ID" + }, + "keyColumnNames" : [ "T_ID" ], + "selectExpressions" : [ "T_NAME AS NAME", "T_VALUE AS VALUE", "TT_F1 AS F1", "TT_F2 AS F2" ], + "internalFormats" : { + "keyFormat" : { + "format" : "KAFKA", + "properties" : { } + }, + "valueFormat" : { + "format" : "PROTOBUF", + "properties" : { + "unwrapPrimitives" : "true" + } + } + } + }, + "formats" : { + "keyFormat" : { + "format" : "KAFKA", + "properties" : { } + }, + "valueFormat" : { + "format" : "PROTOBUF", + "properties" : { + "unwrapPrimitives" : "true" + } + } + }, + "topicName" : "OUTPUT", + "timestampColumn" : null + }, + "queryId" : "CTAS_OUTPUT_0", + "runtimeId" : null + } + } ], + "configs" : { + "ksql.extension.dir" : "ext", + "ksql.streams.cache.max.bytes.buffering" : "0", + "metric.reporters" : "", + "ksql.query.status.running.threshold.seconds" : "300", + "ksql.connect.basic.auth.credentials.reload" : "false", + "ksql.output.topic.name.prefix" : "", + "ksql.query.pull.stream.enabled" : "true", + "ksql.query.push.v2.interpreter.enabled" : "true", + "ksql.queryanonymizer.logs_enabled" : "true", + "ksql.variable.substitution.enable" : "true", + "ksql.streams.shutdown.timeout.ms" : "300000", + "ksql.query.pull.max.allowed.offset.lag" : "9223372036854775807", + "ksql.query.pull.max.qps" : "2147483647", + "ksql.access.validator.enable" : "auto", + "ksql.streams.bootstrap.servers" : "localhost:0", + "ksql.query.pull.metrics.enabled" : "true", + "ksql.metrics.extension" : null, + "ksql.query.push.v2.alos.enabled" : "true", + "ksql.query.push.v2.max.hourly.bandwidth.megabytes" : "2147483647", + "ksql.query.pull.range.scan.enabled" : "true", + "ksql.transient.query.cleanup.service.initial.delay.seconds" : "600", + "ksql.hidden.topics" : "_confluent.*,__confluent.*,_schemas,__consumer_offsets,__transaction_state,connect-configs,connect-offsets,connect-status,connect-statuses", + "ksql.lambdas.enabled" : "true", + "ksql.source.table.materialization.enabled" : "true", + "ksql.query.pull.max.hourly.bandwidth.megabytes" : "2147483647", + "ksql.sink.window.change.log.additional.retention" : "1000000", + "ksql.query.persistent.active.limit" : "2147483647", + "ksql.persistence.wrap.single.values" : null, + "ksql.query.transient.max.bytes.buffering.total" : "-1", + "ksql.connect.basic.auth.credentials.source" : "NONE", + "ksql.schema.registry.url" : "schema_registry.url:0", + "ksql.properties.overrides.denylist" : "", + "ksql.service.id" : "some.ksql.service.id", + "ksql.query.push.v2.max.catchup.consumers" : "5", + "ksql.query.push.v2.enabled" : "false", + "ksql.transient.query.cleanup.service.enable" : "true", + "ksql.query.push.v2.metrics.enabled" : "true", + "ksql.rowpartition.rowoffset.enabled" : "true", + "ksql.streams.commit.interval.ms" : "2000", + "ksql.query.pull.table.scan.enabled" : "true", + "ksql.streams.auto.commit.interval.ms" : "0", + "ksql.streams.topology.optimization" : "all", + "ksql.endpoint.migrate.query" : "true", + "ksql.query.push.v2.registry.installed" : "false", + "ksql.streams.num.stream.threads" : "4", + "ksql.metrics.tags.custom" : "", + "ksql.query.push.v2.catchup.consumer.msg.window" : "50", + "ksql.runtime.feature.shared.enabled" : "false", + "ksql.udf.collect.metrics" : "false", + "ksql.new.query.planner.enabled" : "false", + "ksql.connect.request.headers.plugin" : null, + "ksql.security.extension.class" : null, + "ksql.transient.prefix" : "transient_", + "ksql.headers.columns.enabled" : "true", + "ksql.streams.default.deserialization.exception.handler" : "io.confluent.ksql.errors.LogMetricAndContinueExceptionHandler", + "ksql.connect.request.timeout.ms" : "5000", + "ksql.query.pull.enable.standby.reads" : "false", + "ksql.persistence.default.format.key" : "KAFKA", + "ksql.query.persistent.max.bytes.buffering.total" : "-1", + "ksql.query.error.max.queue.size" : "10", + "ksql.query.cleanup.shutdown.timeout.ms" : "30000", + "ksql.internal.topic.min.insync.replicas" : "1", + "ksql.internal.topic.replicas" : "1", + "ksql.insert.into.values.enabled" : "true", + "ksql.queryanonymizer.cluster_namespace" : null, + "ksql.create.or.replace.enabled" : "true", + "ksql.shared.runtimes.count" : "2", + "ksql.cast.strings.preserve.nulls" : "true", + "ksql.authorization.cache.max.entries" : "10000", + "ksql.pull.queries.enable" : "true", + "ksql.transient.query.cleanup.service.period.seconds" : "600", + "ksql.suppress.enabled" : "false", + "ksql.readonly.topics" : "_confluent.*,__confluent.*,_schemas,__consumer_offsets,__transaction_state,connect-configs,connect-offsets,connect-status,connect-statuses", + "ksql.connect.basic.auth.credentials.file" : "", + "ksql.authorization.cache.expiry.time.secs" : "30", + "ksql.query.retry.backoff.initial.ms" : "15000", + "ksql.query.pull.max.concurrent.requests" : "2147483647", + "ksql.streams.auto.offset.reset" : "earliest", + "ksql.connect.url" : "http://localhost:8083", + "ksql.query.push.v2.new.latest.delay.ms" : "5000", + "ksql.query.push.v2.latest.reset.age.ms" : "30000", + "ksql.streams.default.production.exception.handler" : "io.confluent.ksql.errors.ProductionExceptionHandlerUtil$LogAndFailProductionExceptionHandler", + "ksql.query.pull.interpreter.enabled" : "true", + "ksql.query.pull.limit.clause.enabled" : "true", + "ksql.query.pull.router.thread.pool.size" : "50", + "ksql.query.push.v2.continuation.tokens.enabled" : "false", + "ksql.query.retry.backoff.max.ms" : "900000", + "ksql.timestamp.throw.on.invalid" : "false", + "ksql.persistence.default.format.value" : null, + "ksql.udfs.enabled" : "true", + "ksql.udf.enable.security.manager" : "true", + "ksql.connect.worker.config" : "", + "ksql.nested.error.set.null" : "true", + "ksql.query.pull.thread.pool.size" : "50", + "ksql.persistent.prefix" : "query_", + "ksql.metastore.backup.location" : "", + "ksql.error.classifier.regex" : "", + "ksql.suppress.buffer.size.bytes" : "-1" + } +} \ No newline at end of file diff --git a/ksqldb-functional-tests/src/test/resources/historical_plans/joins_-_table_table_left_join_-_PROTOBUF_-_PROTOBUF/7.3.0_1651071355879/spec.json b/ksqldb-functional-tests/src/test/resources/historical_plans/joins_-_table_table_left_join_-_PROTOBUF_-_PROTOBUF/7.3.0_1651071355879/spec.json new file mode 100644 index 000000000000..7b075041f197 --- /dev/null +++ b/ksqldb-functional-tests/src/test/resources/historical_plans/joins_-_table_table_left_join_-_PROTOBUF_-_PROTOBUF/7.3.0_1651071355879/spec.json @@ -0,0 +1,367 @@ +{ + "version" : "7.3.0", + "timestamp" : 1651071355879, + "path" : "query-validation-tests/joins.json", + "schemas" : { + "CTAS_OUTPUT_0.PrependAliasLeft" : { + "schema" : "`T_ID` BIGINT KEY, `T_NAME` STRING, `T_VALUE` BIGINT, `T_ROWTIME` BIGINT, `T_ROWPARTITION` INTEGER, `T_ROWOFFSET` BIGINT, `T_ID` BIGINT", + "keyFormat" : { + "format" : "KAFKA" + }, + "valueFormat" : { + "format" : "PROTOBUF", + "properties" : { + "unwrapPrimitives" : "true" + } + } + }, + "CTAS_OUTPUT_0.KafkaTopic_Left.Source.Materialized" : { + "schema" : "`ID` BIGINT KEY, `NAME` STRING, `VALUE` BIGINT, `ROWPARTITION` INTEGER, `ROWOFFSET` BIGINT", + "keyFormat" : { + "format" : "KAFKA" + }, + "valueFormat" : { + "format" : "PROTOBUF", + "properties" : { + "unwrapPrimitives" : "true" + } + } + }, + "CTAS_OUTPUT_0.KafkaTopic_Left.Source" : { + "schema" : "`ID` BIGINT KEY, `NAME` STRING, `VALUE` BIGINT", + "keyFormat" : { + "format" : "KAFKA" + }, + "valueFormat" : { + "format" : "PROTOBUF", + "properties" : { + "unwrapPrimitives" : "true" + } + } + }, + "CTAS_OUTPUT_0.Project" : { + "schema" : "`T_ID` BIGINT KEY, `NAME` STRING, `VALUE` BIGINT, `F1` STRING, `F2` BIGINT", + "keyFormat" : { + "format" : "KAFKA" + }, + "valueFormat" : { + "format" : "PROTOBUF", + "properties" : { + "unwrapPrimitives" : "true" + } + } + }, + "CTAS_OUTPUT_0.KafkaTopic_Right.Source" : { + "schema" : "`ID` BIGINT KEY, `F1` STRING, `F2` BIGINT", + "keyFormat" : { + "format" : "KAFKA" + }, + "valueFormat" : { + "format" : "PROTOBUF", + "properties" : { + "unwrapPrimitives" : "true" + } + } + }, + "CTAS_OUTPUT_0.OUTPUT" : { + "schema" : "`T_ID` BIGINT KEY, `NAME` STRING, `VALUE` BIGINT, `F1` STRING, `F2` BIGINT", + "keyFormat" : { + "format" : "KAFKA" + }, + "valueFormat" : { + "format" : "PROTOBUF", + "properties" : { + "unwrapPrimitives" : "true" + } + } + }, + "CTAS_OUTPUT_0.PrependAliasRight" : { + "schema" : "`TT_ID` BIGINT KEY, `TT_F1` STRING, `TT_F2` BIGINT, `TT_ROWTIME` BIGINT, `TT_ROWPARTITION` INTEGER, `TT_ROWOFFSET` BIGINT, `TT_ID` BIGINT", + "keyFormat" : { + "format" : "KAFKA" + }, + "valueFormat" : { + "format" : "PROTOBUF", + "properties" : { + "unwrapPrimitives" : "true" + } + } + }, + "CTAS_OUTPUT_0.KafkaTopic_Right.Source.Materialized" : { + "schema" : "`ID` BIGINT KEY, `F1` STRING, `F2` BIGINT, `ROWPARTITION` INTEGER, `ROWOFFSET` BIGINT", + "keyFormat" : { + "format" : "KAFKA" + }, + "valueFormat" : { + "format" : "PROTOBUF", + "properties" : { + "unwrapPrimitives" : "true" + } + } + } + }, + "testCase" : { + "name" : "table table left join - PROTOBUF - PROTOBUF", + "inputs" : [ { + "topic" : "left_topic", + "key" : 0, + "value" : { + "NAME" : "zero", + "VALUE" : 0 + }, + "timestamp" : 0 + }, { + "topic" : "right_topic", + "key" : 0, + "value" : { + "F1" : "blah", + "F2" : 50 + }, + "timestamp" : 10000 + }, { + "topic" : "left_topic", + "key" : 10, + "value" : { + "NAME" : "100", + "VALUE" : 5 + }, + "timestamp" : 11000 + }, { + "topic" : "left_topic", + "key" : 0, + "value" : { + "NAME" : "foo", + "VALUE" : 100 + }, + "timestamp" : 13000 + }, { + "topic" : "right_topic", + "key" : 0, + "value" : { + "F1" : "a", + "F2" : 10 + }, + "timestamp" : 15000 + }, { + "topic" : "left_topic", + "key" : 0, + "value" : { + "NAME" : "bar", + "VALUE" : 99 + }, + "timestamp" : 16000 + }, { + "topic" : "left_topic", + "key" : 90, + "value" : { + "NAME" : "ninety", + "VALUE" : 90 + }, + "timestamp" : 17000 + } ], + "outputs" : [ { + "topic" : "OUTPUT", + "key" : 0, + "value" : { + "NAME" : "zero", + "VALUE" : 0, + "F1" : "", + "F2" : 0 + }, + "timestamp" : 0 + }, { + "topic" : "OUTPUT", + "key" : 0, + "value" : { + "NAME" : "zero", + "VALUE" : 0, + "F1" : "blah", + "F2" : 50 + }, + "timestamp" : 10000 + }, { + "topic" : "OUTPUT", + "key" : 10, + "value" : { + "NAME" : "100", + "VALUE" : 5, + "F1" : "", + "F2" : 0 + }, + "timestamp" : 11000 + }, { + "topic" : "OUTPUT", + "key" : 0, + "value" : { + "NAME" : "foo", + "VALUE" : 100, + "F1" : "blah", + "F2" : 50 + }, + "timestamp" : 13000 + }, { + "topic" : "OUTPUT", + "key" : 0, + "value" : { + "NAME" : "foo", + "VALUE" : 100, + "F1" : "a", + "F2" : 10 + }, + "timestamp" : 15000 + }, { + "topic" : "OUTPUT", + "key" : 0, + "value" : { + "NAME" : "bar", + "VALUE" : 99, + "F1" : "a", + "F2" : 10 + }, + "timestamp" : 16000 + }, { + "topic" : "OUTPUT", + "key" : 90, + "value" : { + "NAME" : "ninety", + "VALUE" : 90, + "F1" : "", + "F2" : 0 + }, + "timestamp" : 17000 + } ], + "topics" : [ { + "name" : "right_topic", + "valueSchema" : "syntax = \"proto3\";\n\nmessage ConnectDefault1 {\n string F1 = 1;\n int64 F2 = 2;\n}\n", + "valueFormat" : "PROTOBUF", + "replicas" : 1, + "numPartitions" : 4 + }, { + "name" : "OUTPUT", + "replicas" : 1, + "numPartitions" : 4 + }, { + "name" : "left_topic", + "valueSchema" : "syntax = \"proto3\";\n\nmessage ConnectDefault1 {\n string NAME = 1;\n int64 VALUE = 2;\n}\n", + "valueFormat" : "PROTOBUF", + "replicas" : 1, + "numPartitions" : 4 + } ], + "statements" : [ "CREATE TABLE TEST (id BIGINT PRIMARY KEY, name VARCHAR, value BIGINT) WITH (kafka_topic='left_topic', value_format='PROTOBUF');", "CREATE TABLE TEST_TABLE (id BIGINT PRIMARY KEY, f1 VARCHAR, f2 BIGINT) WITH (kafka_topic='right_topic', value_format='PROTOBUF');", "CREATE TABLE OUTPUT as SELECT t.id, name, value, f1, f2 FROM test t left join TEST_TABLE tt on t.id = tt.id;" ], + "post" : { + "sources" : [ { + "name" : "OUTPUT", + "type" : "TABLE", + "schema" : "`T_ID` BIGINT KEY, `NAME` STRING, `VALUE` BIGINT, `F1` STRING, `F2` BIGINT", + "keyFormat" : { + "format" : "KAFKA" + }, + "valueFormat" : "PROTOBUF", + "keyFeatures" : [ ], + "valueFeatures" : [ ], + "isSource" : false + }, { + "name" : "TEST", + "type" : "TABLE", + "schema" : "`ID` BIGINT KEY, `NAME` STRING, `VALUE` BIGINT", + "keyFormat" : { + "format" : "KAFKA" + }, + "valueFormat" : "PROTOBUF", + "keyFeatures" : [ ], + "valueFeatures" : [ ], + "isSource" : false + }, { + "name" : "TEST_TABLE", + "type" : "TABLE", + "schema" : "`ID` BIGINT KEY, `F1` STRING, `F2` BIGINT", + "keyFormat" : { + "format" : "KAFKA" + }, + "valueFormat" : "PROTOBUF", + "keyFeatures" : [ ], + "valueFeatures" : [ ], + "isSource" : false + } ], + "topics" : { + "topics" : [ { + "name" : "_confluent-ksql-some.ksql.service.idquery_CTAS_OUTPUT_0-Project-Last-changelog", + "keyFormat" : { + "format" : "KAFKA" + }, + "valueFormat" : { + "format" : "PROTOBUF", + "properties" : { + "unwrapPrimitives" : "true" + } + }, + "valueSchema" : "syntax = \"proto3\";\n\nmessage ConnectDefault1 {\n string NAME = 1;\n int64 VALUE = 2;\n string F1 = 3;\n int64 F2 = 4;\n}\n" + }, { + "name" : "_confluent-ksql-some.ksql.service.idquery_CTAS_OUTPUT_0-KafkaTopic_Right-Reduce-changelog", + "keyFormat" : { + "format" : "KAFKA" + }, + "valueFormat" : { + "format" : "PROTOBUF", + "properties" : { + "unwrapPrimitives" : "true" + } + }, + "valueSchema" : "syntax = \"proto3\";\n\nmessage ConnectDefault1 {\n string F1 = 1;\n int64 F2 = 2;\n int32 ROWPARTITION = 3;\n int64 ROWOFFSET = 4;\n}\n" + }, { + "name" : "left_topic", + "keyFormat" : { + "format" : "KAFKA" + }, + "valueFormat" : { + "format" : "PROTOBUF", + "properties" : { + "unwrapPrimitives" : "true" + } + }, + "partitions" : 4, + "valueSchema" : "syntax = \"proto3\";\n\nmessage ConnectDefault1 {\n string NAME = 1;\n int64 VALUE = 2;\n}\n" + }, { + "name" : "_confluent-ksql-some.ksql.service.idquery_CTAS_OUTPUT_0-KafkaTopic_Left-Reduce-changelog", + "keyFormat" : { + "format" : "KAFKA" + }, + "valueFormat" : { + "format" : "PROTOBUF", + "properties" : { + "unwrapPrimitives" : "true" + } + }, + "valueSchema" : "syntax = \"proto3\";\n\nmessage ConnectDefault1 {\n string NAME = 1;\n int64 VALUE = 2;\n int32 ROWPARTITION = 3;\n int64 ROWOFFSET = 4;\n}\n" + }, { + "name" : "OUTPUT", + "keyFormat" : { + "format" : "KAFKA" + }, + "valueFormat" : { + "format" : "PROTOBUF", + "properties" : { + "unwrapPrimitives" : "true" + } + }, + "partitions" : 4, + "valueSchema" : "syntax = \"proto3\";\n\nmessage ConnectDefault1 {\n string NAME = 1;\n int64 VALUE = 2;\n string F1 = 3;\n int64 F2 = 4;\n}\n" + }, { + "name" : "right_topic", + "keyFormat" : { + "format" : "KAFKA" + }, + "valueFormat" : { + "format" : "PROTOBUF", + "properties" : { + "unwrapPrimitives" : "true" + } + }, + "partitions" : 4, + "valueSchema" : "syntax = \"proto3\";\n\nmessage ConnectDefault1 {\n string F1 = 1;\n int64 F2 = 2;\n}\n" + } ], + "blackList" : ".*-repartition" + } + } + } +} \ No newline at end of file diff --git a/ksqldb-functional-tests/src/test/resources/historical_plans/joins_-_table_table_left_join_-_PROTOBUF_-_PROTOBUF/7.3.0_1651071355879/topology b/ksqldb-functional-tests/src/test/resources/historical_plans/joins_-_table_table_left_join_-_PROTOBUF_-_PROTOBUF/7.3.0_1651071355879/topology new file mode 100644 index 000000000000..3429d44ba439 --- /dev/null +++ b/ksqldb-functional-tests/src/test/resources/historical_plans/joins_-_table_table_left_join_-_PROTOBUF_-_PROTOBUF/7.3.0_1651071355879/topology @@ -0,0 +1,48 @@ +Topologies: + Sub-topology: 0 + Source: KSTREAM-SOURCE-0000000001 (topics: [left_topic]) + --> KTABLE-SOURCE-0000000002 + Source: KSTREAM-SOURCE-0000000007 (topics: [right_topic]) + --> KTABLE-SOURCE-0000000008 + Processor: KTABLE-SOURCE-0000000002 (stores: []) + --> KTABLE-TRANSFORMVALUES-0000000003 + <-- KSTREAM-SOURCE-0000000001 + Processor: KTABLE-SOURCE-0000000008 (stores: []) + --> KTABLE-TRANSFORMVALUES-0000000009 + <-- KSTREAM-SOURCE-0000000007 + Processor: KTABLE-TRANSFORMVALUES-0000000003 (stores: [KafkaTopic_Left-Reduce]) + --> KTABLE-TRANSFORMVALUES-0000000004 + <-- KTABLE-SOURCE-0000000002 + Processor: KTABLE-TRANSFORMVALUES-0000000009 (stores: [KafkaTopic_Right-Reduce]) + --> KTABLE-TRANSFORMVALUES-0000000010 + <-- KTABLE-SOURCE-0000000008 + Processor: KTABLE-TRANSFORMVALUES-0000000004 (stores: []) + --> PrependAliasLeft + <-- KTABLE-TRANSFORMVALUES-0000000003 + Processor: KTABLE-TRANSFORMVALUES-0000000010 (stores: []) + --> PrependAliasRight + <-- KTABLE-TRANSFORMVALUES-0000000009 + Processor: PrependAliasLeft (stores: []) + --> KTABLE-JOINTHIS-0000000013 + <-- KTABLE-TRANSFORMVALUES-0000000004 + Processor: PrependAliasRight (stores: []) + --> KTABLE-JOINOTHER-0000000014 + <-- KTABLE-TRANSFORMVALUES-0000000010 + Processor: KTABLE-JOINOTHER-0000000014 (stores: [KafkaTopic_Left-Reduce]) + --> KTABLE-MERGE-0000000012 + <-- PrependAliasRight + Processor: KTABLE-JOINTHIS-0000000013 (stores: [KafkaTopic_Right-Reduce]) + --> KTABLE-MERGE-0000000012 + <-- PrependAliasLeft + Processor: KTABLE-MERGE-0000000012 (stores: []) + --> KTABLE-TRANSFORMVALUES-0000000015 + <-- KTABLE-JOINTHIS-0000000013, KTABLE-JOINOTHER-0000000014 + Processor: KTABLE-TRANSFORMVALUES-0000000015 (stores: [Project-Last]) + --> KTABLE-TOSTREAM-0000000016 + <-- KTABLE-MERGE-0000000012 + Processor: KTABLE-TOSTREAM-0000000016 (stores: []) + --> KSTREAM-SINK-0000000017 + <-- KTABLE-TRANSFORMVALUES-0000000015 + Sink: KSTREAM-SINK-0000000017 (topic: OUTPUT) + <-- KTABLE-TOSTREAM-0000000016 + diff --git a/ksqldb-functional-tests/src/test/resources/historical_plans/joins_-_table_table_left_join_-_PROTOBUF_-_PROTOBUF_NOSR/7.3.0_1651071358228/plan.json b/ksqldb-functional-tests/src/test/resources/historical_plans/joins_-_table_table_left_join_-_PROTOBUF_-_PROTOBUF_NOSR/7.3.0_1651071358228/plan.json new file mode 100644 index 000000000000..5b51e5e7c0ba --- /dev/null +++ b/ksqldb-functional-tests/src/test/resources/historical_plans/joins_-_table_table_left_join_-_PROTOBUF_-_PROTOBUF_NOSR/7.3.0_1651071358228/plan.json @@ -0,0 +1,320 @@ +{ + "plan" : [ { + "@type" : "ksqlPlanV1", + "statementText" : "CREATE TABLE TEST (ID BIGINT PRIMARY KEY, NAME STRING, VALUE BIGINT) WITH (KAFKA_TOPIC='left_topic', KEY_FORMAT='KAFKA', VALUE_FORMAT='PROTOBUF_NOSR');", + "ddlCommand" : { + "@type" : "createTableV1", + "sourceName" : "TEST", + "schema" : "`ID` BIGINT KEY, `NAME` STRING, `VALUE` BIGINT", + "timestampColumn" : null, + "topicName" : "left_topic", + "formats" : { + "keyFormat" : { + "format" : "KAFKA", + "properties" : { } + }, + "valueFormat" : { + "format" : "PROTOBUF_NOSR", + "properties" : { } + } + }, + "windowInfo" : null, + "orReplace" : false, + "isSource" : false + }, + "queryPlan" : null + }, { + "@type" : "ksqlPlanV1", + "statementText" : "CREATE TABLE TEST_TABLE (ID BIGINT PRIMARY KEY, F1 STRING, F2 BIGINT) WITH (KAFKA_TOPIC='right_topic', KEY_FORMAT='KAFKA', VALUE_FORMAT='PROTOBUF_NOSR');", + "ddlCommand" : { + "@type" : "createTableV1", + "sourceName" : "TEST_TABLE", + "schema" : "`ID` BIGINT KEY, `F1` STRING, `F2` BIGINT", + "timestampColumn" : null, + "topicName" : "right_topic", + "formats" : { + "keyFormat" : { + "format" : "KAFKA", + "properties" : { } + }, + "valueFormat" : { + "format" : "PROTOBUF_NOSR", + "properties" : { } + } + }, + "windowInfo" : null, + "orReplace" : false, + "isSource" : false + }, + "queryPlan" : null + }, { + "@type" : "ksqlPlanV1", + "statementText" : "CREATE TABLE OUTPUT AS SELECT\n T.ID T_ID,\n T.NAME NAME,\n T.VALUE VALUE,\n TT.F1 F1,\n TT.F2 F2\nFROM TEST T\nLEFT OUTER JOIN TEST_TABLE TT ON ((T.ID = TT.ID))\nEMIT CHANGES", + "ddlCommand" : { + "@type" : "createTableV1", + "sourceName" : "OUTPUT", + "schema" : "`T_ID` BIGINT KEY, `NAME` STRING, `VALUE` BIGINT, `F1` STRING, `F2` BIGINT", + "timestampColumn" : null, + "topicName" : "OUTPUT", + "formats" : { + "keyFormat" : { + "format" : "KAFKA", + "properties" : { } + }, + "valueFormat" : { + "format" : "PROTOBUF_NOSR", + "properties" : { } + } + }, + "windowInfo" : null, + "orReplace" : false, + "isSource" : false + }, + "queryPlan" : { + "sources" : [ "TEST", "TEST_TABLE" ], + "sink" : "OUTPUT", + "physicalPlan" : { + "@type" : "tableSinkV1", + "properties" : { + "queryContext" : "OUTPUT" + }, + "source" : { + "@type" : "tableSelectV1", + "properties" : { + "queryContext" : "Project" + }, + "source" : { + "@type" : "tableTableJoinV1", + "properties" : { + "queryContext" : "Join" + }, + "joinType" : "LEFT", + "leftSource" : { + "@type" : "tableSelectV1", + "properties" : { + "queryContext" : "PrependAliasLeft" + }, + "source" : { + "@type" : "tableSourceV2", + "properties" : { + "queryContext" : "KafkaTopic_Left/Source" + }, + "topicName" : "left_topic", + "formats" : { + "keyFormat" : { + "format" : "KAFKA", + "properties" : { } + }, + "valueFormat" : { + "format" : "PROTOBUF_NOSR", + "properties" : { } + } + }, + "timestampColumn" : null, + "sourceSchema" : "`ID` BIGINT KEY, `NAME` STRING, `VALUE` BIGINT", + "pseudoColumnVersion" : 1, + "stateStoreFormats" : { + "keyFormat" : { + "format" : "KAFKA", + "properties" : { } + }, + "valueFormat" : { + "format" : "PROTOBUF_NOSR", + "properties" : { } + } + } + }, + "keyColumnNames" : [ "T_ID" ], + "selectExpressions" : [ "NAME AS T_NAME", "VALUE AS T_VALUE", "ROWTIME AS T_ROWTIME", "ROWPARTITION AS T_ROWPARTITION", "ROWOFFSET AS T_ROWOFFSET", "ID AS T_ID" ], + "internalFormats" : { + "keyFormat" : { + "format" : "KAFKA", + "properties" : { } + }, + "valueFormat" : { + "format" : "PROTOBUF_NOSR", + "properties" : { } + } + } + }, + "rightSource" : { + "@type" : "tableSelectV1", + "properties" : { + "queryContext" : "PrependAliasRight" + }, + "source" : { + "@type" : "tableSourceV2", + "properties" : { + "queryContext" : "KafkaTopic_Right/Source" + }, + "topicName" : "right_topic", + "formats" : { + "keyFormat" : { + "format" : "KAFKA", + "properties" : { } + }, + "valueFormat" : { + "format" : "PROTOBUF_NOSR", + "properties" : { } + } + }, + "timestampColumn" : null, + "sourceSchema" : "`ID` BIGINT KEY, `F1` STRING, `F2` BIGINT", + "pseudoColumnVersion" : 1, + "stateStoreFormats" : { + "keyFormat" : { + "format" : "KAFKA", + "properties" : { } + }, + "valueFormat" : { + "format" : "PROTOBUF_NOSR", + "properties" : { } + } + } + }, + "keyColumnNames" : [ "TT_ID" ], + "selectExpressions" : [ "F1 AS TT_F1", "F2 AS TT_F2", "ROWTIME AS TT_ROWTIME", "ROWPARTITION AS TT_ROWPARTITION", "ROWOFFSET AS TT_ROWOFFSET", "ID AS TT_ID" ], + "internalFormats" : { + "keyFormat" : { + "format" : "KAFKA", + "properties" : { } + }, + "valueFormat" : { + "format" : "PROTOBUF_NOSR", + "properties" : { } + } + } + }, + "keyColName" : "T_ID" + }, + "keyColumnNames" : [ "T_ID" ], + "selectExpressions" : [ "T_NAME AS NAME", "T_VALUE AS VALUE", "TT_F1 AS F1", "TT_F2 AS F2" ], + "internalFormats" : { + "keyFormat" : { + "format" : "KAFKA", + "properties" : { } + }, + "valueFormat" : { + "format" : "PROTOBUF_NOSR", + "properties" : { } + } + } + }, + "formats" : { + "keyFormat" : { + "format" : "KAFKA", + "properties" : { } + }, + "valueFormat" : { + "format" : "PROTOBUF_NOSR", + "properties" : { } + } + }, + "topicName" : "OUTPUT", + "timestampColumn" : null + }, + "queryId" : "CTAS_OUTPUT_0", + "runtimeId" : null + } + } ], + "configs" : { + "ksql.extension.dir" : "ext", + "ksql.streams.cache.max.bytes.buffering" : "0", + "metric.reporters" : "", + "ksql.query.status.running.threshold.seconds" : "300", + "ksql.connect.basic.auth.credentials.reload" : "false", + "ksql.output.topic.name.prefix" : "", + "ksql.query.pull.stream.enabled" : "true", + "ksql.query.push.v2.interpreter.enabled" : "true", + "ksql.queryanonymizer.logs_enabled" : "true", + "ksql.variable.substitution.enable" : "true", + "ksql.streams.shutdown.timeout.ms" : "300000", + "ksql.query.pull.max.allowed.offset.lag" : "9223372036854775807", + "ksql.query.pull.max.qps" : "2147483647", + "ksql.access.validator.enable" : "auto", + "ksql.streams.bootstrap.servers" : "localhost:0", + "ksql.query.pull.metrics.enabled" : "true", + "ksql.metrics.extension" : null, + "ksql.query.push.v2.alos.enabled" : "true", + "ksql.query.push.v2.max.hourly.bandwidth.megabytes" : "2147483647", + "ksql.query.pull.range.scan.enabled" : "true", + "ksql.transient.query.cleanup.service.initial.delay.seconds" : "600", + "ksql.hidden.topics" : "_confluent.*,__confluent.*,_schemas,__consumer_offsets,__transaction_state,connect-configs,connect-offsets,connect-status,connect-statuses", + "ksql.lambdas.enabled" : "true", + "ksql.source.table.materialization.enabled" : "true", + "ksql.query.pull.max.hourly.bandwidth.megabytes" : "2147483647", + "ksql.sink.window.change.log.additional.retention" : "1000000", + "ksql.query.persistent.active.limit" : "2147483647", + "ksql.persistence.wrap.single.values" : null, + "ksql.query.transient.max.bytes.buffering.total" : "-1", + "ksql.connect.basic.auth.credentials.source" : "NONE", + "ksql.schema.registry.url" : "schema_registry.url:0", + "ksql.properties.overrides.denylist" : "", + "ksql.service.id" : "some.ksql.service.id", + "ksql.query.push.v2.max.catchup.consumers" : "5", + "ksql.query.push.v2.enabled" : "false", + "ksql.transient.query.cleanup.service.enable" : "true", + "ksql.query.push.v2.metrics.enabled" : "true", + "ksql.rowpartition.rowoffset.enabled" : "true", + "ksql.streams.commit.interval.ms" : "2000", + "ksql.query.pull.table.scan.enabled" : "true", + "ksql.streams.auto.commit.interval.ms" : "0", + "ksql.streams.topology.optimization" : "all", + "ksql.endpoint.migrate.query" : "true", + "ksql.query.push.v2.registry.installed" : "false", + "ksql.streams.num.stream.threads" : "4", + "ksql.metrics.tags.custom" : "", + "ksql.query.push.v2.catchup.consumer.msg.window" : "50", + "ksql.runtime.feature.shared.enabled" : "false", + "ksql.udf.collect.metrics" : "false", + "ksql.new.query.planner.enabled" : "false", + "ksql.connect.request.headers.plugin" : null, + "ksql.security.extension.class" : null, + "ksql.transient.prefix" : "transient_", + "ksql.headers.columns.enabled" : "true", + "ksql.streams.default.deserialization.exception.handler" : "io.confluent.ksql.errors.LogMetricAndContinueExceptionHandler", + "ksql.connect.request.timeout.ms" : "5000", + "ksql.query.pull.enable.standby.reads" : "false", + "ksql.persistence.default.format.key" : "KAFKA", + "ksql.query.persistent.max.bytes.buffering.total" : "-1", + "ksql.query.error.max.queue.size" : "10", + "ksql.query.cleanup.shutdown.timeout.ms" : "30000", + "ksql.internal.topic.min.insync.replicas" : "1", + "ksql.internal.topic.replicas" : "1", + "ksql.insert.into.values.enabled" : "true", + "ksql.queryanonymizer.cluster_namespace" : null, + "ksql.create.or.replace.enabled" : "true", + "ksql.shared.runtimes.count" : "2", + "ksql.cast.strings.preserve.nulls" : "true", + "ksql.authorization.cache.max.entries" : "10000", + "ksql.pull.queries.enable" : "true", + "ksql.transient.query.cleanup.service.period.seconds" : "600", + "ksql.suppress.enabled" : "false", + "ksql.readonly.topics" : "_confluent.*,__confluent.*,_schemas,__consumer_offsets,__transaction_state,connect-configs,connect-offsets,connect-status,connect-statuses", + "ksql.connect.basic.auth.credentials.file" : "", + "ksql.authorization.cache.expiry.time.secs" : "30", + "ksql.query.retry.backoff.initial.ms" : "15000", + "ksql.query.pull.max.concurrent.requests" : "2147483647", + "ksql.streams.auto.offset.reset" : "earliest", + "ksql.connect.url" : "http://localhost:8083", + "ksql.query.push.v2.new.latest.delay.ms" : "5000", + "ksql.query.push.v2.latest.reset.age.ms" : "30000", + "ksql.streams.default.production.exception.handler" : "io.confluent.ksql.errors.ProductionExceptionHandlerUtil$LogAndFailProductionExceptionHandler", + "ksql.query.pull.interpreter.enabled" : "true", + "ksql.query.pull.limit.clause.enabled" : "true", + "ksql.query.pull.router.thread.pool.size" : "50", + "ksql.query.push.v2.continuation.tokens.enabled" : "false", + "ksql.query.retry.backoff.max.ms" : "900000", + "ksql.timestamp.throw.on.invalid" : "false", + "ksql.persistence.default.format.value" : null, + "ksql.udfs.enabled" : "true", + "ksql.udf.enable.security.manager" : "true", + "ksql.connect.worker.config" : "", + "ksql.nested.error.set.null" : "true", + "ksql.query.pull.thread.pool.size" : "50", + "ksql.persistent.prefix" : "query_", + "ksql.metastore.backup.location" : "", + "ksql.error.classifier.regex" : "", + "ksql.suppress.buffer.size.bytes" : "-1" + } +} \ No newline at end of file diff --git a/ksqldb-functional-tests/src/test/resources/historical_plans/joins_-_table_table_left_join_-_PROTOBUF_-_PROTOBUF_NOSR/7.3.0_1651071358228/spec.json b/ksqldb-functional-tests/src/test/resources/historical_plans/joins_-_table_table_left_join_-_PROTOBUF_-_PROTOBUF_NOSR/7.3.0_1651071358228/spec.json new file mode 100644 index 000000000000..5f3d5dc06465 --- /dev/null +++ b/ksqldb-functional-tests/src/test/resources/historical_plans/joins_-_table_table_left_join_-_PROTOBUF_-_PROTOBUF_NOSR/7.3.0_1651071358228/spec.json @@ -0,0 +1,315 @@ +{ + "version" : "7.3.0", + "timestamp" : 1651071358228, + "path" : "query-validation-tests/joins.json", + "schemas" : { + "CTAS_OUTPUT_0.PrependAliasLeft" : { + "schema" : "`T_ID` BIGINT KEY, `T_NAME` STRING, `T_VALUE` BIGINT, `T_ROWTIME` BIGINT, `T_ROWPARTITION` INTEGER, `T_ROWOFFSET` BIGINT, `T_ID` BIGINT", + "keyFormat" : { + "format" : "KAFKA" + }, + "valueFormat" : { + "format" : "PROTOBUF_NOSR" + } + }, + "CTAS_OUTPUT_0.KafkaTopic_Left.Source.Materialized" : { + "schema" : "`ID` BIGINT KEY, `NAME` STRING, `VALUE` BIGINT, `ROWPARTITION` INTEGER, `ROWOFFSET` BIGINT", + "keyFormat" : { + "format" : "KAFKA" + }, + "valueFormat" : { + "format" : "PROTOBUF_NOSR" + } + }, + "CTAS_OUTPUT_0.KafkaTopic_Left.Source" : { + "schema" : "`ID` BIGINT KEY, `NAME` STRING, `VALUE` BIGINT", + "keyFormat" : { + "format" : "KAFKA" + }, + "valueFormat" : { + "format" : "PROTOBUF_NOSR" + } + }, + "CTAS_OUTPUT_0.Project" : { + "schema" : "`T_ID` BIGINT KEY, `NAME` STRING, `VALUE` BIGINT, `F1` STRING, `F2` BIGINT", + "keyFormat" : { + "format" : "KAFKA" + }, + "valueFormat" : { + "format" : "PROTOBUF_NOSR" + } + }, + "CTAS_OUTPUT_0.KafkaTopic_Right.Source" : { + "schema" : "`ID` BIGINT KEY, `F1` STRING, `F2` BIGINT", + "keyFormat" : { + "format" : "KAFKA" + }, + "valueFormat" : { + "format" : "PROTOBUF_NOSR" + } + }, + "CTAS_OUTPUT_0.OUTPUT" : { + "schema" : "`T_ID` BIGINT KEY, `NAME` STRING, `VALUE` BIGINT, `F1` STRING, `F2` BIGINT", + "keyFormat" : { + "format" : "KAFKA" + }, + "valueFormat" : { + "format" : "PROTOBUF_NOSR" + } + }, + "CTAS_OUTPUT_0.PrependAliasRight" : { + "schema" : "`TT_ID` BIGINT KEY, `TT_F1` STRING, `TT_F2` BIGINT, `TT_ROWTIME` BIGINT, `TT_ROWPARTITION` INTEGER, `TT_ROWOFFSET` BIGINT, `TT_ID` BIGINT", + "keyFormat" : { + "format" : "KAFKA" + }, + "valueFormat" : { + "format" : "PROTOBUF_NOSR" + } + }, + "CTAS_OUTPUT_0.KafkaTopic_Right.Source.Materialized" : { + "schema" : "`ID` BIGINT KEY, `F1` STRING, `F2` BIGINT, `ROWPARTITION` INTEGER, `ROWOFFSET` BIGINT", + "keyFormat" : { + "format" : "KAFKA" + }, + "valueFormat" : { + "format" : "PROTOBUF_NOSR" + } + } + }, + "testCase" : { + "name" : "table table left join - PROTOBUF - PROTOBUF_NOSR", + "inputs" : [ { + "topic" : "left_topic", + "key" : 0, + "value" : { + "NAME" : "zero", + "VALUE" : 0 + }, + "timestamp" : 0 + }, { + "topic" : "right_topic", + "key" : 0, + "value" : { + "F1" : "blah", + "F2" : 50 + }, + "timestamp" : 10000 + }, { + "topic" : "left_topic", + "key" : 10, + "value" : { + "NAME" : "100", + "VALUE" : 5 + }, + "timestamp" : 11000 + }, { + "topic" : "left_topic", + "key" : 0, + "value" : { + "NAME" : "foo", + "VALUE" : 100 + }, + "timestamp" : 13000 + }, { + "topic" : "right_topic", + "key" : 0, + "value" : { + "F1" : "a", + "F2" : 10 + }, + "timestamp" : 15000 + }, { + "topic" : "left_topic", + "key" : 0, + "value" : { + "NAME" : "bar", + "VALUE" : 99 + }, + "timestamp" : 16000 + }, { + "topic" : "left_topic", + "key" : 90, + "value" : { + "NAME" : "ninety", + "VALUE" : 90 + }, + "timestamp" : 17000 + } ], + "outputs" : [ { + "topic" : "OUTPUT", + "key" : 0, + "value" : { + "NAME" : "zero", + "VALUE" : 0, + "F1" : "", + "F2" : 0 + }, + "timestamp" : 0 + }, { + "topic" : "OUTPUT", + "key" : 0, + "value" : { + "NAME" : "zero", + "VALUE" : 0, + "F1" : "blah", + "F2" : 50 + }, + "timestamp" : 10000 + }, { + "topic" : "OUTPUT", + "key" : 10, + "value" : { + "NAME" : "100", + "VALUE" : 5, + "F1" : "", + "F2" : 0 + }, + "timestamp" : 11000 + }, { + "topic" : "OUTPUT", + "key" : 0, + "value" : { + "NAME" : "foo", + "VALUE" : 100, + "F1" : "blah", + "F2" : 50 + }, + "timestamp" : 13000 + }, { + "topic" : "OUTPUT", + "key" : 0, + "value" : { + "NAME" : "foo", + "VALUE" : 100, + "F1" : "a", + "F2" : 10 + }, + "timestamp" : 15000 + }, { + "topic" : "OUTPUT", + "key" : 0, + "value" : { + "NAME" : "bar", + "VALUE" : 99, + "F1" : "a", + "F2" : 10 + }, + "timestamp" : 16000 + }, { + "topic" : "OUTPUT", + "key" : 90, + "value" : { + "NAME" : "ninety", + "VALUE" : 90, + "F1" : "", + "F2" : 0 + }, + "timestamp" : 17000 + } ], + "topics" : [ { + "name" : "right_topic", + "replicas" : 1, + "numPartitions" : 4 + }, { + "name" : "OUTPUT", + "replicas" : 1, + "numPartitions" : 4 + }, { + "name" : "left_topic", + "replicas" : 1, + "numPartitions" : 4 + } ], + "statements" : [ "CREATE TABLE TEST (id BIGINT PRIMARY KEY, name VARCHAR, value BIGINT) WITH (kafka_topic='left_topic', value_format='PROTOBUF_NOSR');", "CREATE TABLE TEST_TABLE (id BIGINT PRIMARY KEY, f1 VARCHAR, f2 BIGINT) WITH (kafka_topic='right_topic', value_format='PROTOBUF_NOSR');", "CREATE TABLE OUTPUT as SELECT t.id, name, value, f1, f2 FROM test t left join TEST_TABLE tt on t.id = tt.id;" ], + "post" : { + "sources" : [ { + "name" : "OUTPUT", + "type" : "TABLE", + "schema" : "`T_ID` BIGINT KEY, `NAME` STRING, `VALUE` BIGINT, `F1` STRING, `F2` BIGINT", + "keyFormat" : { + "format" : "KAFKA" + }, + "valueFormat" : "PROTOBUF_NOSR", + "keyFeatures" : [ ], + "valueFeatures" : [ ], + "isSource" : false + }, { + "name" : "TEST", + "type" : "TABLE", + "schema" : "`ID` BIGINT KEY, `NAME` STRING, `VALUE` BIGINT", + "keyFormat" : { + "format" : "KAFKA" + }, + "valueFormat" : "PROTOBUF_NOSR", + "keyFeatures" : [ ], + "valueFeatures" : [ ], + "isSource" : false + }, { + "name" : "TEST_TABLE", + "type" : "TABLE", + "schema" : "`ID` BIGINT KEY, `F1` STRING, `F2` BIGINT", + "keyFormat" : { + "format" : "KAFKA" + }, + "valueFormat" : "PROTOBUF_NOSR", + "keyFeatures" : [ ], + "valueFeatures" : [ ], + "isSource" : false + } ], + "topics" : { + "topics" : [ { + "name" : "_confluent-ksql-some.ksql.service.idquery_CTAS_OUTPUT_0-Project-Last-changelog", + "keyFormat" : { + "format" : "KAFKA" + }, + "valueFormat" : { + "format" : "PROTOBUF_NOSR" + } + }, { + "name" : "_confluent-ksql-some.ksql.service.idquery_CTAS_OUTPUT_0-KafkaTopic_Right-Reduce-changelog", + "keyFormat" : { + "format" : "KAFKA" + }, + "valueFormat" : { + "format" : "PROTOBUF_NOSR" + } + }, { + "name" : "left_topic", + "keyFormat" : { + "format" : "KAFKA" + }, + "valueFormat" : { + "format" : "PROTOBUF_NOSR" + }, + "partitions" : 4 + }, { + "name" : "_confluent-ksql-some.ksql.service.idquery_CTAS_OUTPUT_0-KafkaTopic_Left-Reduce-changelog", + "keyFormat" : { + "format" : "KAFKA" + }, + "valueFormat" : { + "format" : "PROTOBUF_NOSR" + } + }, { + "name" : "OUTPUT", + "keyFormat" : { + "format" : "KAFKA" + }, + "valueFormat" : { + "format" : "PROTOBUF_NOSR" + }, + "partitions" : 4 + }, { + "name" : "right_topic", + "keyFormat" : { + "format" : "KAFKA" + }, + "valueFormat" : { + "format" : "PROTOBUF_NOSR" + }, + "partitions" : 4 + } ], + "blackList" : ".*-repartition" + } + } + } +} \ No newline at end of file diff --git a/ksqldb-functional-tests/src/test/resources/historical_plans/joins_-_table_table_left_join_-_PROTOBUF_-_PROTOBUF_NOSR/7.3.0_1651071358228/topology b/ksqldb-functional-tests/src/test/resources/historical_plans/joins_-_table_table_left_join_-_PROTOBUF_-_PROTOBUF_NOSR/7.3.0_1651071358228/topology new file mode 100644 index 000000000000..3429d44ba439 --- /dev/null +++ b/ksqldb-functional-tests/src/test/resources/historical_plans/joins_-_table_table_left_join_-_PROTOBUF_-_PROTOBUF_NOSR/7.3.0_1651071358228/topology @@ -0,0 +1,48 @@ +Topologies: + Sub-topology: 0 + Source: KSTREAM-SOURCE-0000000001 (topics: [left_topic]) + --> KTABLE-SOURCE-0000000002 + Source: KSTREAM-SOURCE-0000000007 (topics: [right_topic]) + --> KTABLE-SOURCE-0000000008 + Processor: KTABLE-SOURCE-0000000002 (stores: []) + --> KTABLE-TRANSFORMVALUES-0000000003 + <-- KSTREAM-SOURCE-0000000001 + Processor: KTABLE-SOURCE-0000000008 (stores: []) + --> KTABLE-TRANSFORMVALUES-0000000009 + <-- KSTREAM-SOURCE-0000000007 + Processor: KTABLE-TRANSFORMVALUES-0000000003 (stores: [KafkaTopic_Left-Reduce]) + --> KTABLE-TRANSFORMVALUES-0000000004 + <-- KTABLE-SOURCE-0000000002 + Processor: KTABLE-TRANSFORMVALUES-0000000009 (stores: [KafkaTopic_Right-Reduce]) + --> KTABLE-TRANSFORMVALUES-0000000010 + <-- KTABLE-SOURCE-0000000008 + Processor: KTABLE-TRANSFORMVALUES-0000000004 (stores: []) + --> PrependAliasLeft + <-- KTABLE-TRANSFORMVALUES-0000000003 + Processor: KTABLE-TRANSFORMVALUES-0000000010 (stores: []) + --> PrependAliasRight + <-- KTABLE-TRANSFORMVALUES-0000000009 + Processor: PrependAliasLeft (stores: []) + --> KTABLE-JOINTHIS-0000000013 + <-- KTABLE-TRANSFORMVALUES-0000000004 + Processor: PrependAliasRight (stores: []) + --> KTABLE-JOINOTHER-0000000014 + <-- KTABLE-TRANSFORMVALUES-0000000010 + Processor: KTABLE-JOINOTHER-0000000014 (stores: [KafkaTopic_Left-Reduce]) + --> KTABLE-MERGE-0000000012 + <-- PrependAliasRight + Processor: KTABLE-JOINTHIS-0000000013 (stores: [KafkaTopic_Right-Reduce]) + --> KTABLE-MERGE-0000000012 + <-- PrependAliasLeft + Processor: KTABLE-MERGE-0000000012 (stores: []) + --> KTABLE-TRANSFORMVALUES-0000000015 + <-- KTABLE-JOINTHIS-0000000013, KTABLE-JOINOTHER-0000000014 + Processor: KTABLE-TRANSFORMVALUES-0000000015 (stores: [Project-Last]) + --> KTABLE-TOSTREAM-0000000016 + <-- KTABLE-MERGE-0000000012 + Processor: KTABLE-TOSTREAM-0000000016 (stores: []) + --> KSTREAM-SINK-0000000017 + <-- KTABLE-TRANSFORMVALUES-0000000015 + Sink: KSTREAM-SINK-0000000017 (topic: OUTPUT) + <-- KTABLE-TOSTREAM-0000000016 + diff --git a/ksqldb-functional-tests/src/test/resources/historical_plans/joins_-_table_table_outer_join_-_PROTOBUF_-_PROTOBUF/7.3.0_1651071385572/plan.json b/ksqldb-functional-tests/src/test/resources/historical_plans/joins_-_table_table_outer_join_-_PROTOBUF_-_PROTOBUF/7.3.0_1651071385572/plan.json new file mode 100644 index 000000000000..f034b03ea808 --- /dev/null +++ b/ksqldb-functional-tests/src/test/resources/historical_plans/joins_-_table_table_outer_join_-_PROTOBUF_-_PROTOBUF/7.3.0_1651071385572/plan.json @@ -0,0 +1,342 @@ +{ + "plan" : [ { + "@type" : "ksqlPlanV1", + "statementText" : "CREATE TABLE TEST (ID BIGINT PRIMARY KEY, NAME STRING, VALUE BIGINT) WITH (KAFKA_TOPIC='left_topic', KEY_FORMAT='KAFKA', VALUE_FORMAT='PROTOBUF');", + "ddlCommand" : { + "@type" : "createTableV1", + "sourceName" : "TEST", + "schema" : "`ID` BIGINT KEY, `NAME` STRING, `VALUE` BIGINT", + "timestampColumn" : null, + "topicName" : "left_topic", + "formats" : { + "keyFormat" : { + "format" : "KAFKA", + "properties" : { } + }, + "valueFormat" : { + "format" : "PROTOBUF", + "properties" : { + "unwrapPrimitives" : "true" + } + } + }, + "windowInfo" : null, + "orReplace" : false, + "isSource" : false + }, + "queryPlan" : null + }, { + "@type" : "ksqlPlanV1", + "statementText" : "CREATE TABLE TEST_TABLE (ID BIGINT PRIMARY KEY, F1 STRING, F2 BIGINT) WITH (KAFKA_TOPIC='right_topic', KEY_FORMAT='KAFKA', VALUE_FORMAT='PROTOBUF');", + "ddlCommand" : { + "@type" : "createTableV1", + "sourceName" : "TEST_TABLE", + "schema" : "`ID` BIGINT KEY, `F1` STRING, `F2` BIGINT", + "timestampColumn" : null, + "topicName" : "right_topic", + "formats" : { + "keyFormat" : { + "format" : "KAFKA", + "properties" : { } + }, + "valueFormat" : { + "format" : "PROTOBUF", + "properties" : { + "unwrapPrimitives" : "true" + } + } + }, + "windowInfo" : null, + "orReplace" : false, + "isSource" : false + }, + "queryPlan" : null + }, { + "@type" : "ksqlPlanV1", + "statementText" : "CREATE TABLE OUTER_JOIN AS SELECT\n ROWKEY ID,\n T.ID T_ID,\n TT.ID TT_ID,\n T.NAME NAME,\n T.VALUE VALUE,\n TT.F1 F1,\n TT.F2 F2\nFROM TEST T\nFULL OUTER JOIN TEST_TABLE TT ON ((T.ID = TT.ID))\nEMIT CHANGES", + "ddlCommand" : { + "@type" : "createTableV1", + "sourceName" : "OUTER_JOIN", + "schema" : "`ID` BIGINT KEY, `T_ID` BIGINT, `TT_ID` BIGINT, `NAME` STRING, `VALUE` BIGINT, `F1` STRING, `F2` BIGINT", + "timestampColumn" : null, + "topicName" : "OUTER_JOIN", + "formats" : { + "keyFormat" : { + "format" : "KAFKA", + "properties" : { } + }, + "valueFormat" : { + "format" : "PROTOBUF", + "properties" : { + "unwrapPrimitives" : "true" + } + } + }, + "windowInfo" : null, + "orReplace" : false, + "isSource" : false + }, + "queryPlan" : { + "sources" : [ "TEST", "TEST_TABLE" ], + "sink" : "OUTER_JOIN", + "physicalPlan" : { + "@type" : "tableSinkV1", + "properties" : { + "queryContext" : "OUTER_JOIN" + }, + "source" : { + "@type" : "tableSelectV1", + "properties" : { + "queryContext" : "Project" + }, + "source" : { + "@type" : "tableTableJoinV1", + "properties" : { + "queryContext" : "Join" + }, + "joinType" : "OUTER", + "leftSource" : { + "@type" : "tableSelectV1", + "properties" : { + "queryContext" : "PrependAliasLeft" + }, + "source" : { + "@type" : "tableSourceV2", + "properties" : { + "queryContext" : "KafkaTopic_Left/Source" + }, + "topicName" : "left_topic", + "formats" : { + "keyFormat" : { + "format" : "KAFKA", + "properties" : { } + }, + "valueFormat" : { + "format" : "PROTOBUF", + "properties" : { + "unwrapPrimitives" : "true" + } + } + }, + "timestampColumn" : null, + "sourceSchema" : "`ID` BIGINT KEY, `NAME` STRING, `VALUE` BIGINT", + "pseudoColumnVersion" : 1, + "stateStoreFormats" : { + "keyFormat" : { + "format" : "KAFKA", + "properties" : { } + }, + "valueFormat" : { + "format" : "PROTOBUF", + "properties" : { + "unwrapPrimitives" : "true" + } + } + } + }, + "keyColumnNames" : [ "T_ID" ], + "selectExpressions" : [ "NAME AS T_NAME", "VALUE AS T_VALUE", "ROWTIME AS T_ROWTIME", "ROWPARTITION AS T_ROWPARTITION", "ROWOFFSET AS T_ROWOFFSET", "ID AS T_ID" ], + "internalFormats" : { + "keyFormat" : { + "format" : "KAFKA", + "properties" : { } + }, + "valueFormat" : { + "format" : "PROTOBUF", + "properties" : { + "unwrapPrimitives" : "true" + } + } + } + }, + "rightSource" : { + "@type" : "tableSelectV1", + "properties" : { + "queryContext" : "PrependAliasRight" + }, + "source" : { + "@type" : "tableSourceV2", + "properties" : { + "queryContext" : "KafkaTopic_Right/Source" + }, + "topicName" : "right_topic", + "formats" : { + "keyFormat" : { + "format" : "KAFKA", + "properties" : { } + }, + "valueFormat" : { + "format" : "PROTOBUF", + "properties" : { + "unwrapPrimitives" : "true" + } + } + }, + "timestampColumn" : null, + "sourceSchema" : "`ID` BIGINT KEY, `F1` STRING, `F2` BIGINT", + "pseudoColumnVersion" : 1, + "stateStoreFormats" : { + "keyFormat" : { + "format" : "KAFKA", + "properties" : { } + }, + "valueFormat" : { + "format" : "PROTOBUF", + "properties" : { + "unwrapPrimitives" : "true" + } + } + } + }, + "keyColumnNames" : [ "TT_ID" ], + "selectExpressions" : [ "F1 AS TT_F1", "F2 AS TT_F2", "ROWTIME AS TT_ROWTIME", "ROWPARTITION AS TT_ROWPARTITION", "ROWOFFSET AS TT_ROWOFFSET", "ID AS TT_ID" ], + "internalFormats" : { + "keyFormat" : { + "format" : "KAFKA", + "properties" : { } + }, + "valueFormat" : { + "format" : "PROTOBUF", + "properties" : { + "unwrapPrimitives" : "true" + } + } + } + }, + "keyColName" : "ROWKEY" + }, + "keyColumnNames" : [ "ID" ], + "selectExpressions" : [ "T_ID AS T_ID", "TT_ID AS TT_ID", "T_NAME AS NAME", "T_VALUE AS VALUE", "TT_F1 AS F1", "TT_F2 AS F2" ], + "internalFormats" : { + "keyFormat" : { + "format" : "KAFKA", + "properties" : { } + }, + "valueFormat" : { + "format" : "PROTOBUF", + "properties" : { + "unwrapPrimitives" : "true" + } + } + } + }, + "formats" : { + "keyFormat" : { + "format" : "KAFKA", + "properties" : { } + }, + "valueFormat" : { + "format" : "PROTOBUF", + "properties" : { + "unwrapPrimitives" : "true" + } + } + }, + "topicName" : "OUTER_JOIN", + "timestampColumn" : null + }, + "queryId" : "CTAS_OUTER_JOIN_0", + "runtimeId" : null + } + } ], + "configs" : { + "ksql.extension.dir" : "ext", + "ksql.streams.cache.max.bytes.buffering" : "0", + "metric.reporters" : "", + "ksql.query.status.running.threshold.seconds" : "300", + "ksql.connect.basic.auth.credentials.reload" : "false", + "ksql.output.topic.name.prefix" : "", + "ksql.query.pull.stream.enabled" : "true", + "ksql.query.push.v2.interpreter.enabled" : "true", + "ksql.queryanonymizer.logs_enabled" : "true", + "ksql.variable.substitution.enable" : "true", + "ksql.streams.shutdown.timeout.ms" : "300000", + "ksql.query.pull.max.allowed.offset.lag" : "9223372036854775807", + "ksql.query.pull.max.qps" : "2147483647", + "ksql.access.validator.enable" : "auto", + "ksql.streams.bootstrap.servers" : "localhost:0", + "ksql.query.pull.metrics.enabled" : "true", + "ksql.metrics.extension" : null, + "ksql.query.push.v2.alos.enabled" : "true", + "ksql.query.push.v2.max.hourly.bandwidth.megabytes" : "2147483647", + "ksql.query.pull.range.scan.enabled" : "true", + "ksql.transient.query.cleanup.service.initial.delay.seconds" : "600", + "ksql.hidden.topics" : "_confluent.*,__confluent.*,_schemas,__consumer_offsets,__transaction_state,connect-configs,connect-offsets,connect-status,connect-statuses", + "ksql.lambdas.enabled" : "true", + "ksql.source.table.materialization.enabled" : "true", + "ksql.query.pull.max.hourly.bandwidth.megabytes" : "2147483647", + "ksql.sink.window.change.log.additional.retention" : "1000000", + "ksql.query.persistent.active.limit" : "2147483647", + "ksql.persistence.wrap.single.values" : null, + "ksql.query.transient.max.bytes.buffering.total" : "-1", + "ksql.connect.basic.auth.credentials.source" : "NONE", + "ksql.schema.registry.url" : "schema_registry.url:0", + "ksql.properties.overrides.denylist" : "", + "ksql.service.id" : "some.ksql.service.id", + "ksql.query.push.v2.max.catchup.consumers" : "5", + "ksql.query.push.v2.enabled" : "false", + "ksql.transient.query.cleanup.service.enable" : "true", + "ksql.query.push.v2.metrics.enabled" : "true", + "ksql.rowpartition.rowoffset.enabled" : "true", + "ksql.streams.commit.interval.ms" : "2000", + "ksql.query.pull.table.scan.enabled" : "true", + "ksql.streams.auto.commit.interval.ms" : "0", + "ksql.streams.topology.optimization" : "all", + "ksql.endpoint.migrate.query" : "true", + "ksql.query.push.v2.registry.installed" : "false", + "ksql.streams.num.stream.threads" : "4", + "ksql.metrics.tags.custom" : "", + "ksql.query.push.v2.catchup.consumer.msg.window" : "50", + "ksql.runtime.feature.shared.enabled" : "false", + "ksql.udf.collect.metrics" : "false", + "ksql.new.query.planner.enabled" : "false", + "ksql.connect.request.headers.plugin" : null, + "ksql.security.extension.class" : null, + "ksql.transient.prefix" : "transient_", + "ksql.headers.columns.enabled" : "true", + "ksql.streams.default.deserialization.exception.handler" : "io.confluent.ksql.errors.LogMetricAndContinueExceptionHandler", + "ksql.connect.request.timeout.ms" : "5000", + "ksql.query.pull.enable.standby.reads" : "false", + "ksql.persistence.default.format.key" : "KAFKA", + "ksql.query.persistent.max.bytes.buffering.total" : "-1", + "ksql.query.error.max.queue.size" : "10", + "ksql.query.cleanup.shutdown.timeout.ms" : "30000", + "ksql.internal.topic.min.insync.replicas" : "1", + "ksql.internal.topic.replicas" : "1", + "ksql.insert.into.values.enabled" : "true", + "ksql.queryanonymizer.cluster_namespace" : null, + "ksql.create.or.replace.enabled" : "true", + "ksql.shared.runtimes.count" : "2", + "ksql.cast.strings.preserve.nulls" : "true", + "ksql.authorization.cache.max.entries" : "10000", + "ksql.pull.queries.enable" : "true", + "ksql.transient.query.cleanup.service.period.seconds" : "600", + "ksql.suppress.enabled" : "false", + "ksql.readonly.topics" : "_confluent.*,__confluent.*,_schemas,__consumer_offsets,__transaction_state,connect-configs,connect-offsets,connect-status,connect-statuses", + "ksql.connect.basic.auth.credentials.file" : "", + "ksql.authorization.cache.expiry.time.secs" : "30", + "ksql.query.retry.backoff.initial.ms" : "15000", + "ksql.query.pull.max.concurrent.requests" : "2147483647", + "ksql.streams.auto.offset.reset" : "earliest", + "ksql.connect.url" : "http://localhost:8083", + "ksql.query.push.v2.new.latest.delay.ms" : "5000", + "ksql.query.push.v2.latest.reset.age.ms" : "30000", + "ksql.streams.default.production.exception.handler" : "io.confluent.ksql.errors.ProductionExceptionHandlerUtil$LogAndFailProductionExceptionHandler", + "ksql.query.pull.interpreter.enabled" : "true", + "ksql.query.pull.limit.clause.enabled" : "true", + "ksql.query.pull.router.thread.pool.size" : "50", + "ksql.query.push.v2.continuation.tokens.enabled" : "false", + "ksql.query.retry.backoff.max.ms" : "900000", + "ksql.timestamp.throw.on.invalid" : "false", + "ksql.persistence.default.format.value" : null, + "ksql.udfs.enabled" : "true", + "ksql.udf.enable.security.manager" : "true", + "ksql.connect.worker.config" : "", + "ksql.nested.error.set.null" : "true", + "ksql.query.pull.thread.pool.size" : "50", + "ksql.persistent.prefix" : "query_", + "ksql.metastore.backup.location" : "", + "ksql.error.classifier.regex" : "", + "ksql.suppress.buffer.size.bytes" : "-1" + } +} \ No newline at end of file diff --git a/ksqldb-functional-tests/src/test/resources/historical_plans/joins_-_table_table_outer_join_-_PROTOBUF_-_PROTOBUF/7.3.0_1651071385572/spec.json b/ksqldb-functional-tests/src/test/resources/historical_plans/joins_-_table_table_outer_join_-_PROTOBUF_-_PROTOBUF/7.3.0_1651071385572/spec.json new file mode 100644 index 000000000000..296d5d7d8eec --- /dev/null +++ b/ksqldb-functional-tests/src/test/resources/historical_plans/joins_-_table_table_outer_join_-_PROTOBUF_-_PROTOBUF/7.3.0_1651071385572/spec.json @@ -0,0 +1,381 @@ +{ + "version" : "7.3.0", + "timestamp" : 1651071385572, + "path" : "query-validation-tests/joins.json", + "schemas" : { + "CTAS_OUTER_JOIN_0.Project" : { + "schema" : "`ID` BIGINT KEY, `T_ID` BIGINT, `TT_ID` BIGINT, `NAME` STRING, `VALUE` BIGINT, `F1` STRING, `F2` BIGINT", + "keyFormat" : { + "format" : "KAFKA" + }, + "valueFormat" : { + "format" : "PROTOBUF", + "properties" : { + "unwrapPrimitives" : "true" + } + } + }, + "CTAS_OUTER_JOIN_0.PrependAliasRight" : { + "schema" : "`TT_ID` BIGINT KEY, `TT_F1` STRING, `TT_F2` BIGINT, `TT_ROWTIME` BIGINT, `TT_ROWPARTITION` INTEGER, `TT_ROWOFFSET` BIGINT, `TT_ID` BIGINT", + "keyFormat" : { + "format" : "KAFKA" + }, + "valueFormat" : { + "format" : "PROTOBUF", + "properties" : { + "unwrapPrimitives" : "true" + } + } + }, + "CTAS_OUTER_JOIN_0.KafkaTopic_Right.Source" : { + "schema" : "`ID` BIGINT KEY, `F1` STRING, `F2` BIGINT", + "keyFormat" : { + "format" : "KAFKA" + }, + "valueFormat" : { + "format" : "PROTOBUF", + "properties" : { + "unwrapPrimitives" : "true" + } + } + }, + "CTAS_OUTER_JOIN_0.OUTER_JOIN" : { + "schema" : "`ID` BIGINT KEY, `T_ID` BIGINT, `TT_ID` BIGINT, `NAME` STRING, `VALUE` BIGINT, `F1` STRING, `F2` BIGINT", + "keyFormat" : { + "format" : "KAFKA" + }, + "valueFormat" : { + "format" : "PROTOBUF", + "properties" : { + "unwrapPrimitives" : "true" + } + } + }, + "CTAS_OUTER_JOIN_0.PrependAliasLeft" : { + "schema" : "`T_ID` BIGINT KEY, `T_NAME` STRING, `T_VALUE` BIGINT, `T_ROWTIME` BIGINT, `T_ROWPARTITION` INTEGER, `T_ROWOFFSET` BIGINT, `T_ID` BIGINT", + "keyFormat" : { + "format" : "KAFKA" + }, + "valueFormat" : { + "format" : "PROTOBUF", + "properties" : { + "unwrapPrimitives" : "true" + } + } + }, + "CTAS_OUTER_JOIN_0.KafkaTopic_Left.Source" : { + "schema" : "`ID` BIGINT KEY, `NAME` STRING, `VALUE` BIGINT", + "keyFormat" : { + "format" : "KAFKA" + }, + "valueFormat" : { + "format" : "PROTOBUF", + "properties" : { + "unwrapPrimitives" : "true" + } + } + }, + "CTAS_OUTER_JOIN_0.KafkaTopic_Right.Source.Materialized" : { + "schema" : "`ID` BIGINT KEY, `F1` STRING, `F2` BIGINT, `ROWPARTITION` INTEGER, `ROWOFFSET` BIGINT", + "keyFormat" : { + "format" : "KAFKA" + }, + "valueFormat" : { + "format" : "PROTOBUF", + "properties" : { + "unwrapPrimitives" : "true" + } + } + }, + "CTAS_OUTER_JOIN_0.KafkaTopic_Left.Source.Materialized" : { + "schema" : "`ID` BIGINT KEY, `NAME` STRING, `VALUE` BIGINT, `ROWPARTITION` INTEGER, `ROWOFFSET` BIGINT", + "keyFormat" : { + "format" : "KAFKA" + }, + "valueFormat" : { + "format" : "PROTOBUF", + "properties" : { + "unwrapPrimitives" : "true" + } + } + } + }, + "testCase" : { + "name" : "table table outer join - PROTOBUF - PROTOBUF", + "inputs" : [ { + "topic" : "left_topic", + "key" : 1, + "value" : { + "NAME" : "zero", + "VALUE" : 0 + }, + "timestamp" : 0 + }, { + "topic" : "right_topic", + "key" : 1, + "value" : { + "F1" : "blah", + "F2" : 50 + }, + "timestamp" : 10000 + }, { + "topic" : "left_topic", + "key" : 10, + "value" : { + "NAME" : "100", + "VALUE" : 5 + }, + "timestamp" : 11000 + }, { + "topic" : "left_topic", + "key" : 1, + "value" : { + "NAME" : "foo", + "VALUE" : 100 + }, + "timestamp" : 13000 + }, { + "topic" : "right_topic", + "key" : 1, + "value" : { + "F1" : "a", + "F2" : 10 + }, + "timestamp" : 15000 + }, { + "topic" : "right_topic", + "key" : 15, + "value" : { + "F1" : "c", + "F2" : 20 + }, + "timestamp" : 15500 + }, { + "topic" : "left_topic", + "key" : 1, + "value" : { + "NAME" : "bar", + "VALUE" : 99 + }, + "timestamp" : 16000 + } ], + "outputs" : [ { + "topic" : "OUTER_JOIN", + "key" : 1, + "value" : { + "T_ID" : 1, + "TT_ID" : 0, + "NAME" : "zero", + "VALUE" : 0, + "F1" : "", + "F2" : 0 + }, + "timestamp" : 0 + }, { + "topic" : "OUTER_JOIN", + "key" : 1, + "value" : { + "T_ID" : 1, + "TT_ID" : 1, + "NAME" : "zero", + "VALUE" : 0, + "F1" : "blah", + "F2" : 50 + }, + "timestamp" : 10000 + }, { + "topic" : "OUTER_JOIN", + "key" : 10, + "value" : { + "T_ID" : 10, + "TT_ID" : 0, + "NAME" : "100", + "VALUE" : 5, + "F1" : "", + "F2" : 0 + }, + "timestamp" : 11000 + }, { + "topic" : "OUTER_JOIN", + "key" : 1, + "value" : { + "T_ID" : 1, + "TT_ID" : 1, + "NAME" : "foo", + "VALUE" : 100, + "F1" : "blah", + "F2" : 50 + }, + "timestamp" : 13000 + }, { + "topic" : "OUTER_JOIN", + "key" : 1, + "value" : { + "T_ID" : 1, + "TT_ID" : 1, + "NAME" : "foo", + "VALUE" : 100, + "F1" : "a", + "F2" : 10 + }, + "timestamp" : 15000 + }, { + "topic" : "OUTER_JOIN", + "key" : 15, + "value" : { + "T_ID" : 0, + "TT_ID" : 15, + "NAME" : "", + "VALUE" : 0, + "F1" : "c", + "F2" : 20 + }, + "timestamp" : 15500 + }, { + "topic" : "OUTER_JOIN", + "key" : 1, + "value" : { + "T_ID" : 1, + "TT_ID" : 1, + "NAME" : "bar", + "VALUE" : 99, + "F1" : "a", + "F2" : 10 + }, + "timestamp" : 16000 + } ], + "topics" : [ { + "name" : "OUTER_JOIN", + "replicas" : 1, + "numPartitions" : 4 + }, { + "name" : "right_topic", + "valueSchema" : "syntax = \"proto3\";\n\nmessage ConnectDefault1 {\n string F1 = 1;\n int64 F2 = 2;\n}\n", + "valueFormat" : "PROTOBUF", + "replicas" : 1, + "numPartitions" : 4 + }, { + "name" : "left_topic", + "valueSchema" : "syntax = \"proto3\";\n\nmessage ConnectDefault1 {\n string NAME = 1;\n int64 VALUE = 2;\n}\n", + "valueFormat" : "PROTOBUF", + "replicas" : 1, + "numPartitions" : 4 + } ], + "statements" : [ "CREATE TABLE TEST (id BIGINT PRIMARY KEY, name VARCHAR, value BIGINT) WITH (kafka_topic='left_topic', value_format='PROTOBUF');", "CREATE TABLE TEST_TABLE (id BIGINT PRIMARY KEY, f1 VARCHAR, f2 BIGINT) WITH (kafka_topic='right_topic', value_format='PROTOBUF');", "CREATE TABLE OUTER_JOIN as SELECT ROWKEY AS ID, t.id, tt.id, name, value, f1, f2 FROM test t FULL OUTER join TEST_TABLE tt on t.id = tt.id;" ], + "post" : { + "sources" : [ { + "name" : "OUTER_JOIN", + "type" : "TABLE", + "schema" : "`ID` BIGINT KEY, `T_ID` BIGINT, `TT_ID` BIGINT, `NAME` STRING, `VALUE` BIGINT, `F1` STRING, `F2` BIGINT", + "keyFormat" : { + "format" : "KAFKA" + }, + "valueFormat" : "PROTOBUF", + "keyFeatures" : [ ], + "valueFeatures" : [ ], + "isSource" : false + }, { + "name" : "TEST", + "type" : "TABLE", + "schema" : "`ID` BIGINT KEY, `NAME` STRING, `VALUE` BIGINT", + "keyFormat" : { + "format" : "KAFKA" + }, + "valueFormat" : "PROTOBUF", + "keyFeatures" : [ ], + "valueFeatures" : [ ], + "isSource" : false + }, { + "name" : "TEST_TABLE", + "type" : "TABLE", + "schema" : "`ID` BIGINT KEY, `F1` STRING, `F2` BIGINT", + "keyFormat" : { + "format" : "KAFKA" + }, + "valueFormat" : "PROTOBUF", + "keyFeatures" : [ ], + "valueFeatures" : [ ], + "isSource" : false + } ], + "topics" : { + "topics" : [ { + "name" : "OUTER_JOIN", + "keyFormat" : { + "format" : "KAFKA" + }, + "valueFormat" : { + "format" : "PROTOBUF", + "properties" : { + "unwrapPrimitives" : "true" + } + }, + "partitions" : 4, + "valueSchema" : "syntax = \"proto3\";\n\nmessage ConnectDefault1 {\n int64 T_ID = 1;\n int64 TT_ID = 2;\n string NAME = 3;\n int64 VALUE = 4;\n string F1 = 5;\n int64 F2 = 6;\n}\n" + }, { + "name" : "_confluent-ksql-some.ksql.service.idquery_CTAS_OUTER_JOIN_0-KafkaTopic_Right-Reduce-changelog", + "keyFormat" : { + "format" : "KAFKA" + }, + "valueFormat" : { + "format" : "PROTOBUF", + "properties" : { + "unwrapPrimitives" : "true" + } + }, + "valueSchema" : "syntax = \"proto3\";\n\nmessage ConnectDefault1 {\n string F1 = 1;\n int64 F2 = 2;\n int32 ROWPARTITION = 3;\n int64 ROWOFFSET = 4;\n}\n" + }, { + "name" : "left_topic", + "keyFormat" : { + "format" : "KAFKA" + }, + "valueFormat" : { + "format" : "PROTOBUF", + "properties" : { + "unwrapPrimitives" : "true" + } + }, + "partitions" : 4, + "valueSchema" : "syntax = \"proto3\";\n\nmessage ConnectDefault1 {\n string NAME = 1;\n int64 VALUE = 2;\n}\n" + }, { + "name" : "_confluent-ksql-some.ksql.service.idquery_CTAS_OUTER_JOIN_0-Project-Last-changelog", + "keyFormat" : { + "format" : "KAFKA" + }, + "valueFormat" : { + "format" : "PROTOBUF", + "properties" : { + "unwrapPrimitives" : "true" + } + }, + "valueSchema" : "syntax = \"proto3\";\n\nmessage ConnectDefault1 {\n int64 T_ID = 1;\n int64 TT_ID = 2;\n string NAME = 3;\n int64 VALUE = 4;\n string F1 = 5;\n int64 F2 = 6;\n}\n" + }, { + "name" : "right_topic", + "keyFormat" : { + "format" : "KAFKA" + }, + "valueFormat" : { + "format" : "PROTOBUF", + "properties" : { + "unwrapPrimitives" : "true" + } + }, + "partitions" : 4, + "valueSchema" : "syntax = \"proto3\";\n\nmessage ConnectDefault1 {\n string F1 = 1;\n int64 F2 = 2;\n}\n" + }, { + "name" : "_confluent-ksql-some.ksql.service.idquery_CTAS_OUTER_JOIN_0-KafkaTopic_Left-Reduce-changelog", + "keyFormat" : { + "format" : "KAFKA" + }, + "valueFormat" : { + "format" : "PROTOBUF", + "properties" : { + "unwrapPrimitives" : "true" + } + }, + "valueSchema" : "syntax = \"proto3\";\n\nmessage ConnectDefault1 {\n string NAME = 1;\n int64 VALUE = 2;\n int32 ROWPARTITION = 3;\n int64 ROWOFFSET = 4;\n}\n" + } ], + "blackList" : ".*-repartition" + } + } + } +} \ No newline at end of file diff --git a/ksqldb-functional-tests/src/test/resources/historical_plans/joins_-_table_table_outer_join_-_PROTOBUF_-_PROTOBUF/7.3.0_1651071385572/topology b/ksqldb-functional-tests/src/test/resources/historical_plans/joins_-_table_table_outer_join_-_PROTOBUF_-_PROTOBUF/7.3.0_1651071385572/topology new file mode 100644 index 000000000000..0bce55709556 --- /dev/null +++ b/ksqldb-functional-tests/src/test/resources/historical_plans/joins_-_table_table_outer_join_-_PROTOBUF_-_PROTOBUF/7.3.0_1651071385572/topology @@ -0,0 +1,48 @@ +Topologies: + Sub-topology: 0 + Source: KSTREAM-SOURCE-0000000001 (topics: [left_topic]) + --> KTABLE-SOURCE-0000000002 + Source: KSTREAM-SOURCE-0000000007 (topics: [right_topic]) + --> KTABLE-SOURCE-0000000008 + Processor: KTABLE-SOURCE-0000000002 (stores: []) + --> KTABLE-TRANSFORMVALUES-0000000003 + <-- KSTREAM-SOURCE-0000000001 + Processor: KTABLE-SOURCE-0000000008 (stores: []) + --> KTABLE-TRANSFORMVALUES-0000000009 + <-- KSTREAM-SOURCE-0000000007 + Processor: KTABLE-TRANSFORMVALUES-0000000003 (stores: [KafkaTopic_Left-Reduce]) + --> KTABLE-TRANSFORMVALUES-0000000004 + <-- KTABLE-SOURCE-0000000002 + Processor: KTABLE-TRANSFORMVALUES-0000000009 (stores: [KafkaTopic_Right-Reduce]) + --> KTABLE-TRANSFORMVALUES-0000000010 + <-- KTABLE-SOURCE-0000000008 + Processor: KTABLE-TRANSFORMVALUES-0000000004 (stores: []) + --> PrependAliasLeft + <-- KTABLE-TRANSFORMVALUES-0000000003 + Processor: KTABLE-TRANSFORMVALUES-0000000010 (stores: []) + --> PrependAliasRight + <-- KTABLE-TRANSFORMVALUES-0000000009 + Processor: PrependAliasLeft (stores: []) + --> KTABLE-JOINTHIS-0000000013 + <-- KTABLE-TRANSFORMVALUES-0000000004 + Processor: PrependAliasRight (stores: []) + --> KTABLE-JOINOTHER-0000000014 + <-- KTABLE-TRANSFORMVALUES-0000000010 + Processor: KTABLE-JOINOTHER-0000000014 (stores: [KafkaTopic_Left-Reduce]) + --> KTABLE-MERGE-0000000012 + <-- PrependAliasRight + Processor: KTABLE-JOINTHIS-0000000013 (stores: [KafkaTopic_Right-Reduce]) + --> KTABLE-MERGE-0000000012 + <-- PrependAliasLeft + Processor: KTABLE-MERGE-0000000012 (stores: []) + --> KTABLE-TRANSFORMVALUES-0000000015 + <-- KTABLE-JOINTHIS-0000000013, KTABLE-JOINOTHER-0000000014 + Processor: KTABLE-TRANSFORMVALUES-0000000015 (stores: [Project-Last]) + --> KTABLE-TOSTREAM-0000000016 + <-- KTABLE-MERGE-0000000012 + Processor: KTABLE-TOSTREAM-0000000016 (stores: []) + --> KSTREAM-SINK-0000000017 + <-- KTABLE-TRANSFORMVALUES-0000000015 + Sink: KSTREAM-SINK-0000000017 (topic: OUTER_JOIN) + <-- KTABLE-TOSTREAM-0000000016 + diff --git a/ksqldb-functional-tests/src/test/resources/historical_plans/joins_-_table_table_outer_join_-_PROTOBUF_-_PROTOBUF_NOSR/7.3.0_1651071388051/plan.json b/ksqldb-functional-tests/src/test/resources/historical_plans/joins_-_table_table_outer_join_-_PROTOBUF_-_PROTOBUF_NOSR/7.3.0_1651071388051/plan.json new file mode 100644 index 000000000000..ac0d16fcf1aa --- /dev/null +++ b/ksqldb-functional-tests/src/test/resources/historical_plans/joins_-_table_table_outer_join_-_PROTOBUF_-_PROTOBUF_NOSR/7.3.0_1651071388051/plan.json @@ -0,0 +1,320 @@ +{ + "plan" : [ { + "@type" : "ksqlPlanV1", + "statementText" : "CREATE TABLE TEST (ID BIGINT PRIMARY KEY, NAME STRING, VALUE BIGINT) WITH (KAFKA_TOPIC='left_topic', KEY_FORMAT='KAFKA', VALUE_FORMAT='PROTOBUF_NOSR');", + "ddlCommand" : { + "@type" : "createTableV1", + "sourceName" : "TEST", + "schema" : "`ID` BIGINT KEY, `NAME` STRING, `VALUE` BIGINT", + "timestampColumn" : null, + "topicName" : "left_topic", + "formats" : { + "keyFormat" : { + "format" : "KAFKA", + "properties" : { } + }, + "valueFormat" : { + "format" : "PROTOBUF_NOSR", + "properties" : { } + } + }, + "windowInfo" : null, + "orReplace" : false, + "isSource" : false + }, + "queryPlan" : null + }, { + "@type" : "ksqlPlanV1", + "statementText" : "CREATE TABLE TEST_TABLE (ID BIGINT PRIMARY KEY, F1 STRING, F2 BIGINT) WITH (KAFKA_TOPIC='right_topic', KEY_FORMAT='KAFKA', VALUE_FORMAT='PROTOBUF_NOSR');", + "ddlCommand" : { + "@type" : "createTableV1", + "sourceName" : "TEST_TABLE", + "schema" : "`ID` BIGINT KEY, `F1` STRING, `F2` BIGINT", + "timestampColumn" : null, + "topicName" : "right_topic", + "formats" : { + "keyFormat" : { + "format" : "KAFKA", + "properties" : { } + }, + "valueFormat" : { + "format" : "PROTOBUF_NOSR", + "properties" : { } + } + }, + "windowInfo" : null, + "orReplace" : false, + "isSource" : false + }, + "queryPlan" : null + }, { + "@type" : "ksqlPlanV1", + "statementText" : "CREATE TABLE OUTER_JOIN AS SELECT\n ROWKEY ID,\n T.ID T_ID,\n TT.ID TT_ID,\n T.NAME NAME,\n T.VALUE VALUE,\n TT.F1 F1,\n TT.F2 F2\nFROM TEST T\nFULL OUTER JOIN TEST_TABLE TT ON ((T.ID = TT.ID))\nEMIT CHANGES", + "ddlCommand" : { + "@type" : "createTableV1", + "sourceName" : "OUTER_JOIN", + "schema" : "`ID` BIGINT KEY, `T_ID` BIGINT, `TT_ID` BIGINT, `NAME` STRING, `VALUE` BIGINT, `F1` STRING, `F2` BIGINT", + "timestampColumn" : null, + "topicName" : "OUTER_JOIN", + "formats" : { + "keyFormat" : { + "format" : "KAFKA", + "properties" : { } + }, + "valueFormat" : { + "format" : "PROTOBUF_NOSR", + "properties" : { } + } + }, + "windowInfo" : null, + "orReplace" : false, + "isSource" : false + }, + "queryPlan" : { + "sources" : [ "TEST", "TEST_TABLE" ], + "sink" : "OUTER_JOIN", + "physicalPlan" : { + "@type" : "tableSinkV1", + "properties" : { + "queryContext" : "OUTER_JOIN" + }, + "source" : { + "@type" : "tableSelectV1", + "properties" : { + "queryContext" : "Project" + }, + "source" : { + "@type" : "tableTableJoinV1", + "properties" : { + "queryContext" : "Join" + }, + "joinType" : "OUTER", + "leftSource" : { + "@type" : "tableSelectV1", + "properties" : { + "queryContext" : "PrependAliasLeft" + }, + "source" : { + "@type" : "tableSourceV2", + "properties" : { + "queryContext" : "KafkaTopic_Left/Source" + }, + "topicName" : "left_topic", + "formats" : { + "keyFormat" : { + "format" : "KAFKA", + "properties" : { } + }, + "valueFormat" : { + "format" : "PROTOBUF_NOSR", + "properties" : { } + } + }, + "timestampColumn" : null, + "sourceSchema" : "`ID` BIGINT KEY, `NAME` STRING, `VALUE` BIGINT", + "pseudoColumnVersion" : 1, + "stateStoreFormats" : { + "keyFormat" : { + "format" : "KAFKA", + "properties" : { } + }, + "valueFormat" : { + "format" : "PROTOBUF_NOSR", + "properties" : { } + } + } + }, + "keyColumnNames" : [ "T_ID" ], + "selectExpressions" : [ "NAME AS T_NAME", "VALUE AS T_VALUE", "ROWTIME AS T_ROWTIME", "ROWPARTITION AS T_ROWPARTITION", "ROWOFFSET AS T_ROWOFFSET", "ID AS T_ID" ], + "internalFormats" : { + "keyFormat" : { + "format" : "KAFKA", + "properties" : { } + }, + "valueFormat" : { + "format" : "PROTOBUF_NOSR", + "properties" : { } + } + } + }, + "rightSource" : { + "@type" : "tableSelectV1", + "properties" : { + "queryContext" : "PrependAliasRight" + }, + "source" : { + "@type" : "tableSourceV2", + "properties" : { + "queryContext" : "KafkaTopic_Right/Source" + }, + "topicName" : "right_topic", + "formats" : { + "keyFormat" : { + "format" : "KAFKA", + "properties" : { } + }, + "valueFormat" : { + "format" : "PROTOBUF_NOSR", + "properties" : { } + } + }, + "timestampColumn" : null, + "sourceSchema" : "`ID` BIGINT KEY, `F1` STRING, `F2` BIGINT", + "pseudoColumnVersion" : 1, + "stateStoreFormats" : { + "keyFormat" : { + "format" : "KAFKA", + "properties" : { } + }, + "valueFormat" : { + "format" : "PROTOBUF_NOSR", + "properties" : { } + } + } + }, + "keyColumnNames" : [ "TT_ID" ], + "selectExpressions" : [ "F1 AS TT_F1", "F2 AS TT_F2", "ROWTIME AS TT_ROWTIME", "ROWPARTITION AS TT_ROWPARTITION", "ROWOFFSET AS TT_ROWOFFSET", "ID AS TT_ID" ], + "internalFormats" : { + "keyFormat" : { + "format" : "KAFKA", + "properties" : { } + }, + "valueFormat" : { + "format" : "PROTOBUF_NOSR", + "properties" : { } + } + } + }, + "keyColName" : "ROWKEY" + }, + "keyColumnNames" : [ "ID" ], + "selectExpressions" : [ "T_ID AS T_ID", "TT_ID AS TT_ID", "T_NAME AS NAME", "T_VALUE AS VALUE", "TT_F1 AS F1", "TT_F2 AS F2" ], + "internalFormats" : { + "keyFormat" : { + "format" : "KAFKA", + "properties" : { } + }, + "valueFormat" : { + "format" : "PROTOBUF_NOSR", + "properties" : { } + } + } + }, + "formats" : { + "keyFormat" : { + "format" : "KAFKA", + "properties" : { } + }, + "valueFormat" : { + "format" : "PROTOBUF_NOSR", + "properties" : { } + } + }, + "topicName" : "OUTER_JOIN", + "timestampColumn" : null + }, + "queryId" : "CTAS_OUTER_JOIN_0", + "runtimeId" : null + } + } ], + "configs" : { + "ksql.extension.dir" : "ext", + "ksql.streams.cache.max.bytes.buffering" : "0", + "metric.reporters" : "", + "ksql.query.status.running.threshold.seconds" : "300", + "ksql.connect.basic.auth.credentials.reload" : "false", + "ksql.output.topic.name.prefix" : "", + "ksql.query.pull.stream.enabled" : "true", + "ksql.query.push.v2.interpreter.enabled" : "true", + "ksql.queryanonymizer.logs_enabled" : "true", + "ksql.variable.substitution.enable" : "true", + "ksql.streams.shutdown.timeout.ms" : "300000", + "ksql.query.pull.max.allowed.offset.lag" : "9223372036854775807", + "ksql.query.pull.max.qps" : "2147483647", + "ksql.access.validator.enable" : "auto", + "ksql.streams.bootstrap.servers" : "localhost:0", + "ksql.query.pull.metrics.enabled" : "true", + "ksql.metrics.extension" : null, + "ksql.query.push.v2.alos.enabled" : "true", + "ksql.query.push.v2.max.hourly.bandwidth.megabytes" : "2147483647", + "ksql.query.pull.range.scan.enabled" : "true", + "ksql.transient.query.cleanup.service.initial.delay.seconds" : "600", + "ksql.hidden.topics" : "_confluent.*,__confluent.*,_schemas,__consumer_offsets,__transaction_state,connect-configs,connect-offsets,connect-status,connect-statuses", + "ksql.lambdas.enabled" : "true", + "ksql.source.table.materialization.enabled" : "true", + "ksql.query.pull.max.hourly.bandwidth.megabytes" : "2147483647", + "ksql.sink.window.change.log.additional.retention" : "1000000", + "ksql.query.persistent.active.limit" : "2147483647", + "ksql.persistence.wrap.single.values" : null, + "ksql.query.transient.max.bytes.buffering.total" : "-1", + "ksql.connect.basic.auth.credentials.source" : "NONE", + "ksql.schema.registry.url" : "schema_registry.url:0", + "ksql.properties.overrides.denylist" : "", + "ksql.service.id" : "some.ksql.service.id", + "ksql.query.push.v2.max.catchup.consumers" : "5", + "ksql.query.push.v2.enabled" : "false", + "ksql.transient.query.cleanup.service.enable" : "true", + "ksql.query.push.v2.metrics.enabled" : "true", + "ksql.rowpartition.rowoffset.enabled" : "true", + "ksql.streams.commit.interval.ms" : "2000", + "ksql.query.pull.table.scan.enabled" : "true", + "ksql.streams.auto.commit.interval.ms" : "0", + "ksql.streams.topology.optimization" : "all", + "ksql.endpoint.migrate.query" : "true", + "ksql.query.push.v2.registry.installed" : "false", + "ksql.streams.num.stream.threads" : "4", + "ksql.metrics.tags.custom" : "", + "ksql.query.push.v2.catchup.consumer.msg.window" : "50", + "ksql.runtime.feature.shared.enabled" : "false", + "ksql.udf.collect.metrics" : "false", + "ksql.new.query.planner.enabled" : "false", + "ksql.connect.request.headers.plugin" : null, + "ksql.security.extension.class" : null, + "ksql.transient.prefix" : "transient_", + "ksql.headers.columns.enabled" : "true", + "ksql.streams.default.deserialization.exception.handler" : "io.confluent.ksql.errors.LogMetricAndContinueExceptionHandler", + "ksql.connect.request.timeout.ms" : "5000", + "ksql.query.pull.enable.standby.reads" : "false", + "ksql.persistence.default.format.key" : "KAFKA", + "ksql.query.persistent.max.bytes.buffering.total" : "-1", + "ksql.query.error.max.queue.size" : "10", + "ksql.query.cleanup.shutdown.timeout.ms" : "30000", + "ksql.internal.topic.min.insync.replicas" : "1", + "ksql.internal.topic.replicas" : "1", + "ksql.insert.into.values.enabled" : "true", + "ksql.queryanonymizer.cluster_namespace" : null, + "ksql.create.or.replace.enabled" : "true", + "ksql.shared.runtimes.count" : "2", + "ksql.cast.strings.preserve.nulls" : "true", + "ksql.authorization.cache.max.entries" : "10000", + "ksql.pull.queries.enable" : "true", + "ksql.transient.query.cleanup.service.period.seconds" : "600", + "ksql.suppress.enabled" : "false", + "ksql.readonly.topics" : "_confluent.*,__confluent.*,_schemas,__consumer_offsets,__transaction_state,connect-configs,connect-offsets,connect-status,connect-statuses", + "ksql.connect.basic.auth.credentials.file" : "", + "ksql.authorization.cache.expiry.time.secs" : "30", + "ksql.query.retry.backoff.initial.ms" : "15000", + "ksql.query.pull.max.concurrent.requests" : "2147483647", + "ksql.streams.auto.offset.reset" : "earliest", + "ksql.connect.url" : "http://localhost:8083", + "ksql.query.push.v2.new.latest.delay.ms" : "5000", + "ksql.query.push.v2.latest.reset.age.ms" : "30000", + "ksql.streams.default.production.exception.handler" : "io.confluent.ksql.errors.ProductionExceptionHandlerUtil$LogAndFailProductionExceptionHandler", + "ksql.query.pull.interpreter.enabled" : "true", + "ksql.query.pull.limit.clause.enabled" : "true", + "ksql.query.pull.router.thread.pool.size" : "50", + "ksql.query.push.v2.continuation.tokens.enabled" : "false", + "ksql.query.retry.backoff.max.ms" : "900000", + "ksql.timestamp.throw.on.invalid" : "false", + "ksql.persistence.default.format.value" : null, + "ksql.udfs.enabled" : "true", + "ksql.udf.enable.security.manager" : "true", + "ksql.connect.worker.config" : "", + "ksql.nested.error.set.null" : "true", + "ksql.query.pull.thread.pool.size" : "50", + "ksql.persistent.prefix" : "query_", + "ksql.metastore.backup.location" : "", + "ksql.error.classifier.regex" : "", + "ksql.suppress.buffer.size.bytes" : "-1" + } +} \ No newline at end of file diff --git a/ksqldb-functional-tests/src/test/resources/historical_plans/joins_-_table_table_outer_join_-_PROTOBUF_-_PROTOBUF_NOSR/7.3.0_1651071388051/spec.json b/ksqldb-functional-tests/src/test/resources/historical_plans/joins_-_table_table_outer_join_-_PROTOBUF_-_PROTOBUF_NOSR/7.3.0_1651071388051/spec.json new file mode 100644 index 000000000000..8a73475cbeb2 --- /dev/null +++ b/ksqldb-functional-tests/src/test/resources/historical_plans/joins_-_table_table_outer_join_-_PROTOBUF_-_PROTOBUF_NOSR/7.3.0_1651071388051/spec.json @@ -0,0 +1,329 @@ +{ + "version" : "7.3.0", + "timestamp" : 1651071388051, + "path" : "query-validation-tests/joins.json", + "schemas" : { + "CTAS_OUTER_JOIN_0.Project" : { + "schema" : "`ID` BIGINT KEY, `T_ID` BIGINT, `TT_ID` BIGINT, `NAME` STRING, `VALUE` BIGINT, `F1` STRING, `F2` BIGINT", + "keyFormat" : { + "format" : "KAFKA" + }, + "valueFormat" : { + "format" : "PROTOBUF_NOSR" + } + }, + "CTAS_OUTER_JOIN_0.PrependAliasRight" : { + "schema" : "`TT_ID` BIGINT KEY, `TT_F1` STRING, `TT_F2` BIGINT, `TT_ROWTIME` BIGINT, `TT_ROWPARTITION` INTEGER, `TT_ROWOFFSET` BIGINT, `TT_ID` BIGINT", + "keyFormat" : { + "format" : "KAFKA" + }, + "valueFormat" : { + "format" : "PROTOBUF_NOSR" + } + }, + "CTAS_OUTER_JOIN_0.KafkaTopic_Right.Source" : { + "schema" : "`ID` BIGINT KEY, `F1` STRING, `F2` BIGINT", + "keyFormat" : { + "format" : "KAFKA" + }, + "valueFormat" : { + "format" : "PROTOBUF_NOSR" + } + }, + "CTAS_OUTER_JOIN_0.OUTER_JOIN" : { + "schema" : "`ID` BIGINT KEY, `T_ID` BIGINT, `TT_ID` BIGINT, `NAME` STRING, `VALUE` BIGINT, `F1` STRING, `F2` BIGINT", + "keyFormat" : { + "format" : "KAFKA" + }, + "valueFormat" : { + "format" : "PROTOBUF_NOSR" + } + }, + "CTAS_OUTER_JOIN_0.PrependAliasLeft" : { + "schema" : "`T_ID` BIGINT KEY, `T_NAME` STRING, `T_VALUE` BIGINT, `T_ROWTIME` BIGINT, `T_ROWPARTITION` INTEGER, `T_ROWOFFSET` BIGINT, `T_ID` BIGINT", + "keyFormat" : { + "format" : "KAFKA" + }, + "valueFormat" : { + "format" : "PROTOBUF_NOSR" + } + }, + "CTAS_OUTER_JOIN_0.KafkaTopic_Left.Source" : { + "schema" : "`ID` BIGINT KEY, `NAME` STRING, `VALUE` BIGINT", + "keyFormat" : { + "format" : "KAFKA" + }, + "valueFormat" : { + "format" : "PROTOBUF_NOSR" + } + }, + "CTAS_OUTER_JOIN_0.KafkaTopic_Right.Source.Materialized" : { + "schema" : "`ID` BIGINT KEY, `F1` STRING, `F2` BIGINT, `ROWPARTITION` INTEGER, `ROWOFFSET` BIGINT", + "keyFormat" : { + "format" : "KAFKA" + }, + "valueFormat" : { + "format" : "PROTOBUF_NOSR" + } + }, + "CTAS_OUTER_JOIN_0.KafkaTopic_Left.Source.Materialized" : { + "schema" : "`ID` BIGINT KEY, `NAME` STRING, `VALUE` BIGINT, `ROWPARTITION` INTEGER, `ROWOFFSET` BIGINT", + "keyFormat" : { + "format" : "KAFKA" + }, + "valueFormat" : { + "format" : "PROTOBUF_NOSR" + } + } + }, + "testCase" : { + "name" : "table table outer join - PROTOBUF - PROTOBUF_NOSR", + "inputs" : [ { + "topic" : "left_topic", + "key" : 1, + "value" : { + "NAME" : "zero", + "VALUE" : 0 + }, + "timestamp" : 0 + }, { + "topic" : "right_topic", + "key" : 1, + "value" : { + "F1" : "blah", + "F2" : 50 + }, + "timestamp" : 10000 + }, { + "topic" : "left_topic", + "key" : 10, + "value" : { + "NAME" : "100", + "VALUE" : 5 + }, + "timestamp" : 11000 + }, { + "topic" : "left_topic", + "key" : 1, + "value" : { + "NAME" : "foo", + "VALUE" : 100 + }, + "timestamp" : 13000 + }, { + "topic" : "right_topic", + "key" : 1, + "value" : { + "F1" : "a", + "F2" : 10 + }, + "timestamp" : 15000 + }, { + "topic" : "right_topic", + "key" : 15, + "value" : { + "F1" : "c", + "F2" : 20 + }, + "timestamp" : 15500 + }, { + "topic" : "left_topic", + "key" : 1, + "value" : { + "NAME" : "bar", + "VALUE" : 99 + }, + "timestamp" : 16000 + } ], + "outputs" : [ { + "topic" : "OUTER_JOIN", + "key" : 1, + "value" : { + "T_ID" : 1, + "TT_ID" : 0, + "NAME" : "zero", + "VALUE" : 0, + "F1" : "", + "F2" : 0 + }, + "timestamp" : 0 + }, { + "topic" : "OUTER_JOIN", + "key" : 1, + "value" : { + "T_ID" : 1, + "TT_ID" : 1, + "NAME" : "zero", + "VALUE" : 0, + "F1" : "blah", + "F2" : 50 + }, + "timestamp" : 10000 + }, { + "topic" : "OUTER_JOIN", + "key" : 10, + "value" : { + "T_ID" : 10, + "TT_ID" : 0, + "NAME" : "100", + "VALUE" : 5, + "F1" : "", + "F2" : 0 + }, + "timestamp" : 11000 + }, { + "topic" : "OUTER_JOIN", + "key" : 1, + "value" : { + "T_ID" : 1, + "TT_ID" : 1, + "NAME" : "foo", + "VALUE" : 100, + "F1" : "blah", + "F2" : 50 + }, + "timestamp" : 13000 + }, { + "topic" : "OUTER_JOIN", + "key" : 1, + "value" : { + "T_ID" : 1, + "TT_ID" : 1, + "NAME" : "foo", + "VALUE" : 100, + "F1" : "a", + "F2" : 10 + }, + "timestamp" : 15000 + }, { + "topic" : "OUTER_JOIN", + "key" : 15, + "value" : { + "T_ID" : 0, + "TT_ID" : 15, + "NAME" : "", + "VALUE" : 0, + "F1" : "c", + "F2" : 20 + }, + "timestamp" : 15500 + }, { + "topic" : "OUTER_JOIN", + "key" : 1, + "value" : { + "T_ID" : 1, + "TT_ID" : 1, + "NAME" : "bar", + "VALUE" : 99, + "F1" : "a", + "F2" : 10 + }, + "timestamp" : 16000 + } ], + "topics" : [ { + "name" : "OUTER_JOIN", + "replicas" : 1, + "numPartitions" : 4 + }, { + "name" : "right_topic", + "replicas" : 1, + "numPartitions" : 4 + }, { + "name" : "left_topic", + "replicas" : 1, + "numPartitions" : 4 + } ], + "statements" : [ "CREATE TABLE TEST (id BIGINT PRIMARY KEY, name VARCHAR, value BIGINT) WITH (kafka_topic='left_topic', value_format='PROTOBUF_NOSR');", "CREATE TABLE TEST_TABLE (id BIGINT PRIMARY KEY, f1 VARCHAR, f2 BIGINT) WITH (kafka_topic='right_topic', value_format='PROTOBUF_NOSR');", "CREATE TABLE OUTER_JOIN as SELECT ROWKEY AS ID, t.id, tt.id, name, value, f1, f2 FROM test t FULL OUTER join TEST_TABLE tt on t.id = tt.id;" ], + "post" : { + "sources" : [ { + "name" : "OUTER_JOIN", + "type" : "TABLE", + "schema" : "`ID` BIGINT KEY, `T_ID` BIGINT, `TT_ID` BIGINT, `NAME` STRING, `VALUE` BIGINT, `F1` STRING, `F2` BIGINT", + "keyFormat" : { + "format" : "KAFKA" + }, + "valueFormat" : "PROTOBUF_NOSR", + "keyFeatures" : [ ], + "valueFeatures" : [ ], + "isSource" : false + }, { + "name" : "TEST", + "type" : "TABLE", + "schema" : "`ID` BIGINT KEY, `NAME` STRING, `VALUE` BIGINT", + "keyFormat" : { + "format" : "KAFKA" + }, + "valueFormat" : "PROTOBUF_NOSR", + "keyFeatures" : [ ], + "valueFeatures" : [ ], + "isSource" : false + }, { + "name" : "TEST_TABLE", + "type" : "TABLE", + "schema" : "`ID` BIGINT KEY, `F1` STRING, `F2` BIGINT", + "keyFormat" : { + "format" : "KAFKA" + }, + "valueFormat" : "PROTOBUF_NOSR", + "keyFeatures" : [ ], + "valueFeatures" : [ ], + "isSource" : false + } ], + "topics" : { + "topics" : [ { + "name" : "OUTER_JOIN", + "keyFormat" : { + "format" : "KAFKA" + }, + "valueFormat" : { + "format" : "PROTOBUF_NOSR" + }, + "partitions" : 4 + }, { + "name" : "_confluent-ksql-some.ksql.service.idquery_CTAS_OUTER_JOIN_0-KafkaTopic_Right-Reduce-changelog", + "keyFormat" : { + "format" : "KAFKA" + }, + "valueFormat" : { + "format" : "PROTOBUF_NOSR" + } + }, { + "name" : "left_topic", + "keyFormat" : { + "format" : "KAFKA" + }, + "valueFormat" : { + "format" : "PROTOBUF_NOSR" + }, + "partitions" : 4 + }, { + "name" : "_confluent-ksql-some.ksql.service.idquery_CTAS_OUTER_JOIN_0-Project-Last-changelog", + "keyFormat" : { + "format" : "KAFKA" + }, + "valueFormat" : { + "format" : "PROTOBUF_NOSR" + } + }, { + "name" : "right_topic", + "keyFormat" : { + "format" : "KAFKA" + }, + "valueFormat" : { + "format" : "PROTOBUF_NOSR" + }, + "partitions" : 4 + }, { + "name" : "_confluent-ksql-some.ksql.service.idquery_CTAS_OUTER_JOIN_0-KafkaTopic_Left-Reduce-changelog", + "keyFormat" : { + "format" : "KAFKA" + }, + "valueFormat" : { + "format" : "PROTOBUF_NOSR" + } + } ], + "blackList" : ".*-repartition" + } + } + } +} \ No newline at end of file diff --git a/ksqldb-functional-tests/src/test/resources/historical_plans/joins_-_table_table_outer_join_-_PROTOBUF_-_PROTOBUF_NOSR/7.3.0_1651071388051/topology b/ksqldb-functional-tests/src/test/resources/historical_plans/joins_-_table_table_outer_join_-_PROTOBUF_-_PROTOBUF_NOSR/7.3.0_1651071388051/topology new file mode 100644 index 000000000000..0bce55709556 --- /dev/null +++ b/ksqldb-functional-tests/src/test/resources/historical_plans/joins_-_table_table_outer_join_-_PROTOBUF_-_PROTOBUF_NOSR/7.3.0_1651071388051/topology @@ -0,0 +1,48 @@ +Topologies: + Sub-topology: 0 + Source: KSTREAM-SOURCE-0000000001 (topics: [left_topic]) + --> KTABLE-SOURCE-0000000002 + Source: KSTREAM-SOURCE-0000000007 (topics: [right_topic]) + --> KTABLE-SOURCE-0000000008 + Processor: KTABLE-SOURCE-0000000002 (stores: []) + --> KTABLE-TRANSFORMVALUES-0000000003 + <-- KSTREAM-SOURCE-0000000001 + Processor: KTABLE-SOURCE-0000000008 (stores: []) + --> KTABLE-TRANSFORMVALUES-0000000009 + <-- KSTREAM-SOURCE-0000000007 + Processor: KTABLE-TRANSFORMVALUES-0000000003 (stores: [KafkaTopic_Left-Reduce]) + --> KTABLE-TRANSFORMVALUES-0000000004 + <-- KTABLE-SOURCE-0000000002 + Processor: KTABLE-TRANSFORMVALUES-0000000009 (stores: [KafkaTopic_Right-Reduce]) + --> KTABLE-TRANSFORMVALUES-0000000010 + <-- KTABLE-SOURCE-0000000008 + Processor: KTABLE-TRANSFORMVALUES-0000000004 (stores: []) + --> PrependAliasLeft + <-- KTABLE-TRANSFORMVALUES-0000000003 + Processor: KTABLE-TRANSFORMVALUES-0000000010 (stores: []) + --> PrependAliasRight + <-- KTABLE-TRANSFORMVALUES-0000000009 + Processor: PrependAliasLeft (stores: []) + --> KTABLE-JOINTHIS-0000000013 + <-- KTABLE-TRANSFORMVALUES-0000000004 + Processor: PrependAliasRight (stores: []) + --> KTABLE-JOINOTHER-0000000014 + <-- KTABLE-TRANSFORMVALUES-0000000010 + Processor: KTABLE-JOINOTHER-0000000014 (stores: [KafkaTopic_Left-Reduce]) + --> KTABLE-MERGE-0000000012 + <-- PrependAliasRight + Processor: KTABLE-JOINTHIS-0000000013 (stores: [KafkaTopic_Right-Reduce]) + --> KTABLE-MERGE-0000000012 + <-- PrependAliasLeft + Processor: KTABLE-MERGE-0000000012 (stores: []) + --> KTABLE-TRANSFORMVALUES-0000000015 + <-- KTABLE-JOINTHIS-0000000013, KTABLE-JOINOTHER-0000000014 + Processor: KTABLE-TRANSFORMVALUES-0000000015 (stores: [Project-Last]) + --> KTABLE-TOSTREAM-0000000016 + <-- KTABLE-MERGE-0000000012 + Processor: KTABLE-TOSTREAM-0000000016 (stores: []) + --> KSTREAM-SINK-0000000017 + <-- KTABLE-TRANSFORMVALUES-0000000015 + Sink: KSTREAM-SINK-0000000017 (topic: OUTER_JOIN) + <-- KTABLE-TOSTREAM-0000000016 + diff --git a/ksqldb-functional-tests/src/test/resources/historical_plans/joins_-_table_table_right_join_-_PROTOBUF_-_PROTOBUF/7.3.0_1651071365034/plan.json b/ksqldb-functional-tests/src/test/resources/historical_plans/joins_-_table_table_right_join_-_PROTOBUF_-_PROTOBUF/7.3.0_1651071365034/plan.json new file mode 100644 index 000000000000..f32ed860700e --- /dev/null +++ b/ksqldb-functional-tests/src/test/resources/historical_plans/joins_-_table_table_right_join_-_PROTOBUF_-_PROTOBUF/7.3.0_1651071365034/plan.json @@ -0,0 +1,342 @@ +{ + "plan" : [ { + "@type" : "ksqlPlanV1", + "statementText" : "CREATE TABLE TEST (ID BIGINT PRIMARY KEY, NAME STRING, VALUE BIGINT) WITH (KAFKA_TOPIC='left_topic', KEY_FORMAT='KAFKA', VALUE_FORMAT='PROTOBUF');", + "ddlCommand" : { + "@type" : "createTableV1", + "sourceName" : "TEST", + "schema" : "`ID` BIGINT KEY, `NAME` STRING, `VALUE` BIGINT", + "timestampColumn" : null, + "topicName" : "left_topic", + "formats" : { + "keyFormat" : { + "format" : "KAFKA", + "properties" : { } + }, + "valueFormat" : { + "format" : "PROTOBUF", + "properties" : { + "unwrapPrimitives" : "true" + } + } + }, + "windowInfo" : null, + "orReplace" : false, + "isSource" : false + }, + "queryPlan" : null + }, { + "@type" : "ksqlPlanV1", + "statementText" : "CREATE TABLE TEST_TABLE (ID BIGINT PRIMARY KEY, F1 STRING, F2 BIGINT) WITH (KAFKA_TOPIC='right_topic', KEY_FORMAT='KAFKA', VALUE_FORMAT='PROTOBUF');", + "ddlCommand" : { + "@type" : "createTableV1", + "sourceName" : "TEST_TABLE", + "schema" : "`ID` BIGINT KEY, `F1` STRING, `F2` BIGINT", + "timestampColumn" : null, + "topicName" : "right_topic", + "formats" : { + "keyFormat" : { + "format" : "KAFKA", + "properties" : { } + }, + "valueFormat" : { + "format" : "PROTOBUF", + "properties" : { + "unwrapPrimitives" : "true" + } + } + }, + "windowInfo" : null, + "orReplace" : false, + "isSource" : false + }, + "queryPlan" : null + }, { + "@type" : "ksqlPlanV1", + "statementText" : "CREATE TABLE OUTPUT AS SELECT\n T.ID T_ID,\n T.NAME NAME,\n T.VALUE VALUE,\n TT.F1 F1,\n TT.F2 F2\nFROM TEST T\nRIGHT OUTER JOIN TEST_TABLE TT ON ((T.ID = TT.ID))\nEMIT CHANGES", + "ddlCommand" : { + "@type" : "createTableV1", + "sourceName" : "OUTPUT", + "schema" : "`T_ID` BIGINT KEY, `NAME` STRING, `VALUE` BIGINT, `F1` STRING, `F2` BIGINT", + "timestampColumn" : null, + "topicName" : "OUTPUT", + "formats" : { + "keyFormat" : { + "format" : "KAFKA", + "properties" : { } + }, + "valueFormat" : { + "format" : "PROTOBUF", + "properties" : { + "unwrapPrimitives" : "true" + } + } + }, + "windowInfo" : null, + "orReplace" : false, + "isSource" : false + }, + "queryPlan" : { + "sources" : [ "TEST", "TEST_TABLE" ], + "sink" : "OUTPUT", + "physicalPlan" : { + "@type" : "tableSinkV1", + "properties" : { + "queryContext" : "OUTPUT" + }, + "source" : { + "@type" : "tableSelectV1", + "properties" : { + "queryContext" : "Project" + }, + "source" : { + "@type" : "tableTableJoinV1", + "properties" : { + "queryContext" : "Join" + }, + "joinType" : "RIGHT", + "leftSource" : { + "@type" : "tableSelectV1", + "properties" : { + "queryContext" : "PrependAliasLeft" + }, + "source" : { + "@type" : "tableSourceV2", + "properties" : { + "queryContext" : "KafkaTopic_Left/Source" + }, + "topicName" : "left_topic", + "formats" : { + "keyFormat" : { + "format" : "KAFKA", + "properties" : { } + }, + "valueFormat" : { + "format" : "PROTOBUF", + "properties" : { + "unwrapPrimitives" : "true" + } + } + }, + "timestampColumn" : null, + "sourceSchema" : "`ID` BIGINT KEY, `NAME` STRING, `VALUE` BIGINT", + "pseudoColumnVersion" : 1, + "stateStoreFormats" : { + "keyFormat" : { + "format" : "KAFKA", + "properties" : { } + }, + "valueFormat" : { + "format" : "PROTOBUF", + "properties" : { + "unwrapPrimitives" : "true" + } + } + } + }, + "keyColumnNames" : [ "T_ID" ], + "selectExpressions" : [ "NAME AS T_NAME", "VALUE AS T_VALUE", "ROWTIME AS T_ROWTIME", "ROWPARTITION AS T_ROWPARTITION", "ROWOFFSET AS T_ROWOFFSET", "ID AS T_ID" ], + "internalFormats" : { + "keyFormat" : { + "format" : "KAFKA", + "properties" : { } + }, + "valueFormat" : { + "format" : "PROTOBUF", + "properties" : { + "unwrapPrimitives" : "true" + } + } + } + }, + "rightSource" : { + "@type" : "tableSelectV1", + "properties" : { + "queryContext" : "PrependAliasRight" + }, + "source" : { + "@type" : "tableSourceV2", + "properties" : { + "queryContext" : "KafkaTopic_Right/Source" + }, + "topicName" : "right_topic", + "formats" : { + "keyFormat" : { + "format" : "KAFKA", + "properties" : { } + }, + "valueFormat" : { + "format" : "PROTOBUF", + "properties" : { + "unwrapPrimitives" : "true" + } + } + }, + "timestampColumn" : null, + "sourceSchema" : "`ID` BIGINT KEY, `F1` STRING, `F2` BIGINT", + "pseudoColumnVersion" : 1, + "stateStoreFormats" : { + "keyFormat" : { + "format" : "KAFKA", + "properties" : { } + }, + "valueFormat" : { + "format" : "PROTOBUF", + "properties" : { + "unwrapPrimitives" : "true" + } + } + } + }, + "keyColumnNames" : [ "TT_ID" ], + "selectExpressions" : [ "F1 AS TT_F1", "F2 AS TT_F2", "ROWTIME AS TT_ROWTIME", "ROWPARTITION AS TT_ROWPARTITION", "ROWOFFSET AS TT_ROWOFFSET", "ID AS TT_ID" ], + "internalFormats" : { + "keyFormat" : { + "format" : "KAFKA", + "properties" : { } + }, + "valueFormat" : { + "format" : "PROTOBUF", + "properties" : { + "unwrapPrimitives" : "true" + } + } + } + }, + "keyColName" : "T_ID" + }, + "keyColumnNames" : [ "T_ID" ], + "selectExpressions" : [ "T_NAME AS NAME", "T_VALUE AS VALUE", "TT_F1 AS F1", "TT_F2 AS F2" ], + "internalFormats" : { + "keyFormat" : { + "format" : "KAFKA", + "properties" : { } + }, + "valueFormat" : { + "format" : "PROTOBUF", + "properties" : { + "unwrapPrimitives" : "true" + } + } + } + }, + "formats" : { + "keyFormat" : { + "format" : "KAFKA", + "properties" : { } + }, + "valueFormat" : { + "format" : "PROTOBUF", + "properties" : { + "unwrapPrimitives" : "true" + } + } + }, + "topicName" : "OUTPUT", + "timestampColumn" : null + }, + "queryId" : "CTAS_OUTPUT_0", + "runtimeId" : null + } + } ], + "configs" : { + "ksql.extension.dir" : "ext", + "ksql.streams.cache.max.bytes.buffering" : "0", + "metric.reporters" : "", + "ksql.query.status.running.threshold.seconds" : "300", + "ksql.connect.basic.auth.credentials.reload" : "false", + "ksql.output.topic.name.prefix" : "", + "ksql.query.pull.stream.enabled" : "true", + "ksql.query.push.v2.interpreter.enabled" : "true", + "ksql.queryanonymizer.logs_enabled" : "true", + "ksql.variable.substitution.enable" : "true", + "ksql.streams.shutdown.timeout.ms" : "300000", + "ksql.query.pull.max.allowed.offset.lag" : "9223372036854775807", + "ksql.query.pull.max.qps" : "2147483647", + "ksql.access.validator.enable" : "auto", + "ksql.streams.bootstrap.servers" : "localhost:0", + "ksql.query.pull.metrics.enabled" : "true", + "ksql.metrics.extension" : null, + "ksql.query.push.v2.alos.enabled" : "true", + "ksql.query.push.v2.max.hourly.bandwidth.megabytes" : "2147483647", + "ksql.query.pull.range.scan.enabled" : "true", + "ksql.transient.query.cleanup.service.initial.delay.seconds" : "600", + "ksql.hidden.topics" : "_confluent.*,__confluent.*,_schemas,__consumer_offsets,__transaction_state,connect-configs,connect-offsets,connect-status,connect-statuses", + "ksql.lambdas.enabled" : "true", + "ksql.source.table.materialization.enabled" : "true", + "ksql.query.pull.max.hourly.bandwidth.megabytes" : "2147483647", + "ksql.sink.window.change.log.additional.retention" : "1000000", + "ksql.query.persistent.active.limit" : "2147483647", + "ksql.persistence.wrap.single.values" : null, + "ksql.query.transient.max.bytes.buffering.total" : "-1", + "ksql.connect.basic.auth.credentials.source" : "NONE", + "ksql.schema.registry.url" : "schema_registry.url:0", + "ksql.properties.overrides.denylist" : "", + "ksql.service.id" : "some.ksql.service.id", + "ksql.query.push.v2.max.catchup.consumers" : "5", + "ksql.query.push.v2.enabled" : "false", + "ksql.transient.query.cleanup.service.enable" : "true", + "ksql.query.push.v2.metrics.enabled" : "true", + "ksql.rowpartition.rowoffset.enabled" : "true", + "ksql.streams.commit.interval.ms" : "2000", + "ksql.query.pull.table.scan.enabled" : "true", + "ksql.streams.auto.commit.interval.ms" : "0", + "ksql.streams.topology.optimization" : "all", + "ksql.endpoint.migrate.query" : "true", + "ksql.query.push.v2.registry.installed" : "false", + "ksql.streams.num.stream.threads" : "4", + "ksql.metrics.tags.custom" : "", + "ksql.query.push.v2.catchup.consumer.msg.window" : "50", + "ksql.runtime.feature.shared.enabled" : "false", + "ksql.udf.collect.metrics" : "false", + "ksql.new.query.planner.enabled" : "false", + "ksql.connect.request.headers.plugin" : null, + "ksql.security.extension.class" : null, + "ksql.transient.prefix" : "transient_", + "ksql.headers.columns.enabled" : "true", + "ksql.streams.default.deserialization.exception.handler" : "io.confluent.ksql.errors.LogMetricAndContinueExceptionHandler", + "ksql.connect.request.timeout.ms" : "5000", + "ksql.query.pull.enable.standby.reads" : "false", + "ksql.persistence.default.format.key" : "KAFKA", + "ksql.query.persistent.max.bytes.buffering.total" : "-1", + "ksql.query.error.max.queue.size" : "10", + "ksql.query.cleanup.shutdown.timeout.ms" : "30000", + "ksql.internal.topic.min.insync.replicas" : "1", + "ksql.internal.topic.replicas" : "1", + "ksql.insert.into.values.enabled" : "true", + "ksql.queryanonymizer.cluster_namespace" : null, + "ksql.create.or.replace.enabled" : "true", + "ksql.shared.runtimes.count" : "2", + "ksql.cast.strings.preserve.nulls" : "true", + "ksql.authorization.cache.max.entries" : "10000", + "ksql.pull.queries.enable" : "true", + "ksql.transient.query.cleanup.service.period.seconds" : "600", + "ksql.suppress.enabled" : "false", + "ksql.readonly.topics" : "_confluent.*,__confluent.*,_schemas,__consumer_offsets,__transaction_state,connect-configs,connect-offsets,connect-status,connect-statuses", + "ksql.connect.basic.auth.credentials.file" : "", + "ksql.authorization.cache.expiry.time.secs" : "30", + "ksql.query.retry.backoff.initial.ms" : "15000", + "ksql.query.pull.max.concurrent.requests" : "2147483647", + "ksql.streams.auto.offset.reset" : "earliest", + "ksql.connect.url" : "http://localhost:8083", + "ksql.query.push.v2.new.latest.delay.ms" : "5000", + "ksql.query.push.v2.latest.reset.age.ms" : "30000", + "ksql.streams.default.production.exception.handler" : "io.confluent.ksql.errors.ProductionExceptionHandlerUtil$LogAndFailProductionExceptionHandler", + "ksql.query.pull.interpreter.enabled" : "true", + "ksql.query.pull.limit.clause.enabled" : "true", + "ksql.query.pull.router.thread.pool.size" : "50", + "ksql.query.push.v2.continuation.tokens.enabled" : "false", + "ksql.query.retry.backoff.max.ms" : "900000", + "ksql.timestamp.throw.on.invalid" : "false", + "ksql.persistence.default.format.value" : null, + "ksql.udfs.enabled" : "true", + "ksql.udf.enable.security.manager" : "true", + "ksql.connect.worker.config" : "", + "ksql.nested.error.set.null" : "true", + "ksql.query.pull.thread.pool.size" : "50", + "ksql.persistent.prefix" : "query_", + "ksql.metastore.backup.location" : "", + "ksql.error.classifier.regex" : "", + "ksql.suppress.buffer.size.bytes" : "-1" + } +} \ No newline at end of file diff --git a/ksqldb-functional-tests/src/test/resources/historical_plans/joins_-_table_table_right_join_-_PROTOBUF_-_PROTOBUF/7.3.0_1651071365034/spec.json b/ksqldb-functional-tests/src/test/resources/historical_plans/joins_-_table_table_right_join_-_PROTOBUF_-_PROTOBUF/7.3.0_1651071365034/spec.json new file mode 100644 index 000000000000..0d895cce6108 --- /dev/null +++ b/ksqldb-functional-tests/src/test/resources/historical_plans/joins_-_table_table_right_join_-_PROTOBUF_-_PROTOBUF/7.3.0_1651071365034/spec.json @@ -0,0 +1,355 @@ +{ + "version" : "7.3.0", + "timestamp" : 1651071365034, + "path" : "query-validation-tests/joins.json", + "schemas" : { + "CTAS_OUTPUT_0.PrependAliasLeft" : { + "schema" : "`T_ID` BIGINT KEY, `T_NAME` STRING, `T_VALUE` BIGINT, `T_ROWTIME` BIGINT, `T_ROWPARTITION` INTEGER, `T_ROWOFFSET` BIGINT, `T_ID` BIGINT", + "keyFormat" : { + "format" : "KAFKA" + }, + "valueFormat" : { + "format" : "PROTOBUF", + "properties" : { + "unwrapPrimitives" : "true" + } + } + }, + "CTAS_OUTPUT_0.KafkaTopic_Left.Source.Materialized" : { + "schema" : "`ID` BIGINT KEY, `NAME` STRING, `VALUE` BIGINT, `ROWPARTITION` INTEGER, `ROWOFFSET` BIGINT", + "keyFormat" : { + "format" : "KAFKA" + }, + "valueFormat" : { + "format" : "PROTOBUF", + "properties" : { + "unwrapPrimitives" : "true" + } + } + }, + "CTAS_OUTPUT_0.KafkaTopic_Left.Source" : { + "schema" : "`ID` BIGINT KEY, `NAME` STRING, `VALUE` BIGINT", + "keyFormat" : { + "format" : "KAFKA" + }, + "valueFormat" : { + "format" : "PROTOBUF", + "properties" : { + "unwrapPrimitives" : "true" + } + } + }, + "CTAS_OUTPUT_0.Project" : { + "schema" : "`T_ID` BIGINT KEY, `NAME` STRING, `VALUE` BIGINT, `F1` STRING, `F2` BIGINT", + "keyFormat" : { + "format" : "KAFKA" + }, + "valueFormat" : { + "format" : "PROTOBUF", + "properties" : { + "unwrapPrimitives" : "true" + } + } + }, + "CTAS_OUTPUT_0.KafkaTopic_Right.Source" : { + "schema" : "`ID` BIGINT KEY, `F1` STRING, `F2` BIGINT", + "keyFormat" : { + "format" : "KAFKA" + }, + "valueFormat" : { + "format" : "PROTOBUF", + "properties" : { + "unwrapPrimitives" : "true" + } + } + }, + "CTAS_OUTPUT_0.OUTPUT" : { + "schema" : "`T_ID` BIGINT KEY, `NAME` STRING, `VALUE` BIGINT, `F1` STRING, `F2` BIGINT", + "keyFormat" : { + "format" : "KAFKA" + }, + "valueFormat" : { + "format" : "PROTOBUF", + "properties" : { + "unwrapPrimitives" : "true" + } + } + }, + "CTAS_OUTPUT_0.PrependAliasRight" : { + "schema" : "`TT_ID` BIGINT KEY, `TT_F1` STRING, `TT_F2` BIGINT, `TT_ROWTIME` BIGINT, `TT_ROWPARTITION` INTEGER, `TT_ROWOFFSET` BIGINT, `TT_ID` BIGINT", + "keyFormat" : { + "format" : "KAFKA" + }, + "valueFormat" : { + "format" : "PROTOBUF", + "properties" : { + "unwrapPrimitives" : "true" + } + } + }, + "CTAS_OUTPUT_0.KafkaTopic_Right.Source.Materialized" : { + "schema" : "`ID` BIGINT KEY, `F1` STRING, `F2` BIGINT, `ROWPARTITION` INTEGER, `ROWOFFSET` BIGINT", + "keyFormat" : { + "format" : "KAFKA" + }, + "valueFormat" : { + "format" : "PROTOBUF", + "properties" : { + "unwrapPrimitives" : "true" + } + } + } + }, + "testCase" : { + "name" : "table table right join - PROTOBUF - PROTOBUF", + "inputs" : [ { + "topic" : "left_topic", + "key" : 0, + "value" : { + "NAME" : "zero", + "VALUE" : 0 + }, + "timestamp" : 0 + }, { + "topic" : "right_topic", + "key" : 0, + "value" : { + "F1" : "blah", + "F2" : 50 + }, + "timestamp" : 10000 + }, { + "topic" : "left_topic", + "key" : 10, + "value" : { + "NAME" : "100", + "VALUE" : 5 + }, + "timestamp" : 11000 + }, { + "topic" : "left_topic", + "key" : 0, + "value" : { + "NAME" : "foo", + "VALUE" : 100 + }, + "timestamp" : 13000 + }, { + "topic" : "right_topic", + "key" : 0, + "value" : { + "F1" : "a", + "F2" : 10 + }, + "timestamp" : 15000 + }, { + "topic" : "left_topic", + "key" : 0, + "value" : { + "NAME" : "bar", + "VALUE" : 99 + }, + "timestamp" : 16000 + }, { + "topic" : "left_topic", + "key" : 90, + "value" : { + "NAME" : "ninety", + "VALUE" : 90 + }, + "timestamp" : 17000 + }, { + "topic" : "right_topic", + "key" : 7, + "value" : { + "F1" : "b", + "F2" : 20 + }, + "timestamp" : 18000 + } ], + "outputs" : [ { + "topic" : "OUTPUT", + "key" : 0, + "value" : { + "NAME" : "zero", + "VALUE" : 0, + "F1" : "blah", + "F2" : 50 + }, + "timestamp" : 10000 + }, { + "topic" : "OUTPUT", + "key" : 0, + "value" : { + "NAME" : "foo", + "VALUE" : 100, + "F1" : "blah", + "F2" : 50 + }, + "timestamp" : 13000 + }, { + "topic" : "OUTPUT", + "key" : 0, + "value" : { + "NAME" : "foo", + "VALUE" : 100, + "F1" : "a", + "F2" : 10 + }, + "timestamp" : 15000 + }, { + "topic" : "OUTPUT", + "key" : 0, + "value" : { + "NAME" : "bar", + "VALUE" : 99, + "F1" : "a", + "F2" : 10 + }, + "timestamp" : 16000 + }, { + "topic" : "OUTPUT", + "key" : 7, + "value" : { + "NAME" : "", + "VALUE" : 0, + "F1" : "b", + "F2" : 20 + }, + "timestamp" : 18000 + } ], + "topics" : [ { + "name" : "right_topic", + "valueSchema" : "syntax = \"proto3\";\n\nmessage ConnectDefault1 {\n string F1 = 1;\n int64 F2 = 2;\n}\n", + "valueFormat" : "PROTOBUF", + "replicas" : 1, + "numPartitions" : 4 + }, { + "name" : "OUTPUT", + "replicas" : 1, + "numPartitions" : 4 + }, { + "name" : "left_topic", + "valueSchema" : "syntax = \"proto3\";\n\nmessage ConnectDefault1 {\n string NAME = 1;\n int64 VALUE = 2;\n}\n", + "valueFormat" : "PROTOBUF", + "replicas" : 1, + "numPartitions" : 4 + } ], + "statements" : [ "CREATE TABLE TEST (id BIGINT PRIMARY KEY, name VARCHAR, value BIGINT) WITH (kafka_topic='left_topic', value_format='PROTOBUF');", "CREATE TABLE TEST_TABLE (id BIGINT PRIMARY KEY, f1 VARCHAR, f2 BIGINT) WITH (kafka_topic='right_topic', value_format='PROTOBUF');", "CREATE TABLE OUTPUT AS SELECT t.id, name, value, f1, f2 FROM test t RIGHT JOIN test_table tt ON t.id = tt.id;" ], + "post" : { + "sources" : [ { + "name" : "OUTPUT", + "type" : "TABLE", + "schema" : "`T_ID` BIGINT KEY, `NAME` STRING, `VALUE` BIGINT, `F1` STRING, `F2` BIGINT", + "keyFormat" : { + "format" : "KAFKA" + }, + "valueFormat" : "PROTOBUF", + "keyFeatures" : [ ], + "valueFeatures" : [ ], + "isSource" : false + }, { + "name" : "TEST", + "type" : "TABLE", + "schema" : "`ID` BIGINT KEY, `NAME` STRING, `VALUE` BIGINT", + "keyFormat" : { + "format" : "KAFKA" + }, + "valueFormat" : "PROTOBUF", + "keyFeatures" : [ ], + "valueFeatures" : [ ], + "isSource" : false + }, { + "name" : "TEST_TABLE", + "type" : "TABLE", + "schema" : "`ID` BIGINT KEY, `F1` STRING, `F2` BIGINT", + "keyFormat" : { + "format" : "KAFKA" + }, + "valueFormat" : "PROTOBUF", + "keyFeatures" : [ ], + "valueFeatures" : [ ], + "isSource" : false + } ], + "topics" : { + "topics" : [ { + "name" : "_confluent-ksql-some.ksql.service.idquery_CTAS_OUTPUT_0-Project-Last-changelog", + "keyFormat" : { + "format" : "KAFKA" + }, + "valueFormat" : { + "format" : "PROTOBUF", + "properties" : { + "unwrapPrimitives" : "true" + } + }, + "valueSchema" : "syntax = \"proto3\";\n\nmessage ConnectDefault1 {\n string NAME = 1;\n int64 VALUE = 2;\n string F1 = 3;\n int64 F2 = 4;\n}\n" + }, { + "name" : "_confluent-ksql-some.ksql.service.idquery_CTAS_OUTPUT_0-KafkaTopic_Right-Reduce-changelog", + "keyFormat" : { + "format" : "KAFKA" + }, + "valueFormat" : { + "format" : "PROTOBUF", + "properties" : { + "unwrapPrimitives" : "true" + } + }, + "valueSchema" : "syntax = \"proto3\";\n\nmessage ConnectDefault1 {\n string F1 = 1;\n int64 F2 = 2;\n int32 ROWPARTITION = 3;\n int64 ROWOFFSET = 4;\n}\n" + }, { + "name" : "left_topic", + "keyFormat" : { + "format" : "KAFKA" + }, + "valueFormat" : { + "format" : "PROTOBUF", + "properties" : { + "unwrapPrimitives" : "true" + } + }, + "partitions" : 4, + "valueSchema" : "syntax = \"proto3\";\n\nmessage ConnectDefault1 {\n string NAME = 1;\n int64 VALUE = 2;\n}\n" + }, { + "name" : "_confluent-ksql-some.ksql.service.idquery_CTAS_OUTPUT_0-KafkaTopic_Left-Reduce-changelog", + "keyFormat" : { + "format" : "KAFKA" + }, + "valueFormat" : { + "format" : "PROTOBUF", + "properties" : { + "unwrapPrimitives" : "true" + } + }, + "valueSchema" : "syntax = \"proto3\";\n\nmessage ConnectDefault1 {\n string NAME = 1;\n int64 VALUE = 2;\n int32 ROWPARTITION = 3;\n int64 ROWOFFSET = 4;\n}\n" + }, { + "name" : "OUTPUT", + "keyFormat" : { + "format" : "KAFKA" + }, + "valueFormat" : { + "format" : "PROTOBUF", + "properties" : { + "unwrapPrimitives" : "true" + } + }, + "partitions" : 4, + "valueSchema" : "syntax = \"proto3\";\n\nmessage ConnectDefault1 {\n string NAME = 1;\n int64 VALUE = 2;\n string F1 = 3;\n int64 F2 = 4;\n}\n" + }, { + "name" : "right_topic", + "keyFormat" : { + "format" : "KAFKA" + }, + "valueFormat" : { + "format" : "PROTOBUF", + "properties" : { + "unwrapPrimitives" : "true" + } + }, + "partitions" : 4, + "valueSchema" : "syntax = \"proto3\";\n\nmessage ConnectDefault1 {\n string F1 = 1;\n int64 F2 = 2;\n}\n" + } ], + "blackList" : ".*-repartition" + } + } + } +} \ No newline at end of file diff --git a/ksqldb-functional-tests/src/test/resources/historical_plans/joins_-_table_table_right_join_-_PROTOBUF_-_PROTOBUF/7.3.0_1651071365034/topology b/ksqldb-functional-tests/src/test/resources/historical_plans/joins_-_table_table_right_join_-_PROTOBUF_-_PROTOBUF/7.3.0_1651071365034/topology new file mode 100644 index 000000000000..6febb3fa3d16 --- /dev/null +++ b/ksqldb-functional-tests/src/test/resources/historical_plans/joins_-_table_table_right_join_-_PROTOBUF_-_PROTOBUF/7.3.0_1651071365034/topology @@ -0,0 +1,48 @@ +Topologies: + Sub-topology: 0 + Source: KSTREAM-SOURCE-0000000001 (topics: [left_topic]) + --> KTABLE-SOURCE-0000000002 + Source: KSTREAM-SOURCE-0000000007 (topics: [right_topic]) + --> KTABLE-SOURCE-0000000008 + Processor: KTABLE-SOURCE-0000000002 (stores: []) + --> KTABLE-TRANSFORMVALUES-0000000003 + <-- KSTREAM-SOURCE-0000000001 + Processor: KTABLE-SOURCE-0000000008 (stores: []) + --> KTABLE-TRANSFORMVALUES-0000000009 + <-- KSTREAM-SOURCE-0000000007 + Processor: KTABLE-TRANSFORMVALUES-0000000003 (stores: [KafkaTopic_Left-Reduce]) + --> KTABLE-TRANSFORMVALUES-0000000004 + <-- KTABLE-SOURCE-0000000002 + Processor: KTABLE-TRANSFORMVALUES-0000000009 (stores: [KafkaTopic_Right-Reduce]) + --> KTABLE-TRANSFORMVALUES-0000000010 + <-- KTABLE-SOURCE-0000000008 + Processor: KTABLE-TRANSFORMVALUES-0000000004 (stores: []) + --> PrependAliasLeft + <-- KTABLE-TRANSFORMVALUES-0000000003 + Processor: KTABLE-TRANSFORMVALUES-0000000010 (stores: []) + --> PrependAliasRight + <-- KTABLE-TRANSFORMVALUES-0000000009 + Processor: PrependAliasLeft (stores: []) + --> KTABLE-JOINOTHER-0000000014 + <-- KTABLE-TRANSFORMVALUES-0000000004 + Processor: PrependAliasRight (stores: []) + --> KTABLE-JOINTHIS-0000000013 + <-- KTABLE-TRANSFORMVALUES-0000000010 + Processor: KTABLE-JOINOTHER-0000000014 (stores: [KafkaTopic_Right-Reduce]) + --> KTABLE-MERGE-0000000012 + <-- PrependAliasLeft + Processor: KTABLE-JOINTHIS-0000000013 (stores: [KafkaTopic_Left-Reduce]) + --> KTABLE-MERGE-0000000012 + <-- PrependAliasRight + Processor: KTABLE-MERGE-0000000012 (stores: []) + --> KTABLE-TRANSFORMVALUES-0000000015 + <-- KTABLE-JOINTHIS-0000000013, KTABLE-JOINOTHER-0000000014 + Processor: KTABLE-TRANSFORMVALUES-0000000015 (stores: [Project-Last]) + --> KTABLE-TOSTREAM-0000000016 + <-- KTABLE-MERGE-0000000012 + Processor: KTABLE-TOSTREAM-0000000016 (stores: []) + --> KSTREAM-SINK-0000000017 + <-- KTABLE-TRANSFORMVALUES-0000000015 + Sink: KSTREAM-SINK-0000000017 (topic: OUTPUT) + <-- KTABLE-TOSTREAM-0000000016 + diff --git a/ksqldb-functional-tests/src/test/resources/historical_plans/joins_-_table_table_right_join_-_PROTOBUF_-_PROTOBUF_NOSR/7.3.0_1651071367274/plan.json b/ksqldb-functional-tests/src/test/resources/historical_plans/joins_-_table_table_right_join_-_PROTOBUF_-_PROTOBUF_NOSR/7.3.0_1651071367274/plan.json new file mode 100644 index 000000000000..579f3a96aca9 --- /dev/null +++ b/ksqldb-functional-tests/src/test/resources/historical_plans/joins_-_table_table_right_join_-_PROTOBUF_-_PROTOBUF_NOSR/7.3.0_1651071367274/plan.json @@ -0,0 +1,320 @@ +{ + "plan" : [ { + "@type" : "ksqlPlanV1", + "statementText" : "CREATE TABLE TEST (ID BIGINT PRIMARY KEY, NAME STRING, VALUE BIGINT) WITH (KAFKA_TOPIC='left_topic', KEY_FORMAT='KAFKA', VALUE_FORMAT='PROTOBUF_NOSR');", + "ddlCommand" : { + "@type" : "createTableV1", + "sourceName" : "TEST", + "schema" : "`ID` BIGINT KEY, `NAME` STRING, `VALUE` BIGINT", + "timestampColumn" : null, + "topicName" : "left_topic", + "formats" : { + "keyFormat" : { + "format" : "KAFKA", + "properties" : { } + }, + "valueFormat" : { + "format" : "PROTOBUF_NOSR", + "properties" : { } + } + }, + "windowInfo" : null, + "orReplace" : false, + "isSource" : false + }, + "queryPlan" : null + }, { + "@type" : "ksqlPlanV1", + "statementText" : "CREATE TABLE TEST_TABLE (ID BIGINT PRIMARY KEY, F1 STRING, F2 BIGINT) WITH (KAFKA_TOPIC='right_topic', KEY_FORMAT='KAFKA', VALUE_FORMAT='PROTOBUF_NOSR');", + "ddlCommand" : { + "@type" : "createTableV1", + "sourceName" : "TEST_TABLE", + "schema" : "`ID` BIGINT KEY, `F1` STRING, `F2` BIGINT", + "timestampColumn" : null, + "topicName" : "right_topic", + "formats" : { + "keyFormat" : { + "format" : "KAFKA", + "properties" : { } + }, + "valueFormat" : { + "format" : "PROTOBUF_NOSR", + "properties" : { } + } + }, + "windowInfo" : null, + "orReplace" : false, + "isSource" : false + }, + "queryPlan" : null + }, { + "@type" : "ksqlPlanV1", + "statementText" : "CREATE TABLE OUTPUT AS SELECT\n T.ID T_ID,\n T.NAME NAME,\n T.VALUE VALUE,\n TT.F1 F1,\n TT.F2 F2\nFROM TEST T\nRIGHT OUTER JOIN TEST_TABLE TT ON ((T.ID = TT.ID))\nEMIT CHANGES", + "ddlCommand" : { + "@type" : "createTableV1", + "sourceName" : "OUTPUT", + "schema" : "`T_ID` BIGINT KEY, `NAME` STRING, `VALUE` BIGINT, `F1` STRING, `F2` BIGINT", + "timestampColumn" : null, + "topicName" : "OUTPUT", + "formats" : { + "keyFormat" : { + "format" : "KAFKA", + "properties" : { } + }, + "valueFormat" : { + "format" : "PROTOBUF_NOSR", + "properties" : { } + } + }, + "windowInfo" : null, + "orReplace" : false, + "isSource" : false + }, + "queryPlan" : { + "sources" : [ "TEST", "TEST_TABLE" ], + "sink" : "OUTPUT", + "physicalPlan" : { + "@type" : "tableSinkV1", + "properties" : { + "queryContext" : "OUTPUT" + }, + "source" : { + "@type" : "tableSelectV1", + "properties" : { + "queryContext" : "Project" + }, + "source" : { + "@type" : "tableTableJoinV1", + "properties" : { + "queryContext" : "Join" + }, + "joinType" : "RIGHT", + "leftSource" : { + "@type" : "tableSelectV1", + "properties" : { + "queryContext" : "PrependAliasLeft" + }, + "source" : { + "@type" : "tableSourceV2", + "properties" : { + "queryContext" : "KafkaTopic_Left/Source" + }, + "topicName" : "left_topic", + "formats" : { + "keyFormat" : { + "format" : "KAFKA", + "properties" : { } + }, + "valueFormat" : { + "format" : "PROTOBUF_NOSR", + "properties" : { } + } + }, + "timestampColumn" : null, + "sourceSchema" : "`ID` BIGINT KEY, `NAME` STRING, `VALUE` BIGINT", + "pseudoColumnVersion" : 1, + "stateStoreFormats" : { + "keyFormat" : { + "format" : "KAFKA", + "properties" : { } + }, + "valueFormat" : { + "format" : "PROTOBUF_NOSR", + "properties" : { } + } + } + }, + "keyColumnNames" : [ "T_ID" ], + "selectExpressions" : [ "NAME AS T_NAME", "VALUE AS T_VALUE", "ROWTIME AS T_ROWTIME", "ROWPARTITION AS T_ROWPARTITION", "ROWOFFSET AS T_ROWOFFSET", "ID AS T_ID" ], + "internalFormats" : { + "keyFormat" : { + "format" : "KAFKA", + "properties" : { } + }, + "valueFormat" : { + "format" : "PROTOBUF_NOSR", + "properties" : { } + } + } + }, + "rightSource" : { + "@type" : "tableSelectV1", + "properties" : { + "queryContext" : "PrependAliasRight" + }, + "source" : { + "@type" : "tableSourceV2", + "properties" : { + "queryContext" : "KafkaTopic_Right/Source" + }, + "topicName" : "right_topic", + "formats" : { + "keyFormat" : { + "format" : "KAFKA", + "properties" : { } + }, + "valueFormat" : { + "format" : "PROTOBUF_NOSR", + "properties" : { } + } + }, + "timestampColumn" : null, + "sourceSchema" : "`ID` BIGINT KEY, `F1` STRING, `F2` BIGINT", + "pseudoColumnVersion" : 1, + "stateStoreFormats" : { + "keyFormat" : { + "format" : "KAFKA", + "properties" : { } + }, + "valueFormat" : { + "format" : "PROTOBUF_NOSR", + "properties" : { } + } + } + }, + "keyColumnNames" : [ "TT_ID" ], + "selectExpressions" : [ "F1 AS TT_F1", "F2 AS TT_F2", "ROWTIME AS TT_ROWTIME", "ROWPARTITION AS TT_ROWPARTITION", "ROWOFFSET AS TT_ROWOFFSET", "ID AS TT_ID" ], + "internalFormats" : { + "keyFormat" : { + "format" : "KAFKA", + "properties" : { } + }, + "valueFormat" : { + "format" : "PROTOBUF_NOSR", + "properties" : { } + } + } + }, + "keyColName" : "T_ID" + }, + "keyColumnNames" : [ "T_ID" ], + "selectExpressions" : [ "T_NAME AS NAME", "T_VALUE AS VALUE", "TT_F1 AS F1", "TT_F2 AS F2" ], + "internalFormats" : { + "keyFormat" : { + "format" : "KAFKA", + "properties" : { } + }, + "valueFormat" : { + "format" : "PROTOBUF_NOSR", + "properties" : { } + } + } + }, + "formats" : { + "keyFormat" : { + "format" : "KAFKA", + "properties" : { } + }, + "valueFormat" : { + "format" : "PROTOBUF_NOSR", + "properties" : { } + } + }, + "topicName" : "OUTPUT", + "timestampColumn" : null + }, + "queryId" : "CTAS_OUTPUT_0", + "runtimeId" : null + } + } ], + "configs" : { + "ksql.extension.dir" : "ext", + "ksql.streams.cache.max.bytes.buffering" : "0", + "metric.reporters" : "", + "ksql.query.status.running.threshold.seconds" : "300", + "ksql.connect.basic.auth.credentials.reload" : "false", + "ksql.output.topic.name.prefix" : "", + "ksql.query.pull.stream.enabled" : "true", + "ksql.query.push.v2.interpreter.enabled" : "true", + "ksql.queryanonymizer.logs_enabled" : "true", + "ksql.variable.substitution.enable" : "true", + "ksql.streams.shutdown.timeout.ms" : "300000", + "ksql.query.pull.max.allowed.offset.lag" : "9223372036854775807", + "ksql.query.pull.max.qps" : "2147483647", + "ksql.access.validator.enable" : "auto", + "ksql.streams.bootstrap.servers" : "localhost:0", + "ksql.query.pull.metrics.enabled" : "true", + "ksql.metrics.extension" : null, + "ksql.query.push.v2.alos.enabled" : "true", + "ksql.query.push.v2.max.hourly.bandwidth.megabytes" : "2147483647", + "ksql.query.pull.range.scan.enabled" : "true", + "ksql.transient.query.cleanup.service.initial.delay.seconds" : "600", + "ksql.hidden.topics" : "_confluent.*,__confluent.*,_schemas,__consumer_offsets,__transaction_state,connect-configs,connect-offsets,connect-status,connect-statuses", + "ksql.lambdas.enabled" : "true", + "ksql.source.table.materialization.enabled" : "true", + "ksql.query.pull.max.hourly.bandwidth.megabytes" : "2147483647", + "ksql.sink.window.change.log.additional.retention" : "1000000", + "ksql.query.persistent.active.limit" : "2147483647", + "ksql.persistence.wrap.single.values" : null, + "ksql.query.transient.max.bytes.buffering.total" : "-1", + "ksql.connect.basic.auth.credentials.source" : "NONE", + "ksql.schema.registry.url" : "schema_registry.url:0", + "ksql.properties.overrides.denylist" : "", + "ksql.service.id" : "some.ksql.service.id", + "ksql.query.push.v2.max.catchup.consumers" : "5", + "ksql.query.push.v2.enabled" : "false", + "ksql.transient.query.cleanup.service.enable" : "true", + "ksql.query.push.v2.metrics.enabled" : "true", + "ksql.rowpartition.rowoffset.enabled" : "true", + "ksql.streams.commit.interval.ms" : "2000", + "ksql.query.pull.table.scan.enabled" : "true", + "ksql.streams.auto.commit.interval.ms" : "0", + "ksql.streams.topology.optimization" : "all", + "ksql.endpoint.migrate.query" : "true", + "ksql.query.push.v2.registry.installed" : "false", + "ksql.streams.num.stream.threads" : "4", + "ksql.metrics.tags.custom" : "", + "ksql.query.push.v2.catchup.consumer.msg.window" : "50", + "ksql.runtime.feature.shared.enabled" : "false", + "ksql.udf.collect.metrics" : "false", + "ksql.new.query.planner.enabled" : "false", + "ksql.connect.request.headers.plugin" : null, + "ksql.security.extension.class" : null, + "ksql.transient.prefix" : "transient_", + "ksql.headers.columns.enabled" : "true", + "ksql.streams.default.deserialization.exception.handler" : "io.confluent.ksql.errors.LogMetricAndContinueExceptionHandler", + "ksql.connect.request.timeout.ms" : "5000", + "ksql.query.pull.enable.standby.reads" : "false", + "ksql.persistence.default.format.key" : "KAFKA", + "ksql.query.persistent.max.bytes.buffering.total" : "-1", + "ksql.query.error.max.queue.size" : "10", + "ksql.query.cleanup.shutdown.timeout.ms" : "30000", + "ksql.internal.topic.min.insync.replicas" : "1", + "ksql.internal.topic.replicas" : "1", + "ksql.insert.into.values.enabled" : "true", + "ksql.queryanonymizer.cluster_namespace" : null, + "ksql.create.or.replace.enabled" : "true", + "ksql.shared.runtimes.count" : "2", + "ksql.cast.strings.preserve.nulls" : "true", + "ksql.authorization.cache.max.entries" : "10000", + "ksql.pull.queries.enable" : "true", + "ksql.transient.query.cleanup.service.period.seconds" : "600", + "ksql.suppress.enabled" : "false", + "ksql.readonly.topics" : "_confluent.*,__confluent.*,_schemas,__consumer_offsets,__transaction_state,connect-configs,connect-offsets,connect-status,connect-statuses", + "ksql.connect.basic.auth.credentials.file" : "", + "ksql.authorization.cache.expiry.time.secs" : "30", + "ksql.query.retry.backoff.initial.ms" : "15000", + "ksql.query.pull.max.concurrent.requests" : "2147483647", + "ksql.streams.auto.offset.reset" : "earliest", + "ksql.connect.url" : "http://localhost:8083", + "ksql.query.push.v2.new.latest.delay.ms" : "5000", + "ksql.query.push.v2.latest.reset.age.ms" : "30000", + "ksql.streams.default.production.exception.handler" : "io.confluent.ksql.errors.ProductionExceptionHandlerUtil$LogAndFailProductionExceptionHandler", + "ksql.query.pull.interpreter.enabled" : "true", + "ksql.query.pull.limit.clause.enabled" : "true", + "ksql.query.pull.router.thread.pool.size" : "50", + "ksql.query.push.v2.continuation.tokens.enabled" : "false", + "ksql.query.retry.backoff.max.ms" : "900000", + "ksql.timestamp.throw.on.invalid" : "false", + "ksql.persistence.default.format.value" : null, + "ksql.udfs.enabled" : "true", + "ksql.udf.enable.security.manager" : "true", + "ksql.connect.worker.config" : "", + "ksql.nested.error.set.null" : "true", + "ksql.query.pull.thread.pool.size" : "50", + "ksql.persistent.prefix" : "query_", + "ksql.metastore.backup.location" : "", + "ksql.error.classifier.regex" : "", + "ksql.suppress.buffer.size.bytes" : "-1" + } +} \ No newline at end of file diff --git a/ksqldb-functional-tests/src/test/resources/historical_plans/joins_-_table_table_right_join_-_PROTOBUF_-_PROTOBUF_NOSR/7.3.0_1651071367274/spec.json b/ksqldb-functional-tests/src/test/resources/historical_plans/joins_-_table_table_right_join_-_PROTOBUF_-_PROTOBUF_NOSR/7.3.0_1651071367274/spec.json new file mode 100644 index 000000000000..71d610219295 --- /dev/null +++ b/ksqldb-functional-tests/src/test/resources/historical_plans/joins_-_table_table_right_join_-_PROTOBUF_-_PROTOBUF_NOSR/7.3.0_1651071367274/spec.json @@ -0,0 +1,303 @@ +{ + "version" : "7.3.0", + "timestamp" : 1651071367274, + "path" : "query-validation-tests/joins.json", + "schemas" : { + "CTAS_OUTPUT_0.PrependAliasLeft" : { + "schema" : "`T_ID` BIGINT KEY, `T_NAME` STRING, `T_VALUE` BIGINT, `T_ROWTIME` BIGINT, `T_ROWPARTITION` INTEGER, `T_ROWOFFSET` BIGINT, `T_ID` BIGINT", + "keyFormat" : { + "format" : "KAFKA" + }, + "valueFormat" : { + "format" : "PROTOBUF_NOSR" + } + }, + "CTAS_OUTPUT_0.KafkaTopic_Left.Source.Materialized" : { + "schema" : "`ID` BIGINT KEY, `NAME` STRING, `VALUE` BIGINT, `ROWPARTITION` INTEGER, `ROWOFFSET` BIGINT", + "keyFormat" : { + "format" : "KAFKA" + }, + "valueFormat" : { + "format" : "PROTOBUF_NOSR" + } + }, + "CTAS_OUTPUT_0.KafkaTopic_Left.Source" : { + "schema" : "`ID` BIGINT KEY, `NAME` STRING, `VALUE` BIGINT", + "keyFormat" : { + "format" : "KAFKA" + }, + "valueFormat" : { + "format" : "PROTOBUF_NOSR" + } + }, + "CTAS_OUTPUT_0.Project" : { + "schema" : "`T_ID` BIGINT KEY, `NAME` STRING, `VALUE` BIGINT, `F1` STRING, `F2` BIGINT", + "keyFormat" : { + "format" : "KAFKA" + }, + "valueFormat" : { + "format" : "PROTOBUF_NOSR" + } + }, + "CTAS_OUTPUT_0.KafkaTopic_Right.Source" : { + "schema" : "`ID` BIGINT KEY, `F1` STRING, `F2` BIGINT", + "keyFormat" : { + "format" : "KAFKA" + }, + "valueFormat" : { + "format" : "PROTOBUF_NOSR" + } + }, + "CTAS_OUTPUT_0.OUTPUT" : { + "schema" : "`T_ID` BIGINT KEY, `NAME` STRING, `VALUE` BIGINT, `F1` STRING, `F2` BIGINT", + "keyFormat" : { + "format" : "KAFKA" + }, + "valueFormat" : { + "format" : "PROTOBUF_NOSR" + } + }, + "CTAS_OUTPUT_0.PrependAliasRight" : { + "schema" : "`TT_ID` BIGINT KEY, `TT_F1` STRING, `TT_F2` BIGINT, `TT_ROWTIME` BIGINT, `TT_ROWPARTITION` INTEGER, `TT_ROWOFFSET` BIGINT, `TT_ID` BIGINT", + "keyFormat" : { + "format" : "KAFKA" + }, + "valueFormat" : { + "format" : "PROTOBUF_NOSR" + } + }, + "CTAS_OUTPUT_0.KafkaTopic_Right.Source.Materialized" : { + "schema" : "`ID` BIGINT KEY, `F1` STRING, `F2` BIGINT, `ROWPARTITION` INTEGER, `ROWOFFSET` BIGINT", + "keyFormat" : { + "format" : "KAFKA" + }, + "valueFormat" : { + "format" : "PROTOBUF_NOSR" + } + } + }, + "testCase" : { + "name" : "table table right join - PROTOBUF - PROTOBUF_NOSR", + "inputs" : [ { + "topic" : "left_topic", + "key" : 0, + "value" : { + "NAME" : "zero", + "VALUE" : 0 + }, + "timestamp" : 0 + }, { + "topic" : "right_topic", + "key" : 0, + "value" : { + "F1" : "blah", + "F2" : 50 + }, + "timestamp" : 10000 + }, { + "topic" : "left_topic", + "key" : 10, + "value" : { + "NAME" : "100", + "VALUE" : 5 + }, + "timestamp" : 11000 + }, { + "topic" : "left_topic", + "key" : 0, + "value" : { + "NAME" : "foo", + "VALUE" : 100 + }, + "timestamp" : 13000 + }, { + "topic" : "right_topic", + "key" : 0, + "value" : { + "F1" : "a", + "F2" : 10 + }, + "timestamp" : 15000 + }, { + "topic" : "left_topic", + "key" : 0, + "value" : { + "NAME" : "bar", + "VALUE" : 99 + }, + "timestamp" : 16000 + }, { + "topic" : "left_topic", + "key" : 90, + "value" : { + "NAME" : "ninety", + "VALUE" : 90 + }, + "timestamp" : 17000 + }, { + "topic" : "right_topic", + "key" : 7, + "value" : { + "F1" : "b", + "F2" : 20 + }, + "timestamp" : 18000 + } ], + "outputs" : [ { + "topic" : "OUTPUT", + "key" : 0, + "value" : { + "NAME" : "zero", + "VALUE" : 0, + "F1" : "blah", + "F2" : 50 + }, + "timestamp" : 10000 + }, { + "topic" : "OUTPUT", + "key" : 0, + "value" : { + "NAME" : "foo", + "VALUE" : 100, + "F1" : "blah", + "F2" : 50 + }, + "timestamp" : 13000 + }, { + "topic" : "OUTPUT", + "key" : 0, + "value" : { + "NAME" : "foo", + "VALUE" : 100, + "F1" : "a", + "F2" : 10 + }, + "timestamp" : 15000 + }, { + "topic" : "OUTPUT", + "key" : 0, + "value" : { + "NAME" : "bar", + "VALUE" : 99, + "F1" : "a", + "F2" : 10 + }, + "timestamp" : 16000 + }, { + "topic" : "OUTPUT", + "key" : 7, + "value" : { + "NAME" : "", + "VALUE" : 0, + "F1" : "b", + "F2" : 20 + }, + "timestamp" : 18000 + } ], + "topics" : [ { + "name" : "right_topic", + "replicas" : 1, + "numPartitions" : 4 + }, { + "name" : "OUTPUT", + "replicas" : 1, + "numPartitions" : 4 + }, { + "name" : "left_topic", + "replicas" : 1, + "numPartitions" : 4 + } ], + "statements" : [ "CREATE TABLE TEST (id BIGINT PRIMARY KEY, name VARCHAR, value BIGINT) WITH (kafka_topic='left_topic', value_format='PROTOBUF_NOSR');", "CREATE TABLE TEST_TABLE (id BIGINT PRIMARY KEY, f1 VARCHAR, f2 BIGINT) WITH (kafka_topic='right_topic', value_format='PROTOBUF_NOSR');", "CREATE TABLE OUTPUT AS SELECT t.id, name, value, f1, f2 FROM test t RIGHT JOIN test_table tt ON t.id = tt.id;" ], + "post" : { + "sources" : [ { + "name" : "OUTPUT", + "type" : "TABLE", + "schema" : "`T_ID` BIGINT KEY, `NAME` STRING, `VALUE` BIGINT, `F1` STRING, `F2` BIGINT", + "keyFormat" : { + "format" : "KAFKA" + }, + "valueFormat" : "PROTOBUF_NOSR", + "keyFeatures" : [ ], + "valueFeatures" : [ ], + "isSource" : false + }, { + "name" : "TEST", + "type" : "TABLE", + "schema" : "`ID` BIGINT KEY, `NAME` STRING, `VALUE` BIGINT", + "keyFormat" : { + "format" : "KAFKA" + }, + "valueFormat" : "PROTOBUF_NOSR", + "keyFeatures" : [ ], + "valueFeatures" : [ ], + "isSource" : false + }, { + "name" : "TEST_TABLE", + "type" : "TABLE", + "schema" : "`ID` BIGINT KEY, `F1` STRING, `F2` BIGINT", + "keyFormat" : { + "format" : "KAFKA" + }, + "valueFormat" : "PROTOBUF_NOSR", + "keyFeatures" : [ ], + "valueFeatures" : [ ], + "isSource" : false + } ], + "topics" : { + "topics" : [ { + "name" : "_confluent-ksql-some.ksql.service.idquery_CTAS_OUTPUT_0-Project-Last-changelog", + "keyFormat" : { + "format" : "KAFKA" + }, + "valueFormat" : { + "format" : "PROTOBUF_NOSR" + } + }, { + "name" : "_confluent-ksql-some.ksql.service.idquery_CTAS_OUTPUT_0-KafkaTopic_Right-Reduce-changelog", + "keyFormat" : { + "format" : "KAFKA" + }, + "valueFormat" : { + "format" : "PROTOBUF_NOSR" + } + }, { + "name" : "left_topic", + "keyFormat" : { + "format" : "KAFKA" + }, + "valueFormat" : { + "format" : "PROTOBUF_NOSR" + }, + "partitions" : 4 + }, { + "name" : "_confluent-ksql-some.ksql.service.idquery_CTAS_OUTPUT_0-KafkaTopic_Left-Reduce-changelog", + "keyFormat" : { + "format" : "KAFKA" + }, + "valueFormat" : { + "format" : "PROTOBUF_NOSR" + } + }, { + "name" : "OUTPUT", + "keyFormat" : { + "format" : "KAFKA" + }, + "valueFormat" : { + "format" : "PROTOBUF_NOSR" + }, + "partitions" : 4 + }, { + "name" : "right_topic", + "keyFormat" : { + "format" : "KAFKA" + }, + "valueFormat" : { + "format" : "PROTOBUF_NOSR" + }, + "partitions" : 4 + } ], + "blackList" : ".*-repartition" + } + } + } +} \ No newline at end of file diff --git a/ksqldb-functional-tests/src/test/resources/historical_plans/joins_-_table_table_right_join_-_PROTOBUF_-_PROTOBUF_NOSR/7.3.0_1651071367274/topology b/ksqldb-functional-tests/src/test/resources/historical_plans/joins_-_table_table_right_join_-_PROTOBUF_-_PROTOBUF_NOSR/7.3.0_1651071367274/topology new file mode 100644 index 000000000000..6febb3fa3d16 --- /dev/null +++ b/ksqldb-functional-tests/src/test/resources/historical_plans/joins_-_table_table_right_join_-_PROTOBUF_-_PROTOBUF_NOSR/7.3.0_1651071367274/topology @@ -0,0 +1,48 @@ +Topologies: + Sub-topology: 0 + Source: KSTREAM-SOURCE-0000000001 (topics: [left_topic]) + --> KTABLE-SOURCE-0000000002 + Source: KSTREAM-SOURCE-0000000007 (topics: [right_topic]) + --> KTABLE-SOURCE-0000000008 + Processor: KTABLE-SOURCE-0000000002 (stores: []) + --> KTABLE-TRANSFORMVALUES-0000000003 + <-- KSTREAM-SOURCE-0000000001 + Processor: KTABLE-SOURCE-0000000008 (stores: []) + --> KTABLE-TRANSFORMVALUES-0000000009 + <-- KSTREAM-SOURCE-0000000007 + Processor: KTABLE-TRANSFORMVALUES-0000000003 (stores: [KafkaTopic_Left-Reduce]) + --> KTABLE-TRANSFORMVALUES-0000000004 + <-- KTABLE-SOURCE-0000000002 + Processor: KTABLE-TRANSFORMVALUES-0000000009 (stores: [KafkaTopic_Right-Reduce]) + --> KTABLE-TRANSFORMVALUES-0000000010 + <-- KTABLE-SOURCE-0000000008 + Processor: KTABLE-TRANSFORMVALUES-0000000004 (stores: []) + --> PrependAliasLeft + <-- KTABLE-TRANSFORMVALUES-0000000003 + Processor: KTABLE-TRANSFORMVALUES-0000000010 (stores: []) + --> PrependAliasRight + <-- KTABLE-TRANSFORMVALUES-0000000009 + Processor: PrependAliasLeft (stores: []) + --> KTABLE-JOINOTHER-0000000014 + <-- KTABLE-TRANSFORMVALUES-0000000004 + Processor: PrependAliasRight (stores: []) + --> KTABLE-JOINTHIS-0000000013 + <-- KTABLE-TRANSFORMVALUES-0000000010 + Processor: KTABLE-JOINOTHER-0000000014 (stores: [KafkaTopic_Right-Reduce]) + --> KTABLE-MERGE-0000000012 + <-- PrependAliasLeft + Processor: KTABLE-JOINTHIS-0000000013 (stores: [KafkaTopic_Left-Reduce]) + --> KTABLE-MERGE-0000000012 + <-- PrependAliasRight + Processor: KTABLE-MERGE-0000000012 (stores: []) + --> KTABLE-TRANSFORMVALUES-0000000015 + <-- KTABLE-JOINTHIS-0000000013, KTABLE-JOINOTHER-0000000014 + Processor: KTABLE-TRANSFORMVALUES-0000000015 (stores: [Project-Last]) + --> KTABLE-TOSTREAM-0000000016 + <-- KTABLE-MERGE-0000000012 + Processor: KTABLE-TOSTREAM-0000000016 (stores: []) + --> KSTREAM-SINK-0000000017 + <-- KTABLE-TRANSFORMVALUES-0000000015 + Sink: KSTREAM-SINK-0000000017 (topic: OUTPUT) + <-- KTABLE-TOSTREAM-0000000016 + diff --git a/ksqldb-functional-tests/src/test/resources/historical_plans/max-group-by_-_max_bytes_group_by/7.3.0_1651157910151/plan.json b/ksqldb-functional-tests/src/test/resources/historical_plans/max-group-by_-_max_bytes_group_by/7.3.0_1651157910151/plan.json new file mode 100644 index 000000000000..f28501990e6f --- /dev/null +++ b/ksqldb-functional-tests/src/test/resources/historical_plans/max-group-by_-_max_bytes_group_by/7.3.0_1651157910151/plan.json @@ -0,0 +1,255 @@ +{ + "plan" : [ { + "@type" : "ksqlPlanV1", + "statementText" : "CREATE STREAM TEST (ID BIGINT KEY, VALUE BYTES) WITH (KAFKA_TOPIC='test_topic', KEY_FORMAT='KAFKA', VALUE_FORMAT='AVRO');", + "ddlCommand" : { + "@type" : "createStreamV1", + "sourceName" : "TEST", + "schema" : "`ID` BIGINT KEY, `VALUE` BYTES", + "timestampColumn" : null, + "topicName" : "test_topic", + "formats" : { + "keyFormat" : { + "format" : "KAFKA", + "properties" : { } + }, + "valueFormat" : { + "format" : "AVRO", + "properties" : { } + } + }, + "windowInfo" : null, + "orReplace" : false, + "isSource" : false + }, + "queryPlan" : null + }, { + "@type" : "ksqlPlanV1", + "statementText" : "CREATE TABLE S2 AS SELECT\n TEST.ID ID,\n MAX(TEST.VALUE) VALUE\nFROM TEST TEST\nGROUP BY TEST.ID\nEMIT CHANGES", + "ddlCommand" : { + "@type" : "createTableV1", + "sourceName" : "S2", + "schema" : "`ID` BIGINT KEY, `VALUE` BYTES", + "timestampColumn" : null, + "topicName" : "S2", + "formats" : { + "keyFormat" : { + "format" : "KAFKA", + "properties" : { } + }, + "valueFormat" : { + "format" : "AVRO", + "properties" : { } + } + }, + "windowInfo" : null, + "orReplace" : false, + "isSource" : false + }, + "queryPlan" : { + "sources" : [ "TEST" ], + "sink" : "S2", + "physicalPlan" : { + "@type" : "tableSinkV1", + "properties" : { + "queryContext" : "S2" + }, + "source" : { + "@type" : "tableSelectV1", + "properties" : { + "queryContext" : "Aggregate/Project" + }, + "source" : { + "@type" : "streamAggregateV1", + "properties" : { + "queryContext" : "Aggregate/Aggregate" + }, + "source" : { + "@type" : "streamGroupByKeyV1", + "properties" : { + "queryContext" : "Aggregate/GroupBy" + }, + "source" : { + "@type" : "streamSelectV1", + "properties" : { + "queryContext" : "Aggregate/Prepare" + }, + "source" : { + "@type" : "streamSourceV1", + "properties" : { + "queryContext" : "KsqlTopic/Source" + }, + "topicName" : "test_topic", + "formats" : { + "keyFormat" : { + "format" : "KAFKA", + "properties" : { } + }, + "valueFormat" : { + "format" : "AVRO", + "properties" : { } + } + }, + "timestampColumn" : null, + "sourceSchema" : "`ID` BIGINT KEY, `VALUE` BYTES", + "pseudoColumnVersion" : 1 + }, + "keyColumnNames" : [ "ID" ], + "selectedKeys" : null, + "selectExpressions" : [ "ID AS ID", "VALUE AS VALUE" ] + }, + "internalFormats" : { + "keyFormat" : { + "format" : "KAFKA", + "properties" : { } + }, + "valueFormat" : { + "format" : "AVRO", + "properties" : { } + } + } + }, + "internalFormats" : { + "keyFormat" : { + "format" : "KAFKA", + "properties" : { } + }, + "valueFormat" : { + "format" : "AVRO", + "properties" : { } + } + }, + "nonAggregateColumns" : [ "ID", "VALUE" ], + "aggregationFunctions" : [ "MAX(VALUE)" ] + }, + "keyColumnNames" : [ "ID" ], + "selectExpressions" : [ "KSQL_AGG_VARIABLE_0 AS VALUE" ], + "internalFormats" : { + "keyFormat" : { + "format" : "KAFKA", + "properties" : { } + }, + "valueFormat" : { + "format" : "AVRO", + "properties" : { } + } + } + }, + "formats" : { + "keyFormat" : { + "format" : "KAFKA", + "properties" : { } + }, + "valueFormat" : { + "format" : "AVRO", + "properties" : { } + } + }, + "topicName" : "S2", + "timestampColumn" : null + }, + "queryId" : "CTAS_S2_0", + "runtimeId" : null + } + } ], + "configs" : { + "ksql.extension.dir" : "ext", + "ksql.streams.cache.max.bytes.buffering" : "0", + "metric.reporters" : "", + "ksql.query.status.running.threshold.seconds" : "300", + "ksql.connect.basic.auth.credentials.reload" : "false", + "ksql.output.topic.name.prefix" : "", + "ksql.query.pull.stream.enabled" : "true", + "ksql.query.push.v2.interpreter.enabled" : "true", + "ksql.queryanonymizer.logs_enabled" : "true", + "ksql.variable.substitution.enable" : "true", + "ksql.streams.shutdown.timeout.ms" : "300000", + "ksql.query.pull.max.allowed.offset.lag" : "9223372036854775807", + "ksql.query.pull.max.qps" : "2147483647", + "ksql.access.validator.enable" : "auto", + "ksql.streams.bootstrap.servers" : "localhost:0", + "ksql.query.pull.metrics.enabled" : "true", + "ksql.metrics.extension" : null, + "ksql.query.push.v2.alos.enabled" : "true", + "ksql.query.push.v2.max.hourly.bandwidth.megabytes" : "2147483647", + "ksql.query.pull.range.scan.enabled" : "true", + "ksql.transient.query.cleanup.service.initial.delay.seconds" : "600", + "ksql.hidden.topics" : "_confluent.*,__confluent.*,_schemas,__consumer_offsets,__transaction_state,connect-configs,connect-offsets,connect-status,connect-statuses", + "ksql.lambdas.enabled" : "true", + "ksql.source.table.materialization.enabled" : "true", + "ksql.query.pull.max.hourly.bandwidth.megabytes" : "2147483647", + "ksql.sink.window.change.log.additional.retention" : "1000000", + "ksql.query.persistent.active.limit" : "2147483647", + "ksql.persistence.wrap.single.values" : null, + "ksql.query.transient.max.bytes.buffering.total" : "-1", + "ksql.connect.basic.auth.credentials.source" : "NONE", + "ksql.schema.registry.url" : "schema_registry.url:0", + "ksql.properties.overrides.denylist" : "", + "ksql.service.id" : "some.ksql.service.id", + "ksql.query.push.v2.max.catchup.consumers" : "5", + "ksql.query.push.v2.enabled" : "false", + "ksql.transient.query.cleanup.service.enable" : "true", + "ksql.query.push.v2.metrics.enabled" : "true", + "ksql.rowpartition.rowoffset.enabled" : "true", + "ksql.streams.commit.interval.ms" : "2000", + "ksql.query.pull.table.scan.enabled" : "true", + "ksql.streams.auto.commit.interval.ms" : "0", + "ksql.streams.topology.optimization" : "all", + "ksql.endpoint.migrate.query" : "true", + "ksql.query.push.v2.registry.installed" : "false", + "ksql.streams.num.stream.threads" : "4", + "ksql.metrics.tags.custom" : "", + "ksql.query.push.v2.catchup.consumer.msg.window" : "50", + "ksql.runtime.feature.shared.enabled" : "false", + "ksql.udf.collect.metrics" : "false", + "ksql.new.query.planner.enabled" : "false", + "ksql.connect.request.headers.plugin" : null, + "ksql.security.extension.class" : null, + "ksql.transient.prefix" : "transient_", + "ksql.headers.columns.enabled" : "true", + "ksql.streams.default.deserialization.exception.handler" : "io.confluent.ksql.errors.LogMetricAndContinueExceptionHandler", + "ksql.connect.request.timeout.ms" : "5000", + "ksql.query.pull.enable.standby.reads" : "false", + "ksql.persistence.default.format.key" : "KAFKA", + "ksql.query.persistent.max.bytes.buffering.total" : "-1", + "ksql.query.error.max.queue.size" : "10", + "ksql.query.cleanup.shutdown.timeout.ms" : "30000", + "ksql.internal.topic.min.insync.replicas" : "1", + "ksql.internal.topic.replicas" : "1", + "ksql.insert.into.values.enabled" : "true", + "ksql.queryanonymizer.cluster_namespace" : null, + "ksql.create.or.replace.enabled" : "true", + "ksql.shared.runtimes.count" : "2", + "ksql.cast.strings.preserve.nulls" : "true", + "ksql.authorization.cache.max.entries" : "10000", + "ksql.pull.queries.enable" : "true", + "ksql.transient.query.cleanup.service.period.seconds" : "600", + "ksql.suppress.enabled" : "false", + "ksql.readonly.topics" : "_confluent.*,__confluent.*,_schemas,__consumer_offsets,__transaction_state,connect-configs,connect-offsets,connect-status,connect-statuses", + "ksql.connect.basic.auth.credentials.file" : "", + "ksql.authorization.cache.expiry.time.secs" : "30", + "ksql.query.retry.backoff.initial.ms" : "15000", + "ksql.query.pull.max.concurrent.requests" : "2147483647", + "ksql.streams.auto.offset.reset" : "earliest", + "ksql.connect.url" : "http://localhost:8083", + "ksql.query.push.v2.new.latest.delay.ms" : "5000", + "ksql.query.push.v2.latest.reset.age.ms" : "30000", + "ksql.streams.default.production.exception.handler" : "io.confluent.ksql.errors.ProductionExceptionHandlerUtil$LogAndFailProductionExceptionHandler", + "ksql.query.pull.interpreter.enabled" : "true", + "ksql.query.pull.limit.clause.enabled" : "true", + "ksql.query.pull.router.thread.pool.size" : "50", + "ksql.query.push.v2.continuation.tokens.enabled" : "false", + "ksql.query.retry.backoff.max.ms" : "900000", + "ksql.timestamp.throw.on.invalid" : "false", + "ksql.persistence.default.format.value" : null, + "ksql.udfs.enabled" : "true", + "ksql.udf.enable.security.manager" : "true", + "ksql.connect.worker.config" : "", + "ksql.nested.error.set.null" : "true", + "ksql.query.pull.thread.pool.size" : "50", + "ksql.persistent.prefix" : "query_", + "ksql.metastore.backup.location" : "", + "ksql.error.classifier.regex" : "", + "ksql.suppress.buffer.size.bytes" : "-1" + } +} \ No newline at end of file diff --git a/ksqldb-functional-tests/src/test/resources/historical_plans/max-group-by_-_max_bytes_group_by/7.3.0_1651157910151/spec.json b/ksqldb-functional-tests/src/test/resources/historical_plans/max-group-by_-_max_bytes_group_by/7.3.0_1651157910151/spec.json new file mode 100644 index 000000000000..f64d5d2e99ae --- /dev/null +++ b/ksqldb-functional-tests/src/test/resources/historical_plans/max-group-by_-_max_bytes_group_by/7.3.0_1651157910151/spec.json @@ -0,0 +1,292 @@ +{ + "version" : "7.3.0", + "timestamp" : 1651157910151, + "path" : "query-validation-tests/max-group-by.json", + "schemas" : { + "CTAS_S2_0.Aggregate.Aggregate.Materialize" : { + "schema" : "`ID` BIGINT KEY, `ID` BIGINT, `VALUE` BYTES, `KSQL_AGG_VARIABLE_0` BYTES", + "keyFormat" : { + "format" : "KAFKA" + }, + "valueFormat" : { + "format" : "AVRO" + } + }, + "CTAS_S2_0.Aggregate.Project" : { + "schema" : "`ID` BIGINT KEY, `VALUE` BYTES", + "keyFormat" : { + "format" : "KAFKA" + }, + "valueFormat" : { + "format" : "AVRO" + } + }, + "CTAS_S2_0.KsqlTopic.Source" : { + "schema" : "`ID` BIGINT KEY, `VALUE` BYTES", + "keyFormat" : { + "format" : "KAFKA" + }, + "valueFormat" : { + "format" : "AVRO" + } + }, + "CTAS_S2_0.S2" : { + "schema" : "`ID` BIGINT KEY, `VALUE` BYTES", + "keyFormat" : { + "format" : "KAFKA" + }, + "valueFormat" : { + "format" : "AVRO" + } + }, + "CTAS_S2_0.Aggregate.GroupBy" : { + "schema" : "`ID` BIGINT KEY, `ID` BIGINT, `VALUE` BYTES", + "keyFormat" : { + "format" : "KAFKA" + }, + "valueFormat" : { + "format" : "AVRO" + } + } + }, + "testCase" : { + "name" : "max bytes group by", + "inputs" : [ { + "topic" : "test_topic", + "key" : 1, + "value" : { + "value" : null + } + }, { + "topic" : "test_topic", + "key" : 0, + "value" : { + "value" : null + } + }, { + "topic" : "test_topic", + "key" : 0, + "value" : { + "value" : "Qg==" + } + }, { + "topic" : "test_topic", + "key" : 0, + "value" : { + "value" : "QQ==" + } + }, { + "topic" : "test_topic", + "key" : 0, + "value" : { + "value" : null + } + }, { + "topic" : "test_topic", + "key" : 100, + "value" : { + "value" : "RQ==" + } + }, { + "topic" : "test_topic", + "key" : 100, + "value" : { + "value" : "Qw==" + } + }, { + "topic" : "test_topic", + "key" : 100, + "value" : { + "value" : "Rw==" + } + }, { + "topic" : "test_topic", + "key" : 0, + "value" : { + "value" : "Rg==" + } + }, { + "topic" : "test_topic", + "key" : 0, + "value" : { + "value" : "RA==" + } + } ], + "outputs" : [ { + "topic" : "S2", + "key" : 1, + "value" : { + "VALUE" : null + } + }, { + "topic" : "S2", + "key" : 0, + "value" : { + "VALUE" : null + } + }, { + "topic" : "S2", + "key" : 0, + "value" : { + "VALUE" : "Qg==" + } + }, { + "topic" : "S2", + "key" : 0, + "value" : { + "VALUE" : "Qg==" + } + }, { + "topic" : "S2", + "key" : 0, + "value" : { + "VALUE" : "Qg==" + } + }, { + "topic" : "S2", + "key" : 100, + "value" : { + "VALUE" : "RQ==" + } + }, { + "topic" : "S2", + "key" : 100, + "value" : { + "VALUE" : "RQ==" + } + }, { + "topic" : "S2", + "key" : 100, + "value" : { + "VALUE" : "Rw==" + } + }, { + "topic" : "S2", + "key" : 0, + "value" : { + "VALUE" : "Rg==" + } + }, { + "topic" : "S2", + "key" : 0, + "value" : { + "VALUE" : "Rg==" + } + } ], + "topics" : [ { + "name" : "test_topic", + "valueSchema" : { + "type" : "record", + "name" : "KsqlDataSourceSchema", + "namespace" : "io.confluent.ksql.avro_schemas", + "fields" : [ { + "name" : "VALUE", + "type" : [ "null", "bytes" ], + "default" : null + } ], + "connect.name" : "io.confluent.ksql.avro_schemas.KsqlDataSourceSchema" + }, + "valueFormat" : "AVRO", + "replicas" : 1, + "numPartitions" : 4 + }, { + "name" : "S2", + "replicas" : 1, + "numPartitions" : 4 + } ], + "statements" : [ "CREATE STREAM TEST (ID BIGINT KEY, VALUE BYTES) WITH (kafka_topic='test_topic',value_format='AVRO');", "CREATE TABLE S2 as SELECT ID, max(value) as value FROM test group by id;" ], + "post" : { + "sources" : [ { + "name" : "S2", + "type" : "TABLE", + "schema" : "`ID` BIGINT KEY, `VALUE` BYTES", + "keyFormat" : { + "format" : "KAFKA" + }, + "valueFormat" : "AVRO", + "keyFeatures" : [ ], + "valueFeatures" : [ ], + "isSource" : false + }, { + "name" : "TEST", + "type" : "STREAM", + "schema" : "`ID` BIGINT KEY, `VALUE` BYTES", + "keyFormat" : { + "format" : "KAFKA" + }, + "valueFormat" : "AVRO", + "keyFeatures" : [ ], + "valueFeatures" : [ ], + "isSource" : false + } ], + "topics" : { + "topics" : [ { + "name" : "_confluent-ksql-some.ksql.service.idquery_CTAS_S2_0-Aggregate-Aggregate-Materialize-changelog", + "keyFormat" : { + "format" : "KAFKA" + }, + "valueFormat" : { + "format" : "AVRO" + }, + "valueSchema" : { + "type" : "record", + "name" : "KsqlDataSourceSchema", + "namespace" : "io.confluent.ksql.avro_schemas", + "fields" : [ { + "name" : "ID", + "type" : [ "null", "long" ], + "default" : null + }, { + "name" : "VALUE", + "type" : [ "null", "bytes" ], + "default" : null + }, { + "name" : "KSQL_AGG_VARIABLE_0", + "type" : [ "null", "bytes" ], + "default" : null + } ] + } + }, { + "name" : "test_topic", + "keyFormat" : { + "format" : "KAFKA" + }, + "valueFormat" : { + "format" : "AVRO" + }, + "partitions" : 4, + "valueSchema" : { + "type" : "record", + "name" : "KsqlDataSourceSchema", + "namespace" : "io.confluent.ksql.avro_schemas", + "fields" : [ { + "name" : "VALUE", + "type" : [ "null", "bytes" ], + "default" : null + } ], + "connect.name" : "io.confluent.ksql.avro_schemas.KsqlDataSourceSchema" + } + }, { + "name" : "S2", + "keyFormat" : { + "format" : "KAFKA" + }, + "valueFormat" : { + "format" : "AVRO" + }, + "partitions" : 4, + "valueSchema" : { + "type" : "record", + "name" : "KsqlDataSourceSchema", + "namespace" : "io.confluent.ksql.avro_schemas", + "fields" : [ { + "name" : "VALUE", + "type" : [ "null", "bytes" ], + "default" : null + } ] + } + } ] + } + } + } +} \ No newline at end of file diff --git a/ksqldb-functional-tests/src/test/resources/historical_plans/max-group-by_-_max_bytes_group_by/7.3.0_1651157910151/topology b/ksqldb-functional-tests/src/test/resources/historical_plans/max-group-by_-_max_bytes_group_by/7.3.0_1651157910151/topology new file mode 100644 index 000000000000..fcfc8fb28665 --- /dev/null +++ b/ksqldb-functional-tests/src/test/resources/historical_plans/max-group-by_-_max_bytes_group_by/7.3.0_1651157910151/topology @@ -0,0 +1,25 @@ +Topologies: + Sub-topology: 0 + Source: KSTREAM-SOURCE-0000000000 (topics: [test_topic]) + --> KSTREAM-TRANSFORMVALUES-0000000001 + Processor: KSTREAM-TRANSFORMVALUES-0000000001 (stores: []) + --> Aggregate-Prepare + <-- KSTREAM-SOURCE-0000000000 + Processor: Aggregate-Prepare (stores: []) + --> KSTREAM-AGGREGATE-0000000003 + <-- KSTREAM-TRANSFORMVALUES-0000000001 + Processor: KSTREAM-AGGREGATE-0000000003 (stores: [Aggregate-Aggregate-Materialize]) + --> Aggregate-Aggregate-ToOutputSchema + <-- Aggregate-Prepare + Processor: Aggregate-Aggregate-ToOutputSchema (stores: []) + --> Aggregate-Project + <-- KSTREAM-AGGREGATE-0000000003 + Processor: Aggregate-Project (stores: []) + --> KTABLE-TOSTREAM-0000000006 + <-- Aggregate-Aggregate-ToOutputSchema + Processor: KTABLE-TOSTREAM-0000000006 (stores: []) + --> KSTREAM-SINK-0000000007 + <-- Aggregate-Project + Sink: KSTREAM-SINK-0000000007 (topic: S2) + <-- KTABLE-TOSTREAM-0000000006 + diff --git a/ksqldb-functional-tests/src/test/resources/historical_plans/max-group-by_-_max_string_group_by/7.3.0_1651157908841/plan.json b/ksqldb-functional-tests/src/test/resources/historical_plans/max-group-by_-_max_string_group_by/7.3.0_1651157908841/plan.json new file mode 100644 index 000000000000..23a004085b81 --- /dev/null +++ b/ksqldb-functional-tests/src/test/resources/historical_plans/max-group-by_-_max_string_group_by/7.3.0_1651157908841/plan.json @@ -0,0 +1,255 @@ +{ + "plan" : [ { + "@type" : "ksqlPlanV1", + "statementText" : "CREATE STREAM TEST (ID BIGINT KEY, VALUE STRING) WITH (KAFKA_TOPIC='test_topic', KEY_FORMAT='KAFKA', VALUE_FORMAT='AVRO');", + "ddlCommand" : { + "@type" : "createStreamV1", + "sourceName" : "TEST", + "schema" : "`ID` BIGINT KEY, `VALUE` STRING", + "timestampColumn" : null, + "topicName" : "test_topic", + "formats" : { + "keyFormat" : { + "format" : "KAFKA", + "properties" : { } + }, + "valueFormat" : { + "format" : "AVRO", + "properties" : { } + } + }, + "windowInfo" : null, + "orReplace" : false, + "isSource" : false + }, + "queryPlan" : null + }, { + "@type" : "ksqlPlanV1", + "statementText" : "CREATE TABLE S2 AS SELECT\n TEST.ID ID,\n MAX(TEST.VALUE) VALUE\nFROM TEST TEST\nGROUP BY TEST.ID\nEMIT CHANGES", + "ddlCommand" : { + "@type" : "createTableV1", + "sourceName" : "S2", + "schema" : "`ID` BIGINT KEY, `VALUE` STRING", + "timestampColumn" : null, + "topicName" : "S2", + "formats" : { + "keyFormat" : { + "format" : "KAFKA", + "properties" : { } + }, + "valueFormat" : { + "format" : "AVRO", + "properties" : { } + } + }, + "windowInfo" : null, + "orReplace" : false, + "isSource" : false + }, + "queryPlan" : { + "sources" : [ "TEST" ], + "sink" : "S2", + "physicalPlan" : { + "@type" : "tableSinkV1", + "properties" : { + "queryContext" : "S2" + }, + "source" : { + "@type" : "tableSelectV1", + "properties" : { + "queryContext" : "Aggregate/Project" + }, + "source" : { + "@type" : "streamAggregateV1", + "properties" : { + "queryContext" : "Aggregate/Aggregate" + }, + "source" : { + "@type" : "streamGroupByKeyV1", + "properties" : { + "queryContext" : "Aggregate/GroupBy" + }, + "source" : { + "@type" : "streamSelectV1", + "properties" : { + "queryContext" : "Aggregate/Prepare" + }, + "source" : { + "@type" : "streamSourceV1", + "properties" : { + "queryContext" : "KsqlTopic/Source" + }, + "topicName" : "test_topic", + "formats" : { + "keyFormat" : { + "format" : "KAFKA", + "properties" : { } + }, + "valueFormat" : { + "format" : "AVRO", + "properties" : { } + } + }, + "timestampColumn" : null, + "sourceSchema" : "`ID` BIGINT KEY, `VALUE` STRING", + "pseudoColumnVersion" : 1 + }, + "keyColumnNames" : [ "ID" ], + "selectedKeys" : null, + "selectExpressions" : [ "ID AS ID", "VALUE AS VALUE" ] + }, + "internalFormats" : { + "keyFormat" : { + "format" : "KAFKA", + "properties" : { } + }, + "valueFormat" : { + "format" : "AVRO", + "properties" : { } + } + } + }, + "internalFormats" : { + "keyFormat" : { + "format" : "KAFKA", + "properties" : { } + }, + "valueFormat" : { + "format" : "AVRO", + "properties" : { } + } + }, + "nonAggregateColumns" : [ "ID", "VALUE" ], + "aggregationFunctions" : [ "MAX(VALUE)" ] + }, + "keyColumnNames" : [ "ID" ], + "selectExpressions" : [ "KSQL_AGG_VARIABLE_0 AS VALUE" ], + "internalFormats" : { + "keyFormat" : { + "format" : "KAFKA", + "properties" : { } + }, + "valueFormat" : { + "format" : "AVRO", + "properties" : { } + } + } + }, + "formats" : { + "keyFormat" : { + "format" : "KAFKA", + "properties" : { } + }, + "valueFormat" : { + "format" : "AVRO", + "properties" : { } + } + }, + "topicName" : "S2", + "timestampColumn" : null + }, + "queryId" : "CTAS_S2_0", + "runtimeId" : null + } + } ], + "configs" : { + "ksql.extension.dir" : "ext", + "ksql.streams.cache.max.bytes.buffering" : "0", + "metric.reporters" : "", + "ksql.query.status.running.threshold.seconds" : "300", + "ksql.connect.basic.auth.credentials.reload" : "false", + "ksql.output.topic.name.prefix" : "", + "ksql.query.pull.stream.enabled" : "true", + "ksql.query.push.v2.interpreter.enabled" : "true", + "ksql.queryanonymizer.logs_enabled" : "true", + "ksql.variable.substitution.enable" : "true", + "ksql.streams.shutdown.timeout.ms" : "300000", + "ksql.query.pull.max.allowed.offset.lag" : "9223372036854775807", + "ksql.query.pull.max.qps" : "2147483647", + "ksql.access.validator.enable" : "auto", + "ksql.streams.bootstrap.servers" : "localhost:0", + "ksql.query.pull.metrics.enabled" : "true", + "ksql.metrics.extension" : null, + "ksql.query.push.v2.alos.enabled" : "true", + "ksql.query.push.v2.max.hourly.bandwidth.megabytes" : "2147483647", + "ksql.query.pull.range.scan.enabled" : "true", + "ksql.transient.query.cleanup.service.initial.delay.seconds" : "600", + "ksql.hidden.topics" : "_confluent.*,__confluent.*,_schemas,__consumer_offsets,__transaction_state,connect-configs,connect-offsets,connect-status,connect-statuses", + "ksql.lambdas.enabled" : "true", + "ksql.source.table.materialization.enabled" : "true", + "ksql.query.pull.max.hourly.bandwidth.megabytes" : "2147483647", + "ksql.sink.window.change.log.additional.retention" : "1000000", + "ksql.query.persistent.active.limit" : "2147483647", + "ksql.persistence.wrap.single.values" : null, + "ksql.query.transient.max.bytes.buffering.total" : "-1", + "ksql.connect.basic.auth.credentials.source" : "NONE", + "ksql.schema.registry.url" : "schema_registry.url:0", + "ksql.properties.overrides.denylist" : "", + "ksql.service.id" : "some.ksql.service.id", + "ksql.query.push.v2.max.catchup.consumers" : "5", + "ksql.query.push.v2.enabled" : "false", + "ksql.transient.query.cleanup.service.enable" : "true", + "ksql.query.push.v2.metrics.enabled" : "true", + "ksql.rowpartition.rowoffset.enabled" : "true", + "ksql.streams.commit.interval.ms" : "2000", + "ksql.query.pull.table.scan.enabled" : "true", + "ksql.streams.auto.commit.interval.ms" : "0", + "ksql.streams.topology.optimization" : "all", + "ksql.endpoint.migrate.query" : "true", + "ksql.query.push.v2.registry.installed" : "false", + "ksql.streams.num.stream.threads" : "4", + "ksql.metrics.tags.custom" : "", + "ksql.query.push.v2.catchup.consumer.msg.window" : "50", + "ksql.runtime.feature.shared.enabled" : "false", + "ksql.udf.collect.metrics" : "false", + "ksql.new.query.planner.enabled" : "false", + "ksql.connect.request.headers.plugin" : null, + "ksql.security.extension.class" : null, + "ksql.transient.prefix" : "transient_", + "ksql.headers.columns.enabled" : "true", + "ksql.streams.default.deserialization.exception.handler" : "io.confluent.ksql.errors.LogMetricAndContinueExceptionHandler", + "ksql.connect.request.timeout.ms" : "5000", + "ksql.query.pull.enable.standby.reads" : "false", + "ksql.persistence.default.format.key" : "KAFKA", + "ksql.query.persistent.max.bytes.buffering.total" : "-1", + "ksql.query.error.max.queue.size" : "10", + "ksql.query.cleanup.shutdown.timeout.ms" : "30000", + "ksql.internal.topic.min.insync.replicas" : "1", + "ksql.internal.topic.replicas" : "1", + "ksql.insert.into.values.enabled" : "true", + "ksql.queryanonymizer.cluster_namespace" : null, + "ksql.create.or.replace.enabled" : "true", + "ksql.shared.runtimes.count" : "2", + "ksql.cast.strings.preserve.nulls" : "true", + "ksql.authorization.cache.max.entries" : "10000", + "ksql.pull.queries.enable" : "true", + "ksql.transient.query.cleanup.service.period.seconds" : "600", + "ksql.suppress.enabled" : "false", + "ksql.readonly.topics" : "_confluent.*,__confluent.*,_schemas,__consumer_offsets,__transaction_state,connect-configs,connect-offsets,connect-status,connect-statuses", + "ksql.connect.basic.auth.credentials.file" : "", + "ksql.authorization.cache.expiry.time.secs" : "30", + "ksql.query.retry.backoff.initial.ms" : "15000", + "ksql.query.pull.max.concurrent.requests" : "2147483647", + "ksql.streams.auto.offset.reset" : "earliest", + "ksql.connect.url" : "http://localhost:8083", + "ksql.query.push.v2.new.latest.delay.ms" : "5000", + "ksql.query.push.v2.latest.reset.age.ms" : "30000", + "ksql.streams.default.production.exception.handler" : "io.confluent.ksql.errors.ProductionExceptionHandlerUtil$LogAndFailProductionExceptionHandler", + "ksql.query.pull.interpreter.enabled" : "true", + "ksql.query.pull.limit.clause.enabled" : "true", + "ksql.query.pull.router.thread.pool.size" : "50", + "ksql.query.push.v2.continuation.tokens.enabled" : "false", + "ksql.query.retry.backoff.max.ms" : "900000", + "ksql.timestamp.throw.on.invalid" : "false", + "ksql.persistence.default.format.value" : null, + "ksql.udfs.enabled" : "true", + "ksql.udf.enable.security.manager" : "true", + "ksql.connect.worker.config" : "", + "ksql.nested.error.set.null" : "true", + "ksql.query.pull.thread.pool.size" : "50", + "ksql.persistent.prefix" : "query_", + "ksql.metastore.backup.location" : "", + "ksql.error.classifier.regex" : "", + "ksql.suppress.buffer.size.bytes" : "-1" + } +} \ No newline at end of file diff --git a/ksqldb-functional-tests/src/test/resources/historical_plans/max-group-by_-_max_string_group_by/7.3.0_1651157908841/spec.json b/ksqldb-functional-tests/src/test/resources/historical_plans/max-group-by_-_max_string_group_by/7.3.0_1651157908841/spec.json new file mode 100644 index 000000000000..90f67feead7c --- /dev/null +++ b/ksqldb-functional-tests/src/test/resources/historical_plans/max-group-by_-_max_string_group_by/7.3.0_1651157908841/spec.json @@ -0,0 +1,292 @@ +{ + "version" : "7.3.0", + "timestamp" : 1651157908841, + "path" : "query-validation-tests/max-group-by.json", + "schemas" : { + "CTAS_S2_0.Aggregate.Aggregate.Materialize" : { + "schema" : "`ID` BIGINT KEY, `ID` BIGINT, `VALUE` STRING, `KSQL_AGG_VARIABLE_0` STRING", + "keyFormat" : { + "format" : "KAFKA" + }, + "valueFormat" : { + "format" : "AVRO" + } + }, + "CTAS_S2_0.Aggregate.Project" : { + "schema" : "`ID` BIGINT KEY, `VALUE` STRING", + "keyFormat" : { + "format" : "KAFKA" + }, + "valueFormat" : { + "format" : "AVRO" + } + }, + "CTAS_S2_0.KsqlTopic.Source" : { + "schema" : "`ID` BIGINT KEY, `VALUE` STRING", + "keyFormat" : { + "format" : "KAFKA" + }, + "valueFormat" : { + "format" : "AVRO" + } + }, + "CTAS_S2_0.S2" : { + "schema" : "`ID` BIGINT KEY, `VALUE` STRING", + "keyFormat" : { + "format" : "KAFKA" + }, + "valueFormat" : { + "format" : "AVRO" + } + }, + "CTAS_S2_0.Aggregate.GroupBy" : { + "schema" : "`ID` BIGINT KEY, `ID` BIGINT, `VALUE` STRING", + "keyFormat" : { + "format" : "KAFKA" + }, + "valueFormat" : { + "format" : "AVRO" + } + } + }, + "testCase" : { + "name" : "max string group by", + "inputs" : [ { + "topic" : "test_topic", + "key" : 1, + "value" : { + "value" : null + } + }, { + "topic" : "test_topic", + "key" : 0, + "value" : { + "value" : null + } + }, { + "topic" : "test_topic", + "key" : 0, + "value" : { + "value" : "B" + } + }, { + "topic" : "test_topic", + "key" : 0, + "value" : { + "value" : "A" + } + }, { + "topic" : "test_topic", + "key" : 0, + "value" : { + "value" : null + } + }, { + "topic" : "test_topic", + "key" : 100, + "value" : { + "value" : "E" + } + }, { + "topic" : "test_topic", + "key" : 100, + "value" : { + "value" : "C" + } + }, { + "topic" : "test_topic", + "key" : 100, + "value" : { + "value" : "G" + } + }, { + "topic" : "test_topic", + "key" : 0, + "value" : { + "value" : "F" + } + }, { + "topic" : "test_topic", + "key" : 0, + "value" : { + "value" : "D" + } + } ], + "outputs" : [ { + "topic" : "S2", + "key" : 1, + "value" : { + "VALUE" : null + } + }, { + "topic" : "S2", + "key" : 0, + "value" : { + "VALUE" : null + } + }, { + "topic" : "S2", + "key" : 0, + "value" : { + "VALUE" : "B" + } + }, { + "topic" : "S2", + "key" : 0, + "value" : { + "VALUE" : "B" + } + }, { + "topic" : "S2", + "key" : 0, + "value" : { + "VALUE" : "B" + } + }, { + "topic" : "S2", + "key" : 100, + "value" : { + "VALUE" : "E" + } + }, { + "topic" : "S2", + "key" : 100, + "value" : { + "VALUE" : "E" + } + }, { + "topic" : "S2", + "key" : 100, + "value" : { + "VALUE" : "G" + } + }, { + "topic" : "S2", + "key" : 0, + "value" : { + "VALUE" : "F" + } + }, { + "topic" : "S2", + "key" : 0, + "value" : { + "VALUE" : "F" + } + } ], + "topics" : [ { + "name" : "test_topic", + "valueSchema" : { + "type" : "record", + "name" : "KsqlDataSourceSchema", + "namespace" : "io.confluent.ksql.avro_schemas", + "fields" : [ { + "name" : "VALUE", + "type" : [ "null", "string" ], + "default" : null + } ], + "connect.name" : "io.confluent.ksql.avro_schemas.KsqlDataSourceSchema" + }, + "valueFormat" : "AVRO", + "replicas" : 1, + "numPartitions" : 4 + }, { + "name" : "S2", + "replicas" : 1, + "numPartitions" : 4 + } ], + "statements" : [ "CREATE STREAM TEST (ID BIGINT KEY, VALUE STRING) WITH (kafka_topic='test_topic',value_format='AVRO');", "CREATE TABLE S2 as SELECT ID, max(value) as value FROM test group by id;" ], + "post" : { + "sources" : [ { + "name" : "S2", + "type" : "TABLE", + "schema" : "`ID` BIGINT KEY, `VALUE` STRING", + "keyFormat" : { + "format" : "KAFKA" + }, + "valueFormat" : "AVRO", + "keyFeatures" : [ ], + "valueFeatures" : [ ], + "isSource" : false + }, { + "name" : "TEST", + "type" : "STREAM", + "schema" : "`ID` BIGINT KEY, `VALUE` STRING", + "keyFormat" : { + "format" : "KAFKA" + }, + "valueFormat" : "AVRO", + "keyFeatures" : [ ], + "valueFeatures" : [ ], + "isSource" : false + } ], + "topics" : { + "topics" : [ { + "name" : "_confluent-ksql-some.ksql.service.idquery_CTAS_S2_0-Aggregate-Aggregate-Materialize-changelog", + "keyFormat" : { + "format" : "KAFKA" + }, + "valueFormat" : { + "format" : "AVRO" + }, + "valueSchema" : { + "type" : "record", + "name" : "KsqlDataSourceSchema", + "namespace" : "io.confluent.ksql.avro_schemas", + "fields" : [ { + "name" : "ID", + "type" : [ "null", "long" ], + "default" : null + }, { + "name" : "VALUE", + "type" : [ "null", "string" ], + "default" : null + }, { + "name" : "KSQL_AGG_VARIABLE_0", + "type" : [ "null", "string" ], + "default" : null + } ] + } + }, { + "name" : "test_topic", + "keyFormat" : { + "format" : "KAFKA" + }, + "valueFormat" : { + "format" : "AVRO" + }, + "partitions" : 4, + "valueSchema" : { + "type" : "record", + "name" : "KsqlDataSourceSchema", + "namespace" : "io.confluent.ksql.avro_schemas", + "fields" : [ { + "name" : "VALUE", + "type" : [ "null", "string" ], + "default" : null + } ], + "connect.name" : "io.confluent.ksql.avro_schemas.KsqlDataSourceSchema" + } + }, { + "name" : "S2", + "keyFormat" : { + "format" : "KAFKA" + }, + "valueFormat" : { + "format" : "AVRO" + }, + "partitions" : 4, + "valueSchema" : { + "type" : "record", + "name" : "KsqlDataSourceSchema", + "namespace" : "io.confluent.ksql.avro_schemas", + "fields" : [ { + "name" : "VALUE", + "type" : [ "null", "string" ], + "default" : null + } ] + } + } ] + } + } + } +} \ No newline at end of file diff --git a/ksqldb-functional-tests/src/test/resources/historical_plans/max-group-by_-_max_string_group_by/7.3.0_1651157908841/topology b/ksqldb-functional-tests/src/test/resources/historical_plans/max-group-by_-_max_string_group_by/7.3.0_1651157908841/topology new file mode 100644 index 000000000000..fcfc8fb28665 --- /dev/null +++ b/ksqldb-functional-tests/src/test/resources/historical_plans/max-group-by_-_max_string_group_by/7.3.0_1651157908841/topology @@ -0,0 +1,25 @@ +Topologies: + Sub-topology: 0 + Source: KSTREAM-SOURCE-0000000000 (topics: [test_topic]) + --> KSTREAM-TRANSFORMVALUES-0000000001 + Processor: KSTREAM-TRANSFORMVALUES-0000000001 (stores: []) + --> Aggregate-Prepare + <-- KSTREAM-SOURCE-0000000000 + Processor: Aggregate-Prepare (stores: []) + --> KSTREAM-AGGREGATE-0000000003 + <-- KSTREAM-TRANSFORMVALUES-0000000001 + Processor: KSTREAM-AGGREGATE-0000000003 (stores: [Aggregate-Aggregate-Materialize]) + --> Aggregate-Aggregate-ToOutputSchema + <-- Aggregate-Prepare + Processor: Aggregate-Aggregate-ToOutputSchema (stores: []) + --> Aggregate-Project + <-- KSTREAM-AGGREGATE-0000000003 + Processor: Aggregate-Project (stores: []) + --> KTABLE-TOSTREAM-0000000006 + <-- Aggregate-Aggregate-ToOutputSchema + Processor: KTABLE-TOSTREAM-0000000006 (stores: []) + --> KSTREAM-SINK-0000000007 + <-- Aggregate-Project + Sink: KSTREAM-SINK-0000000007 (topic: S2) + <-- KTABLE-TOSTREAM-0000000006 + diff --git a/ksqldb-functional-tests/src/test/resources/historical_plans/min-group-by_-_min_bytes_group_by/7.3.0_1651157920145/plan.json b/ksqldb-functional-tests/src/test/resources/historical_plans/min-group-by_-_min_bytes_group_by/7.3.0_1651157920145/plan.json new file mode 100644 index 000000000000..edeed71dd9d3 --- /dev/null +++ b/ksqldb-functional-tests/src/test/resources/historical_plans/min-group-by_-_min_bytes_group_by/7.3.0_1651157920145/plan.json @@ -0,0 +1,255 @@ +{ + "plan" : [ { + "@type" : "ksqlPlanV1", + "statementText" : "CREATE STREAM TEST (ID BIGINT KEY, VALUE BYTES) WITH (KAFKA_TOPIC='test_topic', KEY_FORMAT='KAFKA', VALUE_FORMAT='AVRO');", + "ddlCommand" : { + "@type" : "createStreamV1", + "sourceName" : "TEST", + "schema" : "`ID` BIGINT KEY, `VALUE` BYTES", + "timestampColumn" : null, + "topicName" : "test_topic", + "formats" : { + "keyFormat" : { + "format" : "KAFKA", + "properties" : { } + }, + "valueFormat" : { + "format" : "AVRO", + "properties" : { } + } + }, + "windowInfo" : null, + "orReplace" : false, + "isSource" : false + }, + "queryPlan" : null + }, { + "@type" : "ksqlPlanV1", + "statementText" : "CREATE TABLE S2 AS SELECT\n TEST.ID ID,\n MIN(TEST.VALUE) VALUE\nFROM TEST TEST\nGROUP BY TEST.ID\nEMIT CHANGES", + "ddlCommand" : { + "@type" : "createTableV1", + "sourceName" : "S2", + "schema" : "`ID` BIGINT KEY, `VALUE` BYTES", + "timestampColumn" : null, + "topicName" : "S2", + "formats" : { + "keyFormat" : { + "format" : "KAFKA", + "properties" : { } + }, + "valueFormat" : { + "format" : "AVRO", + "properties" : { } + } + }, + "windowInfo" : null, + "orReplace" : false, + "isSource" : false + }, + "queryPlan" : { + "sources" : [ "TEST" ], + "sink" : "S2", + "physicalPlan" : { + "@type" : "tableSinkV1", + "properties" : { + "queryContext" : "S2" + }, + "source" : { + "@type" : "tableSelectV1", + "properties" : { + "queryContext" : "Aggregate/Project" + }, + "source" : { + "@type" : "streamAggregateV1", + "properties" : { + "queryContext" : "Aggregate/Aggregate" + }, + "source" : { + "@type" : "streamGroupByKeyV1", + "properties" : { + "queryContext" : "Aggregate/GroupBy" + }, + "source" : { + "@type" : "streamSelectV1", + "properties" : { + "queryContext" : "Aggregate/Prepare" + }, + "source" : { + "@type" : "streamSourceV1", + "properties" : { + "queryContext" : "KsqlTopic/Source" + }, + "topicName" : "test_topic", + "formats" : { + "keyFormat" : { + "format" : "KAFKA", + "properties" : { } + }, + "valueFormat" : { + "format" : "AVRO", + "properties" : { } + } + }, + "timestampColumn" : null, + "sourceSchema" : "`ID` BIGINT KEY, `VALUE` BYTES", + "pseudoColumnVersion" : 1 + }, + "keyColumnNames" : [ "ID" ], + "selectedKeys" : null, + "selectExpressions" : [ "ID AS ID", "VALUE AS VALUE" ] + }, + "internalFormats" : { + "keyFormat" : { + "format" : "KAFKA", + "properties" : { } + }, + "valueFormat" : { + "format" : "AVRO", + "properties" : { } + } + } + }, + "internalFormats" : { + "keyFormat" : { + "format" : "KAFKA", + "properties" : { } + }, + "valueFormat" : { + "format" : "AVRO", + "properties" : { } + } + }, + "nonAggregateColumns" : [ "ID", "VALUE" ], + "aggregationFunctions" : [ "MIN(VALUE)" ] + }, + "keyColumnNames" : [ "ID" ], + "selectExpressions" : [ "KSQL_AGG_VARIABLE_0 AS VALUE" ], + "internalFormats" : { + "keyFormat" : { + "format" : "KAFKA", + "properties" : { } + }, + "valueFormat" : { + "format" : "AVRO", + "properties" : { } + } + } + }, + "formats" : { + "keyFormat" : { + "format" : "KAFKA", + "properties" : { } + }, + "valueFormat" : { + "format" : "AVRO", + "properties" : { } + } + }, + "topicName" : "S2", + "timestampColumn" : null + }, + "queryId" : "CTAS_S2_0", + "runtimeId" : null + } + } ], + "configs" : { + "ksql.extension.dir" : "ext", + "ksql.streams.cache.max.bytes.buffering" : "0", + "metric.reporters" : "", + "ksql.query.status.running.threshold.seconds" : "300", + "ksql.connect.basic.auth.credentials.reload" : "false", + "ksql.output.topic.name.prefix" : "", + "ksql.query.pull.stream.enabled" : "true", + "ksql.query.push.v2.interpreter.enabled" : "true", + "ksql.queryanonymizer.logs_enabled" : "true", + "ksql.variable.substitution.enable" : "true", + "ksql.streams.shutdown.timeout.ms" : "300000", + "ksql.query.pull.max.allowed.offset.lag" : "9223372036854775807", + "ksql.query.pull.max.qps" : "2147483647", + "ksql.access.validator.enable" : "auto", + "ksql.streams.bootstrap.servers" : "localhost:0", + "ksql.query.pull.metrics.enabled" : "true", + "ksql.metrics.extension" : null, + "ksql.query.push.v2.alos.enabled" : "true", + "ksql.query.push.v2.max.hourly.bandwidth.megabytes" : "2147483647", + "ksql.query.pull.range.scan.enabled" : "true", + "ksql.transient.query.cleanup.service.initial.delay.seconds" : "600", + "ksql.hidden.topics" : "_confluent.*,__confluent.*,_schemas,__consumer_offsets,__transaction_state,connect-configs,connect-offsets,connect-status,connect-statuses", + "ksql.lambdas.enabled" : "true", + "ksql.source.table.materialization.enabled" : "true", + "ksql.query.pull.max.hourly.bandwidth.megabytes" : "2147483647", + "ksql.sink.window.change.log.additional.retention" : "1000000", + "ksql.query.persistent.active.limit" : "2147483647", + "ksql.persistence.wrap.single.values" : null, + "ksql.query.transient.max.bytes.buffering.total" : "-1", + "ksql.connect.basic.auth.credentials.source" : "NONE", + "ksql.schema.registry.url" : "schema_registry.url:0", + "ksql.properties.overrides.denylist" : "", + "ksql.service.id" : "some.ksql.service.id", + "ksql.query.push.v2.max.catchup.consumers" : "5", + "ksql.query.push.v2.enabled" : "false", + "ksql.transient.query.cleanup.service.enable" : "true", + "ksql.query.push.v2.metrics.enabled" : "true", + "ksql.rowpartition.rowoffset.enabled" : "true", + "ksql.streams.commit.interval.ms" : "2000", + "ksql.query.pull.table.scan.enabled" : "true", + "ksql.streams.auto.commit.interval.ms" : "0", + "ksql.streams.topology.optimization" : "all", + "ksql.endpoint.migrate.query" : "true", + "ksql.query.push.v2.registry.installed" : "false", + "ksql.streams.num.stream.threads" : "4", + "ksql.metrics.tags.custom" : "", + "ksql.query.push.v2.catchup.consumer.msg.window" : "50", + "ksql.runtime.feature.shared.enabled" : "false", + "ksql.udf.collect.metrics" : "false", + "ksql.new.query.planner.enabled" : "false", + "ksql.connect.request.headers.plugin" : null, + "ksql.security.extension.class" : null, + "ksql.transient.prefix" : "transient_", + "ksql.headers.columns.enabled" : "true", + "ksql.streams.default.deserialization.exception.handler" : "io.confluent.ksql.errors.LogMetricAndContinueExceptionHandler", + "ksql.connect.request.timeout.ms" : "5000", + "ksql.query.pull.enable.standby.reads" : "false", + "ksql.persistence.default.format.key" : "KAFKA", + "ksql.query.persistent.max.bytes.buffering.total" : "-1", + "ksql.query.error.max.queue.size" : "10", + "ksql.query.cleanup.shutdown.timeout.ms" : "30000", + "ksql.internal.topic.min.insync.replicas" : "1", + "ksql.internal.topic.replicas" : "1", + "ksql.insert.into.values.enabled" : "true", + "ksql.queryanonymizer.cluster_namespace" : null, + "ksql.create.or.replace.enabled" : "true", + "ksql.shared.runtimes.count" : "2", + "ksql.cast.strings.preserve.nulls" : "true", + "ksql.authorization.cache.max.entries" : "10000", + "ksql.pull.queries.enable" : "true", + "ksql.transient.query.cleanup.service.period.seconds" : "600", + "ksql.suppress.enabled" : "false", + "ksql.readonly.topics" : "_confluent.*,__confluent.*,_schemas,__consumer_offsets,__transaction_state,connect-configs,connect-offsets,connect-status,connect-statuses", + "ksql.connect.basic.auth.credentials.file" : "", + "ksql.authorization.cache.expiry.time.secs" : "30", + "ksql.query.retry.backoff.initial.ms" : "15000", + "ksql.query.pull.max.concurrent.requests" : "2147483647", + "ksql.streams.auto.offset.reset" : "earliest", + "ksql.connect.url" : "http://localhost:8083", + "ksql.query.push.v2.new.latest.delay.ms" : "5000", + "ksql.query.push.v2.latest.reset.age.ms" : "30000", + "ksql.streams.default.production.exception.handler" : "io.confluent.ksql.errors.ProductionExceptionHandlerUtil$LogAndFailProductionExceptionHandler", + "ksql.query.pull.interpreter.enabled" : "true", + "ksql.query.pull.limit.clause.enabled" : "true", + "ksql.query.pull.router.thread.pool.size" : "50", + "ksql.query.push.v2.continuation.tokens.enabled" : "false", + "ksql.query.retry.backoff.max.ms" : "900000", + "ksql.timestamp.throw.on.invalid" : "false", + "ksql.persistence.default.format.value" : null, + "ksql.udfs.enabled" : "true", + "ksql.udf.enable.security.manager" : "true", + "ksql.connect.worker.config" : "", + "ksql.nested.error.set.null" : "true", + "ksql.query.pull.thread.pool.size" : "50", + "ksql.persistent.prefix" : "query_", + "ksql.metastore.backup.location" : "", + "ksql.error.classifier.regex" : "", + "ksql.suppress.buffer.size.bytes" : "-1" + } +} \ No newline at end of file diff --git a/ksqldb-functional-tests/src/test/resources/historical_plans/min-group-by_-_min_bytes_group_by/7.3.0_1651157920145/spec.json b/ksqldb-functional-tests/src/test/resources/historical_plans/min-group-by_-_min_bytes_group_by/7.3.0_1651157920145/spec.json new file mode 100644 index 000000000000..dc95e848e98f --- /dev/null +++ b/ksqldb-functional-tests/src/test/resources/historical_plans/min-group-by_-_min_bytes_group_by/7.3.0_1651157920145/spec.json @@ -0,0 +1,292 @@ +{ + "version" : "7.3.0", + "timestamp" : 1651157920145, + "path" : "query-validation-tests/min-group-by.json", + "schemas" : { + "CTAS_S2_0.Aggregate.Aggregate.Materialize" : { + "schema" : "`ID` BIGINT KEY, `ID` BIGINT, `VALUE` BYTES, `KSQL_AGG_VARIABLE_0` BYTES", + "keyFormat" : { + "format" : "KAFKA" + }, + "valueFormat" : { + "format" : "AVRO" + } + }, + "CTAS_S2_0.Aggregate.Project" : { + "schema" : "`ID` BIGINT KEY, `VALUE` BYTES", + "keyFormat" : { + "format" : "KAFKA" + }, + "valueFormat" : { + "format" : "AVRO" + } + }, + "CTAS_S2_0.KsqlTopic.Source" : { + "schema" : "`ID` BIGINT KEY, `VALUE` BYTES", + "keyFormat" : { + "format" : "KAFKA" + }, + "valueFormat" : { + "format" : "AVRO" + } + }, + "CTAS_S2_0.S2" : { + "schema" : "`ID` BIGINT KEY, `VALUE` BYTES", + "keyFormat" : { + "format" : "KAFKA" + }, + "valueFormat" : { + "format" : "AVRO" + } + }, + "CTAS_S2_0.Aggregate.GroupBy" : { + "schema" : "`ID` BIGINT KEY, `ID` BIGINT, `VALUE` BYTES", + "keyFormat" : { + "format" : "KAFKA" + }, + "valueFormat" : { + "format" : "AVRO" + } + } + }, + "testCase" : { + "name" : "min bytes group by", + "inputs" : [ { + "topic" : "test_topic", + "key" : 1, + "value" : { + "value" : null + } + }, { + "topic" : "test_topic", + "key" : 0, + "value" : { + "value" : null + } + }, { + "topic" : "test_topic", + "key" : 0, + "value" : { + "value" : "Qg==" + } + }, { + "topic" : "test_topic", + "key" : 0, + "value" : { + "value" : "QQ==" + } + }, { + "topic" : "test_topic", + "key" : 0, + "value" : { + "value" : null + } + }, { + "topic" : "test_topic", + "key" : 100, + "value" : { + "value" : "RQ==" + } + }, { + "topic" : "test_topic", + "key" : 100, + "value" : { + "value" : "Qw==" + } + }, { + "topic" : "test_topic", + "key" : 100, + "value" : { + "value" : "Rw==" + } + }, { + "topic" : "test_topic", + "key" : 0, + "value" : { + "value" : "Rg==" + } + }, { + "topic" : "test_topic", + "key" : 0, + "value" : { + "value" : "RA==" + } + } ], + "outputs" : [ { + "topic" : "S2", + "key" : 1, + "value" : { + "VALUE" : null + } + }, { + "topic" : "S2", + "key" : 0, + "value" : { + "VALUE" : null + } + }, { + "topic" : "S2", + "key" : 0, + "value" : { + "VALUE" : "Qg==" + } + }, { + "topic" : "S2", + "key" : 0, + "value" : { + "VALUE" : "QQ==" + } + }, { + "topic" : "S2", + "key" : 0, + "value" : { + "VALUE" : "QQ==" + } + }, { + "topic" : "S2", + "key" : 100, + "value" : { + "VALUE" : "RQ==" + } + }, { + "topic" : "S2", + "key" : 100, + "value" : { + "VALUE" : "Qw==" + } + }, { + "topic" : "S2", + "key" : 100, + "value" : { + "VALUE" : "Qw==" + } + }, { + "topic" : "S2", + "key" : 0, + "value" : { + "VALUE" : "QQ==" + } + }, { + "topic" : "S2", + "key" : 0, + "value" : { + "VALUE" : "QQ==" + } + } ], + "topics" : [ { + "name" : "test_topic", + "valueSchema" : { + "type" : "record", + "name" : "KsqlDataSourceSchema", + "namespace" : "io.confluent.ksql.avro_schemas", + "fields" : [ { + "name" : "VALUE", + "type" : [ "null", "bytes" ], + "default" : null + } ], + "connect.name" : "io.confluent.ksql.avro_schemas.KsqlDataSourceSchema" + }, + "valueFormat" : "AVRO", + "replicas" : 1, + "numPartitions" : 4 + }, { + "name" : "S2", + "replicas" : 1, + "numPartitions" : 4 + } ], + "statements" : [ "CREATE STREAM TEST (ID BIGINT KEY, VALUE BYTES) WITH (kafka_topic='test_topic',value_format='AVRO');", "CREATE TABLE S2 as SELECT ID, min(value) as value FROM test group by id;" ], + "post" : { + "sources" : [ { + "name" : "S2", + "type" : "TABLE", + "schema" : "`ID` BIGINT KEY, `VALUE` BYTES", + "keyFormat" : { + "format" : "KAFKA" + }, + "valueFormat" : "AVRO", + "keyFeatures" : [ ], + "valueFeatures" : [ ], + "isSource" : false + }, { + "name" : "TEST", + "type" : "STREAM", + "schema" : "`ID` BIGINT KEY, `VALUE` BYTES", + "keyFormat" : { + "format" : "KAFKA" + }, + "valueFormat" : "AVRO", + "keyFeatures" : [ ], + "valueFeatures" : [ ], + "isSource" : false + } ], + "topics" : { + "topics" : [ { + "name" : "_confluent-ksql-some.ksql.service.idquery_CTAS_S2_0-Aggregate-Aggregate-Materialize-changelog", + "keyFormat" : { + "format" : "KAFKA" + }, + "valueFormat" : { + "format" : "AVRO" + }, + "valueSchema" : { + "type" : "record", + "name" : "KsqlDataSourceSchema", + "namespace" : "io.confluent.ksql.avro_schemas", + "fields" : [ { + "name" : "ID", + "type" : [ "null", "long" ], + "default" : null + }, { + "name" : "VALUE", + "type" : [ "null", "bytes" ], + "default" : null + }, { + "name" : "KSQL_AGG_VARIABLE_0", + "type" : [ "null", "bytes" ], + "default" : null + } ] + } + }, { + "name" : "test_topic", + "keyFormat" : { + "format" : "KAFKA" + }, + "valueFormat" : { + "format" : "AVRO" + }, + "partitions" : 4, + "valueSchema" : { + "type" : "record", + "name" : "KsqlDataSourceSchema", + "namespace" : "io.confluent.ksql.avro_schemas", + "fields" : [ { + "name" : "VALUE", + "type" : [ "null", "bytes" ], + "default" : null + } ], + "connect.name" : "io.confluent.ksql.avro_schemas.KsqlDataSourceSchema" + } + }, { + "name" : "S2", + "keyFormat" : { + "format" : "KAFKA" + }, + "valueFormat" : { + "format" : "AVRO" + }, + "partitions" : 4, + "valueSchema" : { + "type" : "record", + "name" : "KsqlDataSourceSchema", + "namespace" : "io.confluent.ksql.avro_schemas", + "fields" : [ { + "name" : "VALUE", + "type" : [ "null", "bytes" ], + "default" : null + } ] + } + } ] + } + } + } +} \ No newline at end of file diff --git a/ksqldb-functional-tests/src/test/resources/historical_plans/min-group-by_-_min_bytes_group_by/7.3.0_1651157920145/topology b/ksqldb-functional-tests/src/test/resources/historical_plans/min-group-by_-_min_bytes_group_by/7.3.0_1651157920145/topology new file mode 100644 index 000000000000..fcfc8fb28665 --- /dev/null +++ b/ksqldb-functional-tests/src/test/resources/historical_plans/min-group-by_-_min_bytes_group_by/7.3.0_1651157920145/topology @@ -0,0 +1,25 @@ +Topologies: + Sub-topology: 0 + Source: KSTREAM-SOURCE-0000000000 (topics: [test_topic]) + --> KSTREAM-TRANSFORMVALUES-0000000001 + Processor: KSTREAM-TRANSFORMVALUES-0000000001 (stores: []) + --> Aggregate-Prepare + <-- KSTREAM-SOURCE-0000000000 + Processor: Aggregate-Prepare (stores: []) + --> KSTREAM-AGGREGATE-0000000003 + <-- KSTREAM-TRANSFORMVALUES-0000000001 + Processor: KSTREAM-AGGREGATE-0000000003 (stores: [Aggregate-Aggregate-Materialize]) + --> Aggregate-Aggregate-ToOutputSchema + <-- Aggregate-Prepare + Processor: Aggregate-Aggregate-ToOutputSchema (stores: []) + --> Aggregate-Project + <-- KSTREAM-AGGREGATE-0000000003 + Processor: Aggregate-Project (stores: []) + --> KTABLE-TOSTREAM-0000000006 + <-- Aggregate-Aggregate-ToOutputSchema + Processor: KTABLE-TOSTREAM-0000000006 (stores: []) + --> KSTREAM-SINK-0000000007 + <-- Aggregate-Project + Sink: KSTREAM-SINK-0000000007 (topic: S2) + <-- KTABLE-TOSTREAM-0000000006 + diff --git a/ksqldb-functional-tests/src/test/resources/historical_plans/min-group-by_-_min_string_group_by/7.3.0_1651157918955/plan.json b/ksqldb-functional-tests/src/test/resources/historical_plans/min-group-by_-_min_string_group_by/7.3.0_1651157918955/plan.json new file mode 100644 index 000000000000..09f536c26386 --- /dev/null +++ b/ksqldb-functional-tests/src/test/resources/historical_plans/min-group-by_-_min_string_group_by/7.3.0_1651157918955/plan.json @@ -0,0 +1,255 @@ +{ + "plan" : [ { + "@type" : "ksqlPlanV1", + "statementText" : "CREATE STREAM TEST (ID BIGINT KEY, VALUE STRING) WITH (KAFKA_TOPIC='test_topic', KEY_FORMAT='KAFKA', VALUE_FORMAT='AVRO');", + "ddlCommand" : { + "@type" : "createStreamV1", + "sourceName" : "TEST", + "schema" : "`ID` BIGINT KEY, `VALUE` STRING", + "timestampColumn" : null, + "topicName" : "test_topic", + "formats" : { + "keyFormat" : { + "format" : "KAFKA", + "properties" : { } + }, + "valueFormat" : { + "format" : "AVRO", + "properties" : { } + } + }, + "windowInfo" : null, + "orReplace" : false, + "isSource" : false + }, + "queryPlan" : null + }, { + "@type" : "ksqlPlanV1", + "statementText" : "CREATE TABLE S2 AS SELECT\n TEST.ID ID,\n MIN(TEST.VALUE) VALUE\nFROM TEST TEST\nGROUP BY TEST.ID\nEMIT CHANGES", + "ddlCommand" : { + "@type" : "createTableV1", + "sourceName" : "S2", + "schema" : "`ID` BIGINT KEY, `VALUE` STRING", + "timestampColumn" : null, + "topicName" : "S2", + "formats" : { + "keyFormat" : { + "format" : "KAFKA", + "properties" : { } + }, + "valueFormat" : { + "format" : "AVRO", + "properties" : { } + } + }, + "windowInfo" : null, + "orReplace" : false, + "isSource" : false + }, + "queryPlan" : { + "sources" : [ "TEST" ], + "sink" : "S2", + "physicalPlan" : { + "@type" : "tableSinkV1", + "properties" : { + "queryContext" : "S2" + }, + "source" : { + "@type" : "tableSelectV1", + "properties" : { + "queryContext" : "Aggregate/Project" + }, + "source" : { + "@type" : "streamAggregateV1", + "properties" : { + "queryContext" : "Aggregate/Aggregate" + }, + "source" : { + "@type" : "streamGroupByKeyV1", + "properties" : { + "queryContext" : "Aggregate/GroupBy" + }, + "source" : { + "@type" : "streamSelectV1", + "properties" : { + "queryContext" : "Aggregate/Prepare" + }, + "source" : { + "@type" : "streamSourceV1", + "properties" : { + "queryContext" : "KsqlTopic/Source" + }, + "topicName" : "test_topic", + "formats" : { + "keyFormat" : { + "format" : "KAFKA", + "properties" : { } + }, + "valueFormat" : { + "format" : "AVRO", + "properties" : { } + } + }, + "timestampColumn" : null, + "sourceSchema" : "`ID` BIGINT KEY, `VALUE` STRING", + "pseudoColumnVersion" : 1 + }, + "keyColumnNames" : [ "ID" ], + "selectedKeys" : null, + "selectExpressions" : [ "ID AS ID", "VALUE AS VALUE" ] + }, + "internalFormats" : { + "keyFormat" : { + "format" : "KAFKA", + "properties" : { } + }, + "valueFormat" : { + "format" : "AVRO", + "properties" : { } + } + } + }, + "internalFormats" : { + "keyFormat" : { + "format" : "KAFKA", + "properties" : { } + }, + "valueFormat" : { + "format" : "AVRO", + "properties" : { } + } + }, + "nonAggregateColumns" : [ "ID", "VALUE" ], + "aggregationFunctions" : [ "MIN(VALUE)" ] + }, + "keyColumnNames" : [ "ID" ], + "selectExpressions" : [ "KSQL_AGG_VARIABLE_0 AS VALUE" ], + "internalFormats" : { + "keyFormat" : { + "format" : "KAFKA", + "properties" : { } + }, + "valueFormat" : { + "format" : "AVRO", + "properties" : { } + } + } + }, + "formats" : { + "keyFormat" : { + "format" : "KAFKA", + "properties" : { } + }, + "valueFormat" : { + "format" : "AVRO", + "properties" : { } + } + }, + "topicName" : "S2", + "timestampColumn" : null + }, + "queryId" : "CTAS_S2_0", + "runtimeId" : null + } + } ], + "configs" : { + "ksql.extension.dir" : "ext", + "ksql.streams.cache.max.bytes.buffering" : "0", + "metric.reporters" : "", + "ksql.query.status.running.threshold.seconds" : "300", + "ksql.connect.basic.auth.credentials.reload" : "false", + "ksql.output.topic.name.prefix" : "", + "ksql.query.pull.stream.enabled" : "true", + "ksql.query.push.v2.interpreter.enabled" : "true", + "ksql.queryanonymizer.logs_enabled" : "true", + "ksql.variable.substitution.enable" : "true", + "ksql.streams.shutdown.timeout.ms" : "300000", + "ksql.query.pull.max.allowed.offset.lag" : "9223372036854775807", + "ksql.query.pull.max.qps" : "2147483647", + "ksql.access.validator.enable" : "auto", + "ksql.streams.bootstrap.servers" : "localhost:0", + "ksql.query.pull.metrics.enabled" : "true", + "ksql.metrics.extension" : null, + "ksql.query.push.v2.alos.enabled" : "true", + "ksql.query.push.v2.max.hourly.bandwidth.megabytes" : "2147483647", + "ksql.query.pull.range.scan.enabled" : "true", + "ksql.transient.query.cleanup.service.initial.delay.seconds" : "600", + "ksql.hidden.topics" : "_confluent.*,__confluent.*,_schemas,__consumer_offsets,__transaction_state,connect-configs,connect-offsets,connect-status,connect-statuses", + "ksql.lambdas.enabled" : "true", + "ksql.source.table.materialization.enabled" : "true", + "ksql.query.pull.max.hourly.bandwidth.megabytes" : "2147483647", + "ksql.sink.window.change.log.additional.retention" : "1000000", + "ksql.query.persistent.active.limit" : "2147483647", + "ksql.persistence.wrap.single.values" : null, + "ksql.query.transient.max.bytes.buffering.total" : "-1", + "ksql.connect.basic.auth.credentials.source" : "NONE", + "ksql.schema.registry.url" : "schema_registry.url:0", + "ksql.properties.overrides.denylist" : "", + "ksql.service.id" : "some.ksql.service.id", + "ksql.query.push.v2.max.catchup.consumers" : "5", + "ksql.query.push.v2.enabled" : "false", + "ksql.transient.query.cleanup.service.enable" : "true", + "ksql.query.push.v2.metrics.enabled" : "true", + "ksql.rowpartition.rowoffset.enabled" : "true", + "ksql.streams.commit.interval.ms" : "2000", + "ksql.query.pull.table.scan.enabled" : "true", + "ksql.streams.auto.commit.interval.ms" : "0", + "ksql.streams.topology.optimization" : "all", + "ksql.endpoint.migrate.query" : "true", + "ksql.query.push.v2.registry.installed" : "false", + "ksql.streams.num.stream.threads" : "4", + "ksql.metrics.tags.custom" : "", + "ksql.query.push.v2.catchup.consumer.msg.window" : "50", + "ksql.runtime.feature.shared.enabled" : "false", + "ksql.udf.collect.metrics" : "false", + "ksql.new.query.planner.enabled" : "false", + "ksql.connect.request.headers.plugin" : null, + "ksql.security.extension.class" : null, + "ksql.transient.prefix" : "transient_", + "ksql.headers.columns.enabled" : "true", + "ksql.streams.default.deserialization.exception.handler" : "io.confluent.ksql.errors.LogMetricAndContinueExceptionHandler", + "ksql.connect.request.timeout.ms" : "5000", + "ksql.query.pull.enable.standby.reads" : "false", + "ksql.persistence.default.format.key" : "KAFKA", + "ksql.query.persistent.max.bytes.buffering.total" : "-1", + "ksql.query.error.max.queue.size" : "10", + "ksql.query.cleanup.shutdown.timeout.ms" : "30000", + "ksql.internal.topic.min.insync.replicas" : "1", + "ksql.internal.topic.replicas" : "1", + "ksql.insert.into.values.enabled" : "true", + "ksql.queryanonymizer.cluster_namespace" : null, + "ksql.create.or.replace.enabled" : "true", + "ksql.shared.runtimes.count" : "2", + "ksql.cast.strings.preserve.nulls" : "true", + "ksql.authorization.cache.max.entries" : "10000", + "ksql.pull.queries.enable" : "true", + "ksql.transient.query.cleanup.service.period.seconds" : "600", + "ksql.suppress.enabled" : "false", + "ksql.readonly.topics" : "_confluent.*,__confluent.*,_schemas,__consumer_offsets,__transaction_state,connect-configs,connect-offsets,connect-status,connect-statuses", + "ksql.connect.basic.auth.credentials.file" : "", + "ksql.authorization.cache.expiry.time.secs" : "30", + "ksql.query.retry.backoff.initial.ms" : "15000", + "ksql.query.pull.max.concurrent.requests" : "2147483647", + "ksql.streams.auto.offset.reset" : "earliest", + "ksql.connect.url" : "http://localhost:8083", + "ksql.query.push.v2.new.latest.delay.ms" : "5000", + "ksql.query.push.v2.latest.reset.age.ms" : "30000", + "ksql.streams.default.production.exception.handler" : "io.confluent.ksql.errors.ProductionExceptionHandlerUtil$LogAndFailProductionExceptionHandler", + "ksql.query.pull.interpreter.enabled" : "true", + "ksql.query.pull.limit.clause.enabled" : "true", + "ksql.query.pull.router.thread.pool.size" : "50", + "ksql.query.push.v2.continuation.tokens.enabled" : "false", + "ksql.query.retry.backoff.max.ms" : "900000", + "ksql.timestamp.throw.on.invalid" : "false", + "ksql.persistence.default.format.value" : null, + "ksql.udfs.enabled" : "true", + "ksql.udf.enable.security.manager" : "true", + "ksql.connect.worker.config" : "", + "ksql.nested.error.set.null" : "true", + "ksql.query.pull.thread.pool.size" : "50", + "ksql.persistent.prefix" : "query_", + "ksql.metastore.backup.location" : "", + "ksql.error.classifier.regex" : "", + "ksql.suppress.buffer.size.bytes" : "-1" + } +} \ No newline at end of file diff --git a/ksqldb-functional-tests/src/test/resources/historical_plans/min-group-by_-_min_string_group_by/7.3.0_1651157918955/spec.json b/ksqldb-functional-tests/src/test/resources/historical_plans/min-group-by_-_min_string_group_by/7.3.0_1651157918955/spec.json new file mode 100644 index 000000000000..61c469d7960a --- /dev/null +++ b/ksqldb-functional-tests/src/test/resources/historical_plans/min-group-by_-_min_string_group_by/7.3.0_1651157918955/spec.json @@ -0,0 +1,292 @@ +{ + "version" : "7.3.0", + "timestamp" : 1651157918955, + "path" : "query-validation-tests/min-group-by.json", + "schemas" : { + "CTAS_S2_0.Aggregate.Aggregate.Materialize" : { + "schema" : "`ID` BIGINT KEY, `ID` BIGINT, `VALUE` STRING, `KSQL_AGG_VARIABLE_0` STRING", + "keyFormat" : { + "format" : "KAFKA" + }, + "valueFormat" : { + "format" : "AVRO" + } + }, + "CTAS_S2_0.Aggregate.Project" : { + "schema" : "`ID` BIGINT KEY, `VALUE` STRING", + "keyFormat" : { + "format" : "KAFKA" + }, + "valueFormat" : { + "format" : "AVRO" + } + }, + "CTAS_S2_0.KsqlTopic.Source" : { + "schema" : "`ID` BIGINT KEY, `VALUE` STRING", + "keyFormat" : { + "format" : "KAFKA" + }, + "valueFormat" : { + "format" : "AVRO" + } + }, + "CTAS_S2_0.S2" : { + "schema" : "`ID` BIGINT KEY, `VALUE` STRING", + "keyFormat" : { + "format" : "KAFKA" + }, + "valueFormat" : { + "format" : "AVRO" + } + }, + "CTAS_S2_0.Aggregate.GroupBy" : { + "schema" : "`ID` BIGINT KEY, `ID` BIGINT, `VALUE` STRING", + "keyFormat" : { + "format" : "KAFKA" + }, + "valueFormat" : { + "format" : "AVRO" + } + } + }, + "testCase" : { + "name" : "min string group by", + "inputs" : [ { + "topic" : "test_topic", + "key" : 1, + "value" : { + "value" : null + } + }, { + "topic" : "test_topic", + "key" : 0, + "value" : { + "value" : null + } + }, { + "topic" : "test_topic", + "key" : 0, + "value" : { + "value" : "B" + } + }, { + "topic" : "test_topic", + "key" : 0, + "value" : { + "value" : "A" + } + }, { + "topic" : "test_topic", + "key" : 0, + "value" : { + "value" : null + } + }, { + "topic" : "test_topic", + "key" : 100, + "value" : { + "value" : "E" + } + }, { + "topic" : "test_topic", + "key" : 100, + "value" : { + "value" : "C" + } + }, { + "topic" : "test_topic", + "key" : 100, + "value" : { + "value" : "G" + } + }, { + "topic" : "test_topic", + "key" : 0, + "value" : { + "value" : "F" + } + }, { + "topic" : "test_topic", + "key" : 0, + "value" : { + "value" : "D" + } + } ], + "outputs" : [ { + "topic" : "S2", + "key" : 1, + "value" : { + "VALUE" : null + } + }, { + "topic" : "S2", + "key" : 0, + "value" : { + "VALUE" : null + } + }, { + "topic" : "S2", + "key" : 0, + "value" : { + "VALUE" : "B" + } + }, { + "topic" : "S2", + "key" : 0, + "value" : { + "VALUE" : "A" + } + }, { + "topic" : "S2", + "key" : 0, + "value" : { + "VALUE" : "A" + } + }, { + "topic" : "S2", + "key" : 100, + "value" : { + "VALUE" : "E" + } + }, { + "topic" : "S2", + "key" : 100, + "value" : { + "VALUE" : "C" + } + }, { + "topic" : "S2", + "key" : 100, + "value" : { + "VALUE" : "C" + } + }, { + "topic" : "S2", + "key" : 0, + "value" : { + "VALUE" : "A" + } + }, { + "topic" : "S2", + "key" : 0, + "value" : { + "VALUE" : "A" + } + } ], + "topics" : [ { + "name" : "test_topic", + "valueSchema" : { + "type" : "record", + "name" : "KsqlDataSourceSchema", + "namespace" : "io.confluent.ksql.avro_schemas", + "fields" : [ { + "name" : "VALUE", + "type" : [ "null", "string" ], + "default" : null + } ], + "connect.name" : "io.confluent.ksql.avro_schemas.KsqlDataSourceSchema" + }, + "valueFormat" : "AVRO", + "replicas" : 1, + "numPartitions" : 4 + }, { + "name" : "S2", + "replicas" : 1, + "numPartitions" : 4 + } ], + "statements" : [ "CREATE STREAM TEST (ID BIGINT KEY, VALUE STRING) WITH (kafka_topic='test_topic',value_format='AVRO');", "CREATE TABLE S2 as SELECT ID, min(value) as value FROM test group by id;" ], + "post" : { + "sources" : [ { + "name" : "S2", + "type" : "TABLE", + "schema" : "`ID` BIGINT KEY, `VALUE` STRING", + "keyFormat" : { + "format" : "KAFKA" + }, + "valueFormat" : "AVRO", + "keyFeatures" : [ ], + "valueFeatures" : [ ], + "isSource" : false + }, { + "name" : "TEST", + "type" : "STREAM", + "schema" : "`ID` BIGINT KEY, `VALUE` STRING", + "keyFormat" : { + "format" : "KAFKA" + }, + "valueFormat" : "AVRO", + "keyFeatures" : [ ], + "valueFeatures" : [ ], + "isSource" : false + } ], + "topics" : { + "topics" : [ { + "name" : "_confluent-ksql-some.ksql.service.idquery_CTAS_S2_0-Aggregate-Aggregate-Materialize-changelog", + "keyFormat" : { + "format" : "KAFKA" + }, + "valueFormat" : { + "format" : "AVRO" + }, + "valueSchema" : { + "type" : "record", + "name" : "KsqlDataSourceSchema", + "namespace" : "io.confluent.ksql.avro_schemas", + "fields" : [ { + "name" : "ID", + "type" : [ "null", "long" ], + "default" : null + }, { + "name" : "VALUE", + "type" : [ "null", "string" ], + "default" : null + }, { + "name" : "KSQL_AGG_VARIABLE_0", + "type" : [ "null", "string" ], + "default" : null + } ] + } + }, { + "name" : "test_topic", + "keyFormat" : { + "format" : "KAFKA" + }, + "valueFormat" : { + "format" : "AVRO" + }, + "partitions" : 4, + "valueSchema" : { + "type" : "record", + "name" : "KsqlDataSourceSchema", + "namespace" : "io.confluent.ksql.avro_schemas", + "fields" : [ { + "name" : "VALUE", + "type" : [ "null", "string" ], + "default" : null + } ], + "connect.name" : "io.confluent.ksql.avro_schemas.KsqlDataSourceSchema" + } + }, { + "name" : "S2", + "keyFormat" : { + "format" : "KAFKA" + }, + "valueFormat" : { + "format" : "AVRO" + }, + "partitions" : 4, + "valueSchema" : { + "type" : "record", + "name" : "KsqlDataSourceSchema", + "namespace" : "io.confluent.ksql.avro_schemas", + "fields" : [ { + "name" : "VALUE", + "type" : [ "null", "string" ], + "default" : null + } ] + } + } ] + } + } + } +} \ No newline at end of file diff --git a/ksqldb-functional-tests/src/test/resources/historical_plans/min-group-by_-_min_string_group_by/7.3.0_1651157918955/topology b/ksqldb-functional-tests/src/test/resources/historical_plans/min-group-by_-_min_string_group_by/7.3.0_1651157918955/topology new file mode 100644 index 000000000000..fcfc8fb28665 --- /dev/null +++ b/ksqldb-functional-tests/src/test/resources/historical_plans/min-group-by_-_min_string_group_by/7.3.0_1651157918955/topology @@ -0,0 +1,25 @@ +Topologies: + Sub-topology: 0 + Source: KSTREAM-SOURCE-0000000000 (topics: [test_topic]) + --> KSTREAM-TRANSFORMVALUES-0000000001 + Processor: KSTREAM-TRANSFORMVALUES-0000000001 (stores: []) + --> Aggregate-Prepare + <-- KSTREAM-SOURCE-0000000000 + Processor: Aggregate-Prepare (stores: []) + --> KSTREAM-AGGREGATE-0000000003 + <-- KSTREAM-TRANSFORMVALUES-0000000001 + Processor: KSTREAM-AGGREGATE-0000000003 (stores: [Aggregate-Aggregate-Materialize]) + --> Aggregate-Aggregate-ToOutputSchema + <-- Aggregate-Prepare + Processor: Aggregate-Aggregate-ToOutputSchema (stores: []) + --> Aggregate-Project + <-- KSTREAM-AGGREGATE-0000000003 + Processor: Aggregate-Project (stores: []) + --> KTABLE-TOSTREAM-0000000006 + <-- Aggregate-Aggregate-ToOutputSchema + Processor: KTABLE-TOSTREAM-0000000006 (stores: []) + --> KSTREAM-SINK-0000000007 + <-- Aggregate-Project + Sink: KSTREAM-SINK-0000000007 (topic: S2) + <-- KTABLE-TOSTREAM-0000000006 + diff --git a/ksqldb-functional-tests/src/test/resources/historical_plans/sum_-_sum_decimal_should_not_overflow_with_cast_to_higher_precision/7.3.0_1651526575310/plan.json b/ksqldb-functional-tests/src/test/resources/historical_plans/sum_-_sum_decimal_should_not_overflow_with_cast_to_higher_precision/7.3.0_1651526575310/plan.json new file mode 100644 index 000000000000..05d98c28644c --- /dev/null +++ b/ksqldb-functional-tests/src/test/resources/historical_plans/sum_-_sum_decimal_should_not_overflow_with_cast_to_higher_precision/7.3.0_1651526575310/plan.json @@ -0,0 +1,256 @@ +{ + "plan" : [ { + "@type" : "ksqlPlanV1", + "statementText" : "CREATE STREAM TEST (ID BIGINT KEY, VALUE DECIMAL(4, 1)) WITH (KAFKA_TOPIC='test_topic', KEY_FORMAT='KAFKA', VALUE_FORMAT='AVRO');", + "ddlCommand" : { + "@type" : "createStreamV1", + "sourceName" : "TEST", + "schema" : "`ID` BIGINT KEY, `VALUE` DECIMAL(4, 1)", + "timestampColumn" : null, + "topicName" : "test_topic", + "formats" : { + "keyFormat" : { + "format" : "KAFKA", + "properties" : { } + }, + "valueFormat" : { + "format" : "AVRO", + "properties" : { } + } + }, + "windowInfo" : null, + "orReplace" : false, + "isSource" : false + }, + "queryPlan" : null + }, { + "@type" : "ksqlPlanV1", + "statementText" : "CREATE TABLE S2 AS SELECT\n TEST.ID ID,\n SUM(CAST(TEST.VALUE AS DECIMAL(6, 2))) SUM\nFROM TEST TEST\nGROUP BY TEST.ID\nEMIT CHANGES", + "ddlCommand" : { + "@type" : "createTableV1", + "sourceName" : "S2", + "schema" : "`ID` BIGINT KEY, `SUM` DECIMAL(6, 2)", + "timestampColumn" : null, + "topicName" : "S2", + "formats" : { + "keyFormat" : { + "format" : "KAFKA", + "properties" : { } + }, + "valueFormat" : { + "format" : "AVRO", + "properties" : { } + } + }, + "windowInfo" : null, + "orReplace" : false, + "isSource" : false + }, + "queryPlan" : { + "sources" : [ "TEST" ], + "sink" : "S2", + "physicalPlan" : { + "@type" : "tableSinkV1", + "properties" : { + "queryContext" : "S2" + }, + "source" : { + "@type" : "tableSelectV1", + "properties" : { + "queryContext" : "Aggregate/Project" + }, + "source" : { + "@type" : "streamAggregateV1", + "properties" : { + "queryContext" : "Aggregate/Aggregate" + }, + "source" : { + "@type" : "streamGroupByKeyV1", + "properties" : { + "queryContext" : "Aggregate/GroupBy" + }, + "source" : { + "@type" : "streamSelectV1", + "properties" : { + "queryContext" : "Aggregate/Prepare" + }, + "source" : { + "@type" : "streamSourceV1", + "properties" : { + "queryContext" : "KsqlTopic/Source" + }, + "topicName" : "test_topic", + "formats" : { + "keyFormat" : { + "format" : "KAFKA", + "properties" : { } + }, + "valueFormat" : { + "format" : "AVRO", + "properties" : { } + } + }, + "timestampColumn" : null, + "sourceSchema" : "`ID` BIGINT KEY, `VALUE` DECIMAL(4, 1)", + "pseudoColumnVersion" : 1 + }, + "keyColumnNames" : [ "ID" ], + "selectedKeys" : null, + "selectExpressions" : [ "ID AS ID", "VALUE AS VALUE", "CAST(VALUE AS DECIMAL(6, 2)) AS KSQL_INTERNAL_COL_2" ] + }, + "internalFormats" : { + "keyFormat" : { + "format" : "KAFKA", + "properties" : { } + }, + "valueFormat" : { + "format" : "AVRO", + "properties" : { } + } + } + }, + "internalFormats" : { + "keyFormat" : { + "format" : "KAFKA", + "properties" : { } + }, + "valueFormat" : { + "format" : "AVRO", + "properties" : { } + } + }, + "nonAggregateColumns" : [ "ID", "VALUE" ], + "aggregationFunctions" : [ "SUM(KSQL_INTERNAL_COL_2)" ] + }, + "keyColumnNames" : [ "ID" ], + "selectExpressions" : [ "KSQL_AGG_VARIABLE_0 AS SUM" ], + "internalFormats" : { + "keyFormat" : { + "format" : "KAFKA", + "properties" : { } + }, + "valueFormat" : { + "format" : "AVRO", + "properties" : { } + } + } + }, + "formats" : { + "keyFormat" : { + "format" : "KAFKA", + "properties" : { } + }, + "valueFormat" : { + "format" : "AVRO", + "properties" : { } + } + }, + "topicName" : "S2", + "timestampColumn" : null + }, + "queryId" : "CTAS_S2_0", + "runtimeId" : null + } + } ], + "configs" : { + "ksql.extension.dir" : "ext", + "ksql.streams.cache.max.bytes.buffering" : "0", + "metric.reporters" : "", + "ksql.query.status.running.threshold.seconds" : "300", + "ksql.connect.basic.auth.credentials.reload" : "false", + "ksql.output.topic.name.prefix" : "", + "ksql.query.pull.stream.enabled" : "true", + "ksql.query.push.v2.interpreter.enabled" : "true", + "ksql.queryanonymizer.logs_enabled" : "true", + "ksql.variable.substitution.enable" : "true", + "ksql.streams.shutdown.timeout.ms" : "300000", + "ksql.query.pull.max.allowed.offset.lag" : "9223372036854775807", + "ksql.query.pull.max.qps" : "2147483647", + "ksql.access.validator.enable" : "auto", + "ksql.streams.bootstrap.servers" : "localhost:0", + "ksql.query.pull.metrics.enabled" : "true", + "ksql.metrics.extension" : null, + "ksql.query.push.v2.alos.enabled" : "true", + "ksql.query.push.v2.max.hourly.bandwidth.megabytes" : "2147483647", + "ksql.query.pull.range.scan.enabled" : "true", + "ksql.transient.query.cleanup.service.initial.delay.seconds" : "600", + "ksql.hidden.topics" : "_confluent.*,__confluent.*,_schemas,__consumer_offsets,__transaction_state,connect-configs,connect-offsets,connect-status,connect-statuses", + "ksql.lambdas.enabled" : "true", + "ksql.source.table.materialization.enabled" : "true", + "ksql.query.pull.max.hourly.bandwidth.megabytes" : "2147483647", + "ksql.sink.window.change.log.additional.retention" : "1000000", + "ksql.query.persistent.active.limit" : "2147483647", + "ksql.persistence.wrap.single.values" : null, + "ksql.query.transient.max.bytes.buffering.total" : "-1", + "ksql.connect.basic.auth.credentials.source" : "NONE", + "ksql.schema.registry.url" : "schema_registry.url:0", + "ksql.properties.overrides.denylist" : "", + "ksql.service.id" : "some.ksql.service.id", + "ksql.query.push.v2.max.catchup.consumers" : "5", + "ksql.assert.topic.default.timeout.ms" : "1000", + "ksql.query.push.v2.enabled" : "false", + "ksql.transient.query.cleanup.service.enable" : "true", + "ksql.query.push.v2.metrics.enabled" : "true", + "ksql.rowpartition.rowoffset.enabled" : "true", + "ksql.streams.commit.interval.ms" : "2000", + "ksql.query.pull.table.scan.enabled" : "true", + "ksql.streams.auto.commit.interval.ms" : "0", + "ksql.streams.topology.optimization" : "all", + "ksql.endpoint.migrate.query" : "true", + "ksql.query.push.v2.registry.installed" : "false", + "ksql.streams.num.stream.threads" : "4", + "ksql.metrics.tags.custom" : "", + "ksql.query.push.v2.catchup.consumer.msg.window" : "50", + "ksql.runtime.feature.shared.enabled" : "false", + "ksql.udf.collect.metrics" : "false", + "ksql.new.query.planner.enabled" : "false", + "ksql.connect.request.headers.plugin" : null, + "ksql.security.extension.class" : null, + "ksql.transient.prefix" : "transient_", + "ksql.headers.columns.enabled" : "true", + "ksql.streams.default.deserialization.exception.handler" : "io.confluent.ksql.errors.LogMetricAndContinueExceptionHandler", + "ksql.connect.request.timeout.ms" : "5000", + "ksql.query.pull.enable.standby.reads" : "false", + "ksql.persistence.default.format.key" : "KAFKA", + "ksql.query.persistent.max.bytes.buffering.total" : "-1", + "ksql.query.error.max.queue.size" : "10", + "ksql.query.cleanup.shutdown.timeout.ms" : "30000", + "ksql.internal.topic.min.insync.replicas" : "1", + "ksql.internal.topic.replicas" : "1", + "ksql.insert.into.values.enabled" : "true", + "ksql.queryanonymizer.cluster_namespace" : null, + "ksql.create.or.replace.enabled" : "true", + "ksql.shared.runtimes.count" : "2", + "ksql.cast.strings.preserve.nulls" : "true", + "ksql.authorization.cache.max.entries" : "10000", + "ksql.pull.queries.enable" : "true", + "ksql.transient.query.cleanup.service.period.seconds" : "600", + "ksql.suppress.enabled" : "false", + "ksql.readonly.topics" : "_confluent.*,__confluent.*,_schemas,__consumer_offsets,__transaction_state,connect-configs,connect-offsets,connect-status,connect-statuses", + "ksql.connect.basic.auth.credentials.file" : "", + "ksql.authorization.cache.expiry.time.secs" : "30", + "ksql.query.retry.backoff.initial.ms" : "15000", + "ksql.query.pull.max.concurrent.requests" : "2147483647", + "ksql.streams.auto.offset.reset" : "earliest", + "ksql.connect.url" : "http://localhost:8083", + "ksql.query.push.v2.new.latest.delay.ms" : "5000", + "ksql.query.push.v2.latest.reset.age.ms" : "30000", + "ksql.streams.default.production.exception.handler" : "io.confluent.ksql.errors.ProductionExceptionHandlerUtil$LogAndFailProductionExceptionHandler", + "ksql.query.pull.interpreter.enabled" : "true", + "ksql.query.pull.limit.clause.enabled" : "true", + "ksql.query.pull.router.thread.pool.size" : "50", + "ksql.query.push.v2.continuation.tokens.enabled" : "false", + "ksql.query.retry.backoff.max.ms" : "900000", + "ksql.timestamp.throw.on.invalid" : "false", + "ksql.persistence.default.format.value" : null, + "ksql.udfs.enabled" : "true", + "ksql.udf.enable.security.manager" : "true", + "ksql.connect.worker.config" : "", + "ksql.nested.error.set.null" : "true", + "ksql.query.pull.thread.pool.size" : "50", + "ksql.persistent.prefix" : "query_", + "ksql.metastore.backup.location" : "", + "ksql.error.classifier.regex" : "", + "ksql.suppress.buffer.size.bytes" : "-1" + } +} \ No newline at end of file diff --git a/ksqldb-functional-tests/src/test/resources/historical_plans/sum_-_sum_decimal_should_not_overflow_with_cast_to_higher_precision/7.3.0_1651526575310/spec.json b/ksqldb-functional-tests/src/test/resources/historical_plans/sum_-_sum_decimal_should_not_overflow_with_cast_to_higher_precision/7.3.0_1651526575310/spec.json new file mode 100644 index 000000000000..480312c82d95 --- /dev/null +++ b/ksqldb-functional-tests/src/test/resources/historical_plans/sum_-_sum_decimal_should_not_overflow_with_cast_to_higher_precision/7.3.0_1651526575310/spec.json @@ -0,0 +1,269 @@ +{ + "version" : "7.3.0", + "timestamp" : 1651526575310, + "path" : "query-validation-tests/sum.json", + "schemas" : { + "CTAS_S2_0.Aggregate.Aggregate.Materialize" : { + "schema" : "`ID` BIGINT KEY, `ID` BIGINT, `VALUE` DECIMAL(4, 1), `KSQL_AGG_VARIABLE_0` DECIMAL(6, 2)", + "keyFormat" : { + "format" : "KAFKA" + }, + "valueFormat" : { + "format" : "AVRO" + } + }, + "CTAS_S2_0.Aggregate.Project" : { + "schema" : "`ID` BIGINT KEY, `SUM` DECIMAL(6, 2)", + "keyFormat" : { + "format" : "KAFKA" + }, + "valueFormat" : { + "format" : "AVRO" + } + }, + "CTAS_S2_0.KsqlTopic.Source" : { + "schema" : "`ID` BIGINT KEY, `VALUE` DECIMAL(4, 1)", + "keyFormat" : { + "format" : "KAFKA" + }, + "valueFormat" : { + "format" : "AVRO" + } + }, + "CTAS_S2_0.S2" : { + "schema" : "`ID` BIGINT KEY, `SUM` DECIMAL(6, 2)", + "keyFormat" : { + "format" : "KAFKA" + }, + "valueFormat" : { + "format" : "AVRO" + } + }, + "CTAS_S2_0.Aggregate.GroupBy" : { + "schema" : "`ID` BIGINT KEY, `ID` BIGINT, `VALUE` DECIMAL(4, 1), `KSQL_INTERNAL_COL_2` DECIMAL(6, 2)", + "keyFormat" : { + "format" : "KAFKA" + }, + "valueFormat" : { + "format" : "AVRO" + } + } + }, + "testCase" : { + "name" : "sum decimal should not overflow with cast to higher precision", + "inputs" : [ { + "topic" : "test_topic", + "key" : 0, + "value" : { + "value" : "5.4" + } + }, { + "topic" : "test_topic", + "key" : 0, + "value" : { + "value" : "100.1" + } + }, { + "topic" : "test_topic", + "key" : 100, + "value" : { + "value" : "500.9" + } + }, { + "topic" : "test_topic", + "key" : 100, + "value" : { + "value" : "300.8" + } + }, { + "topic" : "test_topic", + "key" : 100, + "value" : { + "value" : "400.8" + } + } ], + "outputs" : [ { + "topic" : "S2", + "key" : 0, + "value" : { + "SUM" : "5.40" + } + }, { + "topic" : "S2", + "key" : 0, + "value" : { + "SUM" : "105.50" + } + }, { + "topic" : "S2", + "key" : 100, + "value" : { + "SUM" : "500.90" + } + }, { + "topic" : "S2", + "key" : 100, + "value" : { + "SUM" : "801.70" + } + }, { + "topic" : "S2", + "key" : 100, + "value" : { + "SUM" : "1202.50" + } + } ], + "topics" : [ { + "name" : "test_topic", + "valueSchema" : { + "type" : "record", + "name" : "KsqlDataSourceSchema", + "namespace" : "io.confluent.ksql.avro_schemas", + "fields" : [ { + "name" : "VALUE", + "type" : [ "null", { + "type" : "bytes", + "scale" : 1, + "precision" : 4, + "connect.version" : 1, + "connect.parameters" : { + "scale" : "1", + "connect.decimal.precision" : "4" + }, + "connect.name" : "org.apache.kafka.connect.data.Decimal", + "logicalType" : "decimal" + } ], + "default" : null + } ], + "connect.name" : "io.confluent.ksql.avro_schemas.KsqlDataSourceSchema" + }, + "valueFormat" : "AVRO", + "replicas" : 1, + "numPartitions" : 4 + }, { + "name" : "S2", + "replicas" : 1, + "numPartitions" : 4 + } ], + "statements" : [ "CREATE STREAM TEST (ID BIGINT KEY, VALUE decimal(4,1)) WITH (kafka_topic='test_topic', value_format='AVRO');", "CREATE TABLE S2 as SELECT ID, sum(CAST(value AS DECIMAL(6,2))) AS SUM FROM test group by id;" ], + "post" : { + "sources" : [ { + "name" : "S2", + "type" : "TABLE", + "schema" : "`ID` BIGINT KEY, `SUM` DECIMAL(6, 2)", + "keyFormat" : { + "format" : "KAFKA" + }, + "valueFormat" : "AVRO", + "keyFeatures" : [ ], + "valueFeatures" : [ ], + "isSource" : false + }, { + "name" : "TEST", + "type" : "STREAM", + "schema" : "`ID` BIGINT KEY, `VALUE` DECIMAL(4, 1)", + "keyFormat" : { + "format" : "KAFKA" + }, + "valueFormat" : "AVRO", + "keyFeatures" : [ ], + "valueFeatures" : [ ], + "isSource" : false + } ], + "topics" : { + "topics" : [ { + "name" : "_confluent-ksql-some.ksql.service.idquery_CTAS_S2_0-Aggregate-Aggregate-Materialize-changelog", + "keyFormat" : { + "format" : "KAFKA" + }, + "valueFormat" : { + "format" : "AVRO" + }, + "valueSchema" : { + "type" : "record", + "name" : "KsqlDataSourceSchema", + "namespace" : "io.confluent.ksql.avro_schemas", + "fields" : [ { + "name" : "ID", + "type" : [ "null", "long" ], + "default" : null + }, { + "name" : "VALUE", + "type" : [ "null", { + "type" : "bytes", + "scale" : 1, + "precision" : 4, + "logicalType" : "decimal" + } ], + "default" : null + }, { + "name" : "KSQL_AGG_VARIABLE_0", + "type" : [ "null", { + "type" : "bytes", + "scale" : 2, + "precision" : 6, + "logicalType" : "decimal" + } ], + "default" : null + } ] + } + }, { + "name" : "test_topic", + "keyFormat" : { + "format" : "KAFKA" + }, + "valueFormat" : { + "format" : "AVRO" + }, + "partitions" : 4, + "valueSchema" : { + "type" : "record", + "name" : "KsqlDataSourceSchema", + "namespace" : "io.confluent.ksql.avro_schemas", + "fields" : [ { + "name" : "VALUE", + "type" : [ "null", { + "type" : "bytes", + "scale" : 1, + "precision" : 4, + "connect.version" : 1, + "connect.parameters" : { + "scale" : "1", + "connect.decimal.precision" : "4" + }, + "connect.name" : "org.apache.kafka.connect.data.Decimal", + "logicalType" : "decimal" + } ], + "default" : null + } ], + "connect.name" : "io.confluent.ksql.avro_schemas.KsqlDataSourceSchema" + } + }, { + "name" : "S2", + "keyFormat" : { + "format" : "KAFKA" + }, + "valueFormat" : { + "format" : "AVRO" + }, + "partitions" : 4, + "valueSchema" : { + "type" : "record", + "name" : "KsqlDataSourceSchema", + "namespace" : "io.confluent.ksql.avro_schemas", + "fields" : [ { + "name" : "SUM", + "type" : [ "null", { + "type" : "bytes", + "scale" : 2, + "precision" : 6, + "logicalType" : "decimal" + } ], + "default" : null + } ] + } + } ] + } + } + } +} \ No newline at end of file diff --git a/ksqldb-functional-tests/src/test/resources/historical_plans/sum_-_sum_decimal_should_not_overflow_with_cast_to_higher_precision/7.3.0_1651526575310/topology b/ksqldb-functional-tests/src/test/resources/historical_plans/sum_-_sum_decimal_should_not_overflow_with_cast_to_higher_precision/7.3.0_1651526575310/topology new file mode 100644 index 000000000000..fcfc8fb28665 --- /dev/null +++ b/ksqldb-functional-tests/src/test/resources/historical_plans/sum_-_sum_decimal_should_not_overflow_with_cast_to_higher_precision/7.3.0_1651526575310/topology @@ -0,0 +1,25 @@ +Topologies: + Sub-topology: 0 + Source: KSTREAM-SOURCE-0000000000 (topics: [test_topic]) + --> KSTREAM-TRANSFORMVALUES-0000000001 + Processor: KSTREAM-TRANSFORMVALUES-0000000001 (stores: []) + --> Aggregate-Prepare + <-- KSTREAM-SOURCE-0000000000 + Processor: Aggregate-Prepare (stores: []) + --> KSTREAM-AGGREGATE-0000000003 + <-- KSTREAM-TRANSFORMVALUES-0000000001 + Processor: KSTREAM-AGGREGATE-0000000003 (stores: [Aggregate-Aggregate-Materialize]) + --> Aggregate-Aggregate-ToOutputSchema + <-- Aggregate-Prepare + Processor: Aggregate-Aggregate-ToOutputSchema (stores: []) + --> Aggregate-Project + <-- KSTREAM-AGGREGATE-0000000003 + Processor: Aggregate-Project (stores: []) + --> KTABLE-TOSTREAM-0000000006 + <-- Aggregate-Aggregate-ToOutputSchema + Processor: KTABLE-TOSTREAM-0000000006 (stores: []) + --> KSTREAM-SINK-0000000007 + <-- Aggregate-Project + Sink: KSTREAM-SINK-0000000007 (topic: S2) + <-- KTABLE-TOSTREAM-0000000006 + diff --git a/ksqldb-functional-tests/src/test/resources/historical_plans/topk-group-by_-_topk_double_-_PROTOBUF_NOSR/7.3.0_1651070251066/plan.json b/ksqldb-functional-tests/src/test/resources/historical_plans/topk-group-by_-_topk_double_-_PROTOBUF_NOSR/7.3.0_1651070251066/plan.json new file mode 100644 index 000000000000..6cadb1c91be1 --- /dev/null +++ b/ksqldb-functional-tests/src/test/resources/historical_plans/topk-group-by_-_topk_double_-_PROTOBUF_NOSR/7.3.0_1651070251066/plan.json @@ -0,0 +1,255 @@ +{ + "plan" : [ { + "@type" : "ksqlPlanV1", + "statementText" : "CREATE STREAM TEST (ID BIGINT KEY, NAME STRING, VALUE DOUBLE) WITH (KAFKA_TOPIC='test_topic', KEY_FORMAT='KAFKA', VALUE_FORMAT='PROTOBUF_NOSR');", + "ddlCommand" : { + "@type" : "createStreamV1", + "sourceName" : "TEST", + "schema" : "`ID` BIGINT KEY, `NAME` STRING, `VALUE` DOUBLE", + "timestampColumn" : null, + "topicName" : "test_topic", + "formats" : { + "keyFormat" : { + "format" : "KAFKA", + "properties" : { } + }, + "valueFormat" : { + "format" : "PROTOBUF_NOSR", + "properties" : { } + } + }, + "windowInfo" : null, + "orReplace" : false, + "isSource" : false + }, + "queryPlan" : null + }, { + "@type" : "ksqlPlanV1", + "statementText" : "CREATE TABLE S2 AS SELECT\n TEST.ID ID,\n TOPK(TEST.VALUE, 3) TOPK\nFROM TEST TEST\nGROUP BY TEST.ID\nEMIT CHANGES", + "ddlCommand" : { + "@type" : "createTableV1", + "sourceName" : "S2", + "schema" : "`ID` BIGINT KEY, `TOPK` ARRAY", + "timestampColumn" : null, + "topicName" : "S2", + "formats" : { + "keyFormat" : { + "format" : "KAFKA", + "properties" : { } + }, + "valueFormat" : { + "format" : "PROTOBUF_NOSR", + "properties" : { } + } + }, + "windowInfo" : null, + "orReplace" : false, + "isSource" : false + }, + "queryPlan" : { + "sources" : [ "TEST" ], + "sink" : "S2", + "physicalPlan" : { + "@type" : "tableSinkV1", + "properties" : { + "queryContext" : "S2" + }, + "source" : { + "@type" : "tableSelectV1", + "properties" : { + "queryContext" : "Aggregate/Project" + }, + "source" : { + "@type" : "streamAggregateV1", + "properties" : { + "queryContext" : "Aggregate/Aggregate" + }, + "source" : { + "@type" : "streamGroupByKeyV1", + "properties" : { + "queryContext" : "Aggregate/GroupBy" + }, + "source" : { + "@type" : "streamSelectV1", + "properties" : { + "queryContext" : "Aggregate/Prepare" + }, + "source" : { + "@type" : "streamSourceV1", + "properties" : { + "queryContext" : "KsqlTopic/Source" + }, + "topicName" : "test_topic", + "formats" : { + "keyFormat" : { + "format" : "KAFKA", + "properties" : { } + }, + "valueFormat" : { + "format" : "PROTOBUF_NOSR", + "properties" : { } + } + }, + "timestampColumn" : null, + "sourceSchema" : "`ID` BIGINT KEY, `NAME` STRING, `VALUE` DOUBLE", + "pseudoColumnVersion" : 1 + }, + "keyColumnNames" : [ "ID" ], + "selectedKeys" : null, + "selectExpressions" : [ "ID AS ID", "VALUE AS VALUE", "3 AS KSQL_INTERNAL_COL_2" ] + }, + "internalFormats" : { + "keyFormat" : { + "format" : "KAFKA", + "properties" : { } + }, + "valueFormat" : { + "format" : "PROTOBUF_NOSR", + "properties" : { } + } + } + }, + "internalFormats" : { + "keyFormat" : { + "format" : "KAFKA", + "properties" : { } + }, + "valueFormat" : { + "format" : "PROTOBUF_NOSR", + "properties" : { } + } + }, + "nonAggregateColumns" : [ "ID", "VALUE" ], + "aggregationFunctions" : [ "TOPK(VALUE, 3)" ] + }, + "keyColumnNames" : [ "ID" ], + "selectExpressions" : [ "KSQL_AGG_VARIABLE_0 AS TOPK" ], + "internalFormats" : { + "keyFormat" : { + "format" : "KAFKA", + "properties" : { } + }, + "valueFormat" : { + "format" : "PROTOBUF_NOSR", + "properties" : { } + } + } + }, + "formats" : { + "keyFormat" : { + "format" : "KAFKA", + "properties" : { } + }, + "valueFormat" : { + "format" : "PROTOBUF_NOSR", + "properties" : { } + } + }, + "topicName" : "S2", + "timestampColumn" : null + }, + "queryId" : "CTAS_S2_0", + "runtimeId" : null + } + } ], + "configs" : { + "ksql.extension.dir" : "ext", + "ksql.streams.cache.max.bytes.buffering" : "0", + "metric.reporters" : "", + "ksql.query.status.running.threshold.seconds" : "300", + "ksql.connect.basic.auth.credentials.reload" : "false", + "ksql.output.topic.name.prefix" : "", + "ksql.query.pull.stream.enabled" : "true", + "ksql.query.push.v2.interpreter.enabled" : "true", + "ksql.queryanonymizer.logs_enabled" : "true", + "ksql.variable.substitution.enable" : "true", + "ksql.streams.shutdown.timeout.ms" : "300000", + "ksql.query.pull.max.allowed.offset.lag" : "9223372036854775807", + "ksql.query.pull.max.qps" : "2147483647", + "ksql.access.validator.enable" : "auto", + "ksql.streams.bootstrap.servers" : "localhost:0", + "ksql.query.pull.metrics.enabled" : "true", + "ksql.metrics.extension" : null, + "ksql.query.push.v2.alos.enabled" : "true", + "ksql.query.push.v2.max.hourly.bandwidth.megabytes" : "2147483647", + "ksql.query.pull.range.scan.enabled" : "true", + "ksql.transient.query.cleanup.service.initial.delay.seconds" : "600", + "ksql.hidden.topics" : "_confluent.*,__confluent.*,_schemas,__consumer_offsets,__transaction_state,connect-configs,connect-offsets,connect-status,connect-statuses", + "ksql.lambdas.enabled" : "true", + "ksql.source.table.materialization.enabled" : "true", + "ksql.query.pull.max.hourly.bandwidth.megabytes" : "2147483647", + "ksql.sink.window.change.log.additional.retention" : "1000000", + "ksql.query.persistent.active.limit" : "2147483647", + "ksql.persistence.wrap.single.values" : null, + "ksql.query.transient.max.bytes.buffering.total" : "-1", + "ksql.connect.basic.auth.credentials.source" : "NONE", + "ksql.schema.registry.url" : "schema_registry.url:0", + "ksql.properties.overrides.denylist" : "", + "ksql.service.id" : "some.ksql.service.id", + "ksql.query.push.v2.max.catchup.consumers" : "5", + "ksql.query.push.v2.enabled" : "false", + "ksql.transient.query.cleanup.service.enable" : "true", + "ksql.query.push.v2.metrics.enabled" : "true", + "ksql.rowpartition.rowoffset.enabled" : "true", + "ksql.streams.commit.interval.ms" : "2000", + "ksql.query.pull.table.scan.enabled" : "true", + "ksql.streams.auto.commit.interval.ms" : "0", + "ksql.streams.topology.optimization" : "all", + "ksql.endpoint.migrate.query" : "true", + "ksql.query.push.v2.registry.installed" : "false", + "ksql.streams.num.stream.threads" : "4", + "ksql.metrics.tags.custom" : "", + "ksql.query.push.v2.catchup.consumer.msg.window" : "50", + "ksql.runtime.feature.shared.enabled" : "false", + "ksql.udf.collect.metrics" : "false", + "ksql.new.query.planner.enabled" : "false", + "ksql.connect.request.headers.plugin" : null, + "ksql.security.extension.class" : null, + "ksql.transient.prefix" : "transient_", + "ksql.headers.columns.enabled" : "true", + "ksql.streams.default.deserialization.exception.handler" : "io.confluent.ksql.errors.LogMetricAndContinueExceptionHandler", + "ksql.connect.request.timeout.ms" : "5000", + "ksql.query.pull.enable.standby.reads" : "false", + "ksql.persistence.default.format.key" : "KAFKA", + "ksql.query.persistent.max.bytes.buffering.total" : "-1", + "ksql.query.error.max.queue.size" : "10", + "ksql.query.cleanup.shutdown.timeout.ms" : "30000", + "ksql.internal.topic.min.insync.replicas" : "1", + "ksql.internal.topic.replicas" : "1", + "ksql.insert.into.values.enabled" : "true", + "ksql.queryanonymizer.cluster_namespace" : null, + "ksql.create.or.replace.enabled" : "true", + "ksql.shared.runtimes.count" : "2", + "ksql.cast.strings.preserve.nulls" : "true", + "ksql.authorization.cache.max.entries" : "10000", + "ksql.pull.queries.enable" : "true", + "ksql.transient.query.cleanup.service.period.seconds" : "600", + "ksql.suppress.enabled" : "false", + "ksql.readonly.topics" : "_confluent.*,__confluent.*,_schemas,__consumer_offsets,__transaction_state,connect-configs,connect-offsets,connect-status,connect-statuses", + "ksql.connect.basic.auth.credentials.file" : "", + "ksql.authorization.cache.expiry.time.secs" : "30", + "ksql.query.retry.backoff.initial.ms" : "15000", + "ksql.query.pull.max.concurrent.requests" : "2147483647", + "ksql.streams.auto.offset.reset" : "earliest", + "ksql.connect.url" : "http://localhost:8083", + "ksql.query.push.v2.new.latest.delay.ms" : "5000", + "ksql.query.push.v2.latest.reset.age.ms" : "30000", + "ksql.streams.default.production.exception.handler" : "io.confluent.ksql.errors.ProductionExceptionHandlerUtil$LogAndFailProductionExceptionHandler", + "ksql.query.pull.interpreter.enabled" : "true", + "ksql.query.pull.limit.clause.enabled" : "true", + "ksql.query.pull.router.thread.pool.size" : "50", + "ksql.query.push.v2.continuation.tokens.enabled" : "false", + "ksql.query.retry.backoff.max.ms" : "900000", + "ksql.timestamp.throw.on.invalid" : "false", + "ksql.persistence.default.format.value" : null, + "ksql.udfs.enabled" : "true", + "ksql.udf.enable.security.manager" : "true", + "ksql.connect.worker.config" : "", + "ksql.nested.error.set.null" : "true", + "ksql.query.pull.thread.pool.size" : "50", + "ksql.persistent.prefix" : "query_", + "ksql.metastore.backup.location" : "", + "ksql.error.classifier.regex" : "", + "ksql.suppress.buffer.size.bytes" : "-1" + } +} \ No newline at end of file diff --git a/ksqldb-functional-tests/src/test/resources/historical_plans/topk-group-by_-_topk_double_-_PROTOBUF_NOSR/7.3.0_1651070251066/spec.json b/ksqldb-functional-tests/src/test/resources/historical_plans/topk-group-by_-_topk_double_-_PROTOBUF_NOSR/7.3.0_1651070251066/spec.json new file mode 100644 index 000000000000..72c608447697 --- /dev/null +++ b/ksqldb-functional-tests/src/test/resources/historical_plans/topk-group-by_-_topk_double_-_PROTOBUF_NOSR/7.3.0_1651070251066/spec.json @@ -0,0 +1,191 @@ +{ + "version" : "7.3.0", + "timestamp" : 1651070251066, + "path" : "query-validation-tests/topk-group-by.json", + "schemas" : { + "CTAS_S2_0.Aggregate.Aggregate.Materialize" : { + "schema" : "`ID` BIGINT KEY, `ID` BIGINT, `VALUE` DOUBLE, `KSQL_AGG_VARIABLE_0` ARRAY", + "keyFormat" : { + "format" : "KAFKA" + }, + "valueFormat" : { + "format" : "PROTOBUF_NOSR" + } + }, + "CTAS_S2_0.Aggregate.Project" : { + "schema" : "`ID` BIGINT KEY, `TOPK` ARRAY", + "keyFormat" : { + "format" : "KAFKA" + }, + "valueFormat" : { + "format" : "PROTOBUF_NOSR" + } + }, + "CTAS_S2_0.KsqlTopic.Source" : { + "schema" : "`ID` BIGINT KEY, `NAME` STRING, `VALUE` DOUBLE", + "keyFormat" : { + "format" : "KAFKA" + }, + "valueFormat" : { + "format" : "PROTOBUF_NOSR" + } + }, + "CTAS_S2_0.S2" : { + "schema" : "`ID` BIGINT KEY, `TOPK` ARRAY", + "keyFormat" : { + "format" : "KAFKA" + }, + "valueFormat" : { + "format" : "PROTOBUF_NOSR" + } + }, + "CTAS_S2_0.Aggregate.GroupBy" : { + "schema" : "`ID` BIGINT KEY, `ID` BIGINT, `VALUE` DOUBLE, `KSQL_INTERNAL_COL_2` INTEGER", + "keyFormat" : { + "format" : "KAFKA" + }, + "valueFormat" : { + "format" : "PROTOBUF_NOSR" + } + } + }, + "testCase" : { + "name" : "topk double - PROTOBUF_NOSR", + "inputs" : [ { + "topic" : "test_topic", + "key" : 0, + "value" : { + "NAME" : "zero", + "key" : 0, + "value" : 2147483648.9 + } + }, { + "topic" : "test_topic", + "key" : 0, + "value" : { + "NAME" : "zero", + "key" : 0, + "value" : 100.5 + } + }, { + "topic" : "test_topic", + "key" : 0, + "value" : { + "NAME" : "zero", + "key" : 0, + "value" : 99.9 + } + }, { + "topic" : "test_topic", + "key" : 0, + "value" : { + "NAME" : "zero", + "key" : 0, + "value" : 7.3 + } + }, { + "topic" : "test_topic", + "key" : 0, + "value" : { + "NAME" : "zero", + "key" : 0, + "value" : 100.5 + } + } ], + "outputs" : [ { + "topic" : "S2", + "key" : 0, + "value" : { + "TOPK" : [ 2147483648.9 ] + } + }, { + "topic" : "S2", + "key" : 0, + "value" : { + "TOPK" : [ 2147483648.9, 100.5 ] + } + }, { + "topic" : "S2", + "key" : 0, + "value" : { + "TOPK" : [ 2147483648.9, 100.5, 99.9 ] + } + }, { + "topic" : "S2", + "key" : 0, + "value" : { + "TOPK" : [ 2147483648.9, 100.5, 99.9 ] + } + }, { + "topic" : "S2", + "key" : 0, + "value" : { + "TOPK" : [ 2147483648.9, 100.5, 100.5 ] + } + } ], + "topics" : [ { + "name" : "test_topic", + "replicas" : 1, + "numPartitions" : 4 + }, { + "name" : "S2", + "replicas" : 1, + "numPartitions" : 4 + } ], + "statements" : [ "CREATE STREAM TEST (ID BIGINT KEY, NAME varchar, VALUE double) WITH (kafka_topic='test_topic', value_format='PROTOBUF_NOSR');", "CREATE TABLE S2 as SELECT ID, topk(value, 3) as topk FROM test group by id;" ], + "post" : { + "sources" : [ { + "name" : "S2", + "type" : "TABLE", + "schema" : "`ID` BIGINT KEY, `TOPK` ARRAY", + "keyFormat" : { + "format" : "KAFKA" + }, + "valueFormat" : "PROTOBUF_NOSR", + "keyFeatures" : [ ], + "valueFeatures" : [ ], + "isSource" : false + }, { + "name" : "TEST", + "type" : "STREAM", + "schema" : "`ID` BIGINT KEY, `NAME` STRING, `VALUE` DOUBLE", + "keyFormat" : { + "format" : "KAFKA" + }, + "valueFormat" : "PROTOBUF_NOSR", + "keyFeatures" : [ ], + "valueFeatures" : [ ], + "isSource" : false + } ], + "topics" : { + "topics" : [ { + "name" : "_confluent-ksql-some.ksql.service.idquery_CTAS_S2_0-Aggregate-Aggregate-Materialize-changelog", + "keyFormat" : { + "format" : "KAFKA" + }, + "valueFormat" : { + "format" : "PROTOBUF_NOSR" + } + }, { + "name" : "test_topic", + "keyFormat" : { + "format" : "KAFKA" + }, + "valueFormat" : { + "format" : "PROTOBUF_NOSR" + }, + "partitions" : 4 + }, { + "name" : "S2", + "keyFormat" : { + "format" : "KAFKA" + }, + "valueFormat" : { + "format" : "PROTOBUF_NOSR" + }, + "partitions" : 4 + } ] + } + } + } +} \ No newline at end of file diff --git a/ksqldb-functional-tests/src/test/resources/historical_plans/topk-group-by_-_topk_double_-_PROTOBUF_NOSR/7.3.0_1651070251066/topology b/ksqldb-functional-tests/src/test/resources/historical_plans/topk-group-by_-_topk_double_-_PROTOBUF_NOSR/7.3.0_1651070251066/topology new file mode 100644 index 000000000000..fcfc8fb28665 --- /dev/null +++ b/ksqldb-functional-tests/src/test/resources/historical_plans/topk-group-by_-_topk_double_-_PROTOBUF_NOSR/7.3.0_1651070251066/topology @@ -0,0 +1,25 @@ +Topologies: + Sub-topology: 0 + Source: KSTREAM-SOURCE-0000000000 (topics: [test_topic]) + --> KSTREAM-TRANSFORMVALUES-0000000001 + Processor: KSTREAM-TRANSFORMVALUES-0000000001 (stores: []) + --> Aggregate-Prepare + <-- KSTREAM-SOURCE-0000000000 + Processor: Aggregate-Prepare (stores: []) + --> KSTREAM-AGGREGATE-0000000003 + <-- KSTREAM-TRANSFORMVALUES-0000000001 + Processor: KSTREAM-AGGREGATE-0000000003 (stores: [Aggregate-Aggregate-Materialize]) + --> Aggregate-Aggregate-ToOutputSchema + <-- Aggregate-Prepare + Processor: Aggregate-Aggregate-ToOutputSchema (stores: []) + --> Aggregate-Project + <-- KSTREAM-AGGREGATE-0000000003 + Processor: Aggregate-Project (stores: []) + --> KTABLE-TOSTREAM-0000000006 + <-- Aggregate-Aggregate-ToOutputSchema + Processor: KTABLE-TOSTREAM-0000000006 (stores: []) + --> KSTREAM-SINK-0000000007 + <-- Aggregate-Project + Sink: KSTREAM-SINK-0000000007 (topic: S2) + <-- KTABLE-TOSTREAM-0000000006 + diff --git a/ksqldb-functional-tests/src/test/resources/historical_plans/topk-group-by_-_topk_integer_-_PROTOBUF_NOSR/7.3.0_1651070244684/plan.json b/ksqldb-functional-tests/src/test/resources/historical_plans/topk-group-by_-_topk_integer_-_PROTOBUF_NOSR/7.3.0_1651070244684/plan.json new file mode 100644 index 000000000000..7e86826302ef --- /dev/null +++ b/ksqldb-functional-tests/src/test/resources/historical_plans/topk-group-by_-_topk_integer_-_PROTOBUF_NOSR/7.3.0_1651070244684/plan.json @@ -0,0 +1,255 @@ +{ + "plan" : [ { + "@type" : "ksqlPlanV1", + "statementText" : "CREATE STREAM TEST (ID BIGINT KEY, NAME STRING, VALUE INTEGER) WITH (KAFKA_TOPIC='test_topic', KEY_FORMAT='KAFKA', VALUE_FORMAT='PROTOBUF_NOSR');", + "ddlCommand" : { + "@type" : "createStreamV1", + "sourceName" : "TEST", + "schema" : "`ID` BIGINT KEY, `NAME` STRING, `VALUE` INTEGER", + "timestampColumn" : null, + "topicName" : "test_topic", + "formats" : { + "keyFormat" : { + "format" : "KAFKA", + "properties" : { } + }, + "valueFormat" : { + "format" : "PROTOBUF_NOSR", + "properties" : { } + } + }, + "windowInfo" : null, + "orReplace" : false, + "isSource" : false + }, + "queryPlan" : null + }, { + "@type" : "ksqlPlanV1", + "statementText" : "CREATE TABLE S2 AS SELECT\n TEST.ID ID,\n TOPK(TEST.VALUE, 3) TOPK\nFROM TEST TEST\nGROUP BY TEST.ID\nEMIT CHANGES", + "ddlCommand" : { + "@type" : "createTableV1", + "sourceName" : "S2", + "schema" : "`ID` BIGINT KEY, `TOPK` ARRAY", + "timestampColumn" : null, + "topicName" : "S2", + "formats" : { + "keyFormat" : { + "format" : "KAFKA", + "properties" : { } + }, + "valueFormat" : { + "format" : "PROTOBUF_NOSR", + "properties" : { } + } + }, + "windowInfo" : null, + "orReplace" : false, + "isSource" : false + }, + "queryPlan" : { + "sources" : [ "TEST" ], + "sink" : "S2", + "physicalPlan" : { + "@type" : "tableSinkV1", + "properties" : { + "queryContext" : "S2" + }, + "source" : { + "@type" : "tableSelectV1", + "properties" : { + "queryContext" : "Aggregate/Project" + }, + "source" : { + "@type" : "streamAggregateV1", + "properties" : { + "queryContext" : "Aggregate/Aggregate" + }, + "source" : { + "@type" : "streamGroupByKeyV1", + "properties" : { + "queryContext" : "Aggregate/GroupBy" + }, + "source" : { + "@type" : "streamSelectV1", + "properties" : { + "queryContext" : "Aggregate/Prepare" + }, + "source" : { + "@type" : "streamSourceV1", + "properties" : { + "queryContext" : "KsqlTopic/Source" + }, + "topicName" : "test_topic", + "formats" : { + "keyFormat" : { + "format" : "KAFKA", + "properties" : { } + }, + "valueFormat" : { + "format" : "PROTOBUF_NOSR", + "properties" : { } + } + }, + "timestampColumn" : null, + "sourceSchema" : "`ID` BIGINT KEY, `NAME` STRING, `VALUE` INTEGER", + "pseudoColumnVersion" : 1 + }, + "keyColumnNames" : [ "ID" ], + "selectedKeys" : null, + "selectExpressions" : [ "ID AS ID", "VALUE AS VALUE", "3 AS KSQL_INTERNAL_COL_2" ] + }, + "internalFormats" : { + "keyFormat" : { + "format" : "KAFKA", + "properties" : { } + }, + "valueFormat" : { + "format" : "PROTOBUF_NOSR", + "properties" : { } + } + } + }, + "internalFormats" : { + "keyFormat" : { + "format" : "KAFKA", + "properties" : { } + }, + "valueFormat" : { + "format" : "PROTOBUF_NOSR", + "properties" : { } + } + }, + "nonAggregateColumns" : [ "ID", "VALUE" ], + "aggregationFunctions" : [ "TOPK(VALUE, 3)" ] + }, + "keyColumnNames" : [ "ID" ], + "selectExpressions" : [ "KSQL_AGG_VARIABLE_0 AS TOPK" ], + "internalFormats" : { + "keyFormat" : { + "format" : "KAFKA", + "properties" : { } + }, + "valueFormat" : { + "format" : "PROTOBUF_NOSR", + "properties" : { } + } + } + }, + "formats" : { + "keyFormat" : { + "format" : "KAFKA", + "properties" : { } + }, + "valueFormat" : { + "format" : "PROTOBUF_NOSR", + "properties" : { } + } + }, + "topicName" : "S2", + "timestampColumn" : null + }, + "queryId" : "CTAS_S2_0", + "runtimeId" : null + } + } ], + "configs" : { + "ksql.extension.dir" : "ext", + "ksql.streams.cache.max.bytes.buffering" : "0", + "metric.reporters" : "", + "ksql.query.status.running.threshold.seconds" : "300", + "ksql.connect.basic.auth.credentials.reload" : "false", + "ksql.output.topic.name.prefix" : "", + "ksql.query.pull.stream.enabled" : "true", + "ksql.query.push.v2.interpreter.enabled" : "true", + "ksql.queryanonymizer.logs_enabled" : "true", + "ksql.variable.substitution.enable" : "true", + "ksql.streams.shutdown.timeout.ms" : "300000", + "ksql.query.pull.max.allowed.offset.lag" : "9223372036854775807", + "ksql.query.pull.max.qps" : "2147483647", + "ksql.access.validator.enable" : "auto", + "ksql.streams.bootstrap.servers" : "localhost:0", + "ksql.query.pull.metrics.enabled" : "true", + "ksql.metrics.extension" : null, + "ksql.query.push.v2.alos.enabled" : "true", + "ksql.query.push.v2.max.hourly.bandwidth.megabytes" : "2147483647", + "ksql.query.pull.range.scan.enabled" : "true", + "ksql.transient.query.cleanup.service.initial.delay.seconds" : "600", + "ksql.hidden.topics" : "_confluent.*,__confluent.*,_schemas,__consumer_offsets,__transaction_state,connect-configs,connect-offsets,connect-status,connect-statuses", + "ksql.lambdas.enabled" : "true", + "ksql.source.table.materialization.enabled" : "true", + "ksql.query.pull.max.hourly.bandwidth.megabytes" : "2147483647", + "ksql.sink.window.change.log.additional.retention" : "1000000", + "ksql.query.persistent.active.limit" : "2147483647", + "ksql.persistence.wrap.single.values" : null, + "ksql.query.transient.max.bytes.buffering.total" : "-1", + "ksql.connect.basic.auth.credentials.source" : "NONE", + "ksql.schema.registry.url" : "schema_registry.url:0", + "ksql.properties.overrides.denylist" : "", + "ksql.service.id" : "some.ksql.service.id", + "ksql.query.push.v2.max.catchup.consumers" : "5", + "ksql.query.push.v2.enabled" : "false", + "ksql.transient.query.cleanup.service.enable" : "true", + "ksql.query.push.v2.metrics.enabled" : "true", + "ksql.rowpartition.rowoffset.enabled" : "true", + "ksql.streams.commit.interval.ms" : "2000", + "ksql.query.pull.table.scan.enabled" : "true", + "ksql.streams.auto.commit.interval.ms" : "0", + "ksql.streams.topology.optimization" : "all", + "ksql.endpoint.migrate.query" : "true", + "ksql.query.push.v2.registry.installed" : "false", + "ksql.streams.num.stream.threads" : "4", + "ksql.metrics.tags.custom" : "", + "ksql.query.push.v2.catchup.consumer.msg.window" : "50", + "ksql.runtime.feature.shared.enabled" : "false", + "ksql.udf.collect.metrics" : "false", + "ksql.new.query.planner.enabled" : "false", + "ksql.connect.request.headers.plugin" : null, + "ksql.security.extension.class" : null, + "ksql.transient.prefix" : "transient_", + "ksql.headers.columns.enabled" : "true", + "ksql.streams.default.deserialization.exception.handler" : "io.confluent.ksql.errors.LogMetricAndContinueExceptionHandler", + "ksql.connect.request.timeout.ms" : "5000", + "ksql.query.pull.enable.standby.reads" : "false", + "ksql.persistence.default.format.key" : "KAFKA", + "ksql.query.persistent.max.bytes.buffering.total" : "-1", + "ksql.query.error.max.queue.size" : "10", + "ksql.query.cleanup.shutdown.timeout.ms" : "30000", + "ksql.internal.topic.min.insync.replicas" : "1", + "ksql.internal.topic.replicas" : "1", + "ksql.insert.into.values.enabled" : "true", + "ksql.queryanonymizer.cluster_namespace" : null, + "ksql.create.or.replace.enabled" : "true", + "ksql.shared.runtimes.count" : "2", + "ksql.cast.strings.preserve.nulls" : "true", + "ksql.authorization.cache.max.entries" : "10000", + "ksql.pull.queries.enable" : "true", + "ksql.transient.query.cleanup.service.period.seconds" : "600", + "ksql.suppress.enabled" : "false", + "ksql.readonly.topics" : "_confluent.*,__confluent.*,_schemas,__consumer_offsets,__transaction_state,connect-configs,connect-offsets,connect-status,connect-statuses", + "ksql.connect.basic.auth.credentials.file" : "", + "ksql.authorization.cache.expiry.time.secs" : "30", + "ksql.query.retry.backoff.initial.ms" : "15000", + "ksql.query.pull.max.concurrent.requests" : "2147483647", + "ksql.streams.auto.offset.reset" : "earliest", + "ksql.connect.url" : "http://localhost:8083", + "ksql.query.push.v2.new.latest.delay.ms" : "5000", + "ksql.query.push.v2.latest.reset.age.ms" : "30000", + "ksql.streams.default.production.exception.handler" : "io.confluent.ksql.errors.ProductionExceptionHandlerUtil$LogAndFailProductionExceptionHandler", + "ksql.query.pull.interpreter.enabled" : "true", + "ksql.query.pull.limit.clause.enabled" : "true", + "ksql.query.pull.router.thread.pool.size" : "50", + "ksql.query.push.v2.continuation.tokens.enabled" : "false", + "ksql.query.retry.backoff.max.ms" : "900000", + "ksql.timestamp.throw.on.invalid" : "false", + "ksql.persistence.default.format.value" : null, + "ksql.udfs.enabled" : "true", + "ksql.udf.enable.security.manager" : "true", + "ksql.connect.worker.config" : "", + "ksql.nested.error.set.null" : "true", + "ksql.query.pull.thread.pool.size" : "50", + "ksql.persistent.prefix" : "query_", + "ksql.metastore.backup.location" : "", + "ksql.error.classifier.regex" : "", + "ksql.suppress.buffer.size.bytes" : "-1" + } +} \ No newline at end of file diff --git a/ksqldb-functional-tests/src/test/resources/historical_plans/topk-group-by_-_topk_integer_-_PROTOBUF_NOSR/7.3.0_1651070244684/spec.json b/ksqldb-functional-tests/src/test/resources/historical_plans/topk-group-by_-_topk_integer_-_PROTOBUF_NOSR/7.3.0_1651070244684/spec.json new file mode 100644 index 000000000000..17fbcefd2122 --- /dev/null +++ b/ksqldb-functional-tests/src/test/resources/historical_plans/topk-group-by_-_topk_integer_-_PROTOBUF_NOSR/7.3.0_1651070244684/spec.json @@ -0,0 +1,191 @@ +{ + "version" : "7.3.0", + "timestamp" : 1651070244684, + "path" : "query-validation-tests/topk-group-by.json", + "schemas" : { + "CTAS_S2_0.Aggregate.Aggregate.Materialize" : { + "schema" : "`ID` BIGINT KEY, `ID` BIGINT, `VALUE` INTEGER, `KSQL_AGG_VARIABLE_0` ARRAY", + "keyFormat" : { + "format" : "KAFKA" + }, + "valueFormat" : { + "format" : "PROTOBUF_NOSR" + } + }, + "CTAS_S2_0.Aggregate.Project" : { + "schema" : "`ID` BIGINT KEY, `TOPK` ARRAY", + "keyFormat" : { + "format" : "KAFKA" + }, + "valueFormat" : { + "format" : "PROTOBUF_NOSR" + } + }, + "CTAS_S2_0.KsqlTopic.Source" : { + "schema" : "`ID` BIGINT KEY, `NAME` STRING, `VALUE` INTEGER", + "keyFormat" : { + "format" : "KAFKA" + }, + "valueFormat" : { + "format" : "PROTOBUF_NOSR" + } + }, + "CTAS_S2_0.S2" : { + "schema" : "`ID` BIGINT KEY, `TOPK` ARRAY", + "keyFormat" : { + "format" : "KAFKA" + }, + "valueFormat" : { + "format" : "PROTOBUF_NOSR" + } + }, + "CTAS_S2_0.Aggregate.GroupBy" : { + "schema" : "`ID` BIGINT KEY, `ID` BIGINT, `VALUE` INTEGER, `KSQL_INTERNAL_COL_2` INTEGER", + "keyFormat" : { + "format" : "KAFKA" + }, + "valueFormat" : { + "format" : "PROTOBUF_NOSR" + } + } + }, + "testCase" : { + "name" : "topk integer - PROTOBUF_NOSR", + "inputs" : [ { + "topic" : "test_topic", + "key" : 0, + "value" : { + "NAME" : "zero", + "key" : 0, + "value" : 0 + } + }, { + "topic" : "test_topic", + "key" : 0, + "value" : { + "NAME" : "zero", + "key" : 0, + "value" : 100 + } + }, { + "topic" : "test_topic", + "key" : 0, + "value" : { + "NAME" : "zero", + "key" : 0, + "value" : 99 + } + }, { + "topic" : "test_topic", + "key" : 0, + "value" : { + "NAME" : "zero", + "key" : 0, + "value" : 7 + } + }, { + "topic" : "test_topic", + "key" : 0, + "value" : { + "NAME" : "zero", + "key" : 0, + "value" : 100 + } + } ], + "outputs" : [ { + "topic" : "S2", + "key" : 0, + "value" : { + "TOPK" : [ 0 ] + } + }, { + "topic" : "S2", + "key" : 0, + "value" : { + "TOPK" : [ 100, 0 ] + } + }, { + "topic" : "S2", + "key" : 0, + "value" : { + "TOPK" : [ 100, 99, 0 ] + } + }, { + "topic" : "S2", + "key" : 0, + "value" : { + "TOPK" : [ 100, 99, 7 ] + } + }, { + "topic" : "S2", + "key" : 0, + "value" : { + "TOPK" : [ 100, 100, 99 ] + } + } ], + "topics" : [ { + "name" : "test_topic", + "replicas" : 1, + "numPartitions" : 4 + }, { + "name" : "S2", + "replicas" : 1, + "numPartitions" : 4 + } ], + "statements" : [ "CREATE STREAM TEST (ID BIGINT KEY, NAME varchar, VALUE integer) WITH (kafka_topic='test_topic',value_format='PROTOBUF_NOSR');", "CREATE TABLE S2 as SELECT ID, topk(value, 3) as topk FROM test group by id;" ], + "post" : { + "sources" : [ { + "name" : "S2", + "type" : "TABLE", + "schema" : "`ID` BIGINT KEY, `TOPK` ARRAY", + "keyFormat" : { + "format" : "KAFKA" + }, + "valueFormat" : "PROTOBUF_NOSR", + "keyFeatures" : [ ], + "valueFeatures" : [ ], + "isSource" : false + }, { + "name" : "TEST", + "type" : "STREAM", + "schema" : "`ID` BIGINT KEY, `NAME` STRING, `VALUE` INTEGER", + "keyFormat" : { + "format" : "KAFKA" + }, + "valueFormat" : "PROTOBUF_NOSR", + "keyFeatures" : [ ], + "valueFeatures" : [ ], + "isSource" : false + } ], + "topics" : { + "topics" : [ { + "name" : "_confluent-ksql-some.ksql.service.idquery_CTAS_S2_0-Aggregate-Aggregate-Materialize-changelog", + "keyFormat" : { + "format" : "KAFKA" + }, + "valueFormat" : { + "format" : "PROTOBUF_NOSR" + } + }, { + "name" : "test_topic", + "keyFormat" : { + "format" : "KAFKA" + }, + "valueFormat" : { + "format" : "PROTOBUF_NOSR" + }, + "partitions" : 4 + }, { + "name" : "S2", + "keyFormat" : { + "format" : "KAFKA" + }, + "valueFormat" : { + "format" : "PROTOBUF_NOSR" + }, + "partitions" : 4 + } ] + } + } + } +} \ No newline at end of file diff --git a/ksqldb-functional-tests/src/test/resources/historical_plans/topk-group-by_-_topk_integer_-_PROTOBUF_NOSR/7.3.0_1651070244684/topology b/ksqldb-functional-tests/src/test/resources/historical_plans/topk-group-by_-_topk_integer_-_PROTOBUF_NOSR/7.3.0_1651070244684/topology new file mode 100644 index 000000000000..fcfc8fb28665 --- /dev/null +++ b/ksqldb-functional-tests/src/test/resources/historical_plans/topk-group-by_-_topk_integer_-_PROTOBUF_NOSR/7.3.0_1651070244684/topology @@ -0,0 +1,25 @@ +Topologies: + Sub-topology: 0 + Source: KSTREAM-SOURCE-0000000000 (topics: [test_topic]) + --> KSTREAM-TRANSFORMVALUES-0000000001 + Processor: KSTREAM-TRANSFORMVALUES-0000000001 (stores: []) + --> Aggregate-Prepare + <-- KSTREAM-SOURCE-0000000000 + Processor: Aggregate-Prepare (stores: []) + --> KSTREAM-AGGREGATE-0000000003 + <-- KSTREAM-TRANSFORMVALUES-0000000001 + Processor: KSTREAM-AGGREGATE-0000000003 (stores: [Aggregate-Aggregate-Materialize]) + --> Aggregate-Aggregate-ToOutputSchema + <-- Aggregate-Prepare + Processor: Aggregate-Aggregate-ToOutputSchema (stores: []) + --> Aggregate-Project + <-- KSTREAM-AGGREGATE-0000000003 + Processor: Aggregate-Project (stores: []) + --> KTABLE-TOSTREAM-0000000006 + <-- Aggregate-Aggregate-ToOutputSchema + Processor: KTABLE-TOSTREAM-0000000006 (stores: []) + --> KSTREAM-SINK-0000000007 + <-- Aggregate-Project + Sink: KSTREAM-SINK-0000000007 (topic: S2) + <-- KTABLE-TOSTREAM-0000000006 + diff --git a/ksqldb-functional-tests/src/test/resources/historical_plans/topk-group-by_-_topk_long_-_PROTOBUF_NOSR/7.3.0_1651070247852/plan.json b/ksqldb-functional-tests/src/test/resources/historical_plans/topk-group-by_-_topk_long_-_PROTOBUF_NOSR/7.3.0_1651070247852/plan.json new file mode 100644 index 000000000000..f009f0a50eb3 --- /dev/null +++ b/ksqldb-functional-tests/src/test/resources/historical_plans/topk-group-by_-_topk_long_-_PROTOBUF_NOSR/7.3.0_1651070247852/plan.json @@ -0,0 +1,255 @@ +{ + "plan" : [ { + "@type" : "ksqlPlanV1", + "statementText" : "CREATE STREAM TEST (ID BIGINT KEY, NAME STRING, VALUE BIGINT) WITH (KAFKA_TOPIC='test_topic', KEY_FORMAT='KAFKA', VALUE_FORMAT='PROTOBUF_NOSR');", + "ddlCommand" : { + "@type" : "createStreamV1", + "sourceName" : "TEST", + "schema" : "`ID` BIGINT KEY, `NAME` STRING, `VALUE` BIGINT", + "timestampColumn" : null, + "topicName" : "test_topic", + "formats" : { + "keyFormat" : { + "format" : "KAFKA", + "properties" : { } + }, + "valueFormat" : { + "format" : "PROTOBUF_NOSR", + "properties" : { } + } + }, + "windowInfo" : null, + "orReplace" : false, + "isSource" : false + }, + "queryPlan" : null + }, { + "@type" : "ksqlPlanV1", + "statementText" : "CREATE TABLE S2 AS SELECT\n TEST.ID ID,\n TOPK(TEST.VALUE, 3) TOPK\nFROM TEST TEST\nGROUP BY TEST.ID\nEMIT CHANGES", + "ddlCommand" : { + "@type" : "createTableV1", + "sourceName" : "S2", + "schema" : "`ID` BIGINT KEY, `TOPK` ARRAY", + "timestampColumn" : null, + "topicName" : "S2", + "formats" : { + "keyFormat" : { + "format" : "KAFKA", + "properties" : { } + }, + "valueFormat" : { + "format" : "PROTOBUF_NOSR", + "properties" : { } + } + }, + "windowInfo" : null, + "orReplace" : false, + "isSource" : false + }, + "queryPlan" : { + "sources" : [ "TEST" ], + "sink" : "S2", + "physicalPlan" : { + "@type" : "tableSinkV1", + "properties" : { + "queryContext" : "S2" + }, + "source" : { + "@type" : "tableSelectV1", + "properties" : { + "queryContext" : "Aggregate/Project" + }, + "source" : { + "@type" : "streamAggregateV1", + "properties" : { + "queryContext" : "Aggregate/Aggregate" + }, + "source" : { + "@type" : "streamGroupByKeyV1", + "properties" : { + "queryContext" : "Aggregate/GroupBy" + }, + "source" : { + "@type" : "streamSelectV1", + "properties" : { + "queryContext" : "Aggregate/Prepare" + }, + "source" : { + "@type" : "streamSourceV1", + "properties" : { + "queryContext" : "KsqlTopic/Source" + }, + "topicName" : "test_topic", + "formats" : { + "keyFormat" : { + "format" : "KAFKA", + "properties" : { } + }, + "valueFormat" : { + "format" : "PROTOBUF_NOSR", + "properties" : { } + } + }, + "timestampColumn" : null, + "sourceSchema" : "`ID` BIGINT KEY, `NAME` STRING, `VALUE` BIGINT", + "pseudoColumnVersion" : 1 + }, + "keyColumnNames" : [ "ID" ], + "selectedKeys" : null, + "selectExpressions" : [ "ID AS ID", "VALUE AS VALUE", "3 AS KSQL_INTERNAL_COL_2" ] + }, + "internalFormats" : { + "keyFormat" : { + "format" : "KAFKA", + "properties" : { } + }, + "valueFormat" : { + "format" : "PROTOBUF_NOSR", + "properties" : { } + } + } + }, + "internalFormats" : { + "keyFormat" : { + "format" : "KAFKA", + "properties" : { } + }, + "valueFormat" : { + "format" : "PROTOBUF_NOSR", + "properties" : { } + } + }, + "nonAggregateColumns" : [ "ID", "VALUE" ], + "aggregationFunctions" : [ "TOPK(VALUE, 3)" ] + }, + "keyColumnNames" : [ "ID" ], + "selectExpressions" : [ "KSQL_AGG_VARIABLE_0 AS TOPK" ], + "internalFormats" : { + "keyFormat" : { + "format" : "KAFKA", + "properties" : { } + }, + "valueFormat" : { + "format" : "PROTOBUF_NOSR", + "properties" : { } + } + } + }, + "formats" : { + "keyFormat" : { + "format" : "KAFKA", + "properties" : { } + }, + "valueFormat" : { + "format" : "PROTOBUF_NOSR", + "properties" : { } + } + }, + "topicName" : "S2", + "timestampColumn" : null + }, + "queryId" : "CTAS_S2_0", + "runtimeId" : null + } + } ], + "configs" : { + "ksql.extension.dir" : "ext", + "ksql.streams.cache.max.bytes.buffering" : "0", + "metric.reporters" : "", + "ksql.query.status.running.threshold.seconds" : "300", + "ksql.connect.basic.auth.credentials.reload" : "false", + "ksql.output.topic.name.prefix" : "", + "ksql.query.pull.stream.enabled" : "true", + "ksql.query.push.v2.interpreter.enabled" : "true", + "ksql.queryanonymizer.logs_enabled" : "true", + "ksql.variable.substitution.enable" : "true", + "ksql.streams.shutdown.timeout.ms" : "300000", + "ksql.query.pull.max.allowed.offset.lag" : "9223372036854775807", + "ksql.query.pull.max.qps" : "2147483647", + "ksql.access.validator.enable" : "auto", + "ksql.streams.bootstrap.servers" : "localhost:0", + "ksql.query.pull.metrics.enabled" : "true", + "ksql.metrics.extension" : null, + "ksql.query.push.v2.alos.enabled" : "true", + "ksql.query.push.v2.max.hourly.bandwidth.megabytes" : "2147483647", + "ksql.query.pull.range.scan.enabled" : "true", + "ksql.transient.query.cleanup.service.initial.delay.seconds" : "600", + "ksql.hidden.topics" : "_confluent.*,__confluent.*,_schemas,__consumer_offsets,__transaction_state,connect-configs,connect-offsets,connect-status,connect-statuses", + "ksql.lambdas.enabled" : "true", + "ksql.source.table.materialization.enabled" : "true", + "ksql.query.pull.max.hourly.bandwidth.megabytes" : "2147483647", + "ksql.sink.window.change.log.additional.retention" : "1000000", + "ksql.query.persistent.active.limit" : "2147483647", + "ksql.persistence.wrap.single.values" : null, + "ksql.query.transient.max.bytes.buffering.total" : "-1", + "ksql.connect.basic.auth.credentials.source" : "NONE", + "ksql.schema.registry.url" : "schema_registry.url:0", + "ksql.properties.overrides.denylist" : "", + "ksql.service.id" : "some.ksql.service.id", + "ksql.query.push.v2.max.catchup.consumers" : "5", + "ksql.query.push.v2.enabled" : "false", + "ksql.transient.query.cleanup.service.enable" : "true", + "ksql.query.push.v2.metrics.enabled" : "true", + "ksql.rowpartition.rowoffset.enabled" : "true", + "ksql.streams.commit.interval.ms" : "2000", + "ksql.query.pull.table.scan.enabled" : "true", + "ksql.streams.auto.commit.interval.ms" : "0", + "ksql.streams.topology.optimization" : "all", + "ksql.endpoint.migrate.query" : "true", + "ksql.query.push.v2.registry.installed" : "false", + "ksql.streams.num.stream.threads" : "4", + "ksql.metrics.tags.custom" : "", + "ksql.query.push.v2.catchup.consumer.msg.window" : "50", + "ksql.runtime.feature.shared.enabled" : "false", + "ksql.udf.collect.metrics" : "false", + "ksql.new.query.planner.enabled" : "false", + "ksql.connect.request.headers.plugin" : null, + "ksql.security.extension.class" : null, + "ksql.transient.prefix" : "transient_", + "ksql.headers.columns.enabled" : "true", + "ksql.streams.default.deserialization.exception.handler" : "io.confluent.ksql.errors.LogMetricAndContinueExceptionHandler", + "ksql.connect.request.timeout.ms" : "5000", + "ksql.query.pull.enable.standby.reads" : "false", + "ksql.persistence.default.format.key" : "KAFKA", + "ksql.query.persistent.max.bytes.buffering.total" : "-1", + "ksql.query.error.max.queue.size" : "10", + "ksql.query.cleanup.shutdown.timeout.ms" : "30000", + "ksql.internal.topic.min.insync.replicas" : "1", + "ksql.internal.topic.replicas" : "1", + "ksql.insert.into.values.enabled" : "true", + "ksql.queryanonymizer.cluster_namespace" : null, + "ksql.create.or.replace.enabled" : "true", + "ksql.shared.runtimes.count" : "2", + "ksql.cast.strings.preserve.nulls" : "true", + "ksql.authorization.cache.max.entries" : "10000", + "ksql.pull.queries.enable" : "true", + "ksql.transient.query.cleanup.service.period.seconds" : "600", + "ksql.suppress.enabled" : "false", + "ksql.readonly.topics" : "_confluent.*,__confluent.*,_schemas,__consumer_offsets,__transaction_state,connect-configs,connect-offsets,connect-status,connect-statuses", + "ksql.connect.basic.auth.credentials.file" : "", + "ksql.authorization.cache.expiry.time.secs" : "30", + "ksql.query.retry.backoff.initial.ms" : "15000", + "ksql.query.pull.max.concurrent.requests" : "2147483647", + "ksql.streams.auto.offset.reset" : "earliest", + "ksql.connect.url" : "http://localhost:8083", + "ksql.query.push.v2.new.latest.delay.ms" : "5000", + "ksql.query.push.v2.latest.reset.age.ms" : "30000", + "ksql.streams.default.production.exception.handler" : "io.confluent.ksql.errors.ProductionExceptionHandlerUtil$LogAndFailProductionExceptionHandler", + "ksql.query.pull.interpreter.enabled" : "true", + "ksql.query.pull.limit.clause.enabled" : "true", + "ksql.query.pull.router.thread.pool.size" : "50", + "ksql.query.push.v2.continuation.tokens.enabled" : "false", + "ksql.query.retry.backoff.max.ms" : "900000", + "ksql.timestamp.throw.on.invalid" : "false", + "ksql.persistence.default.format.value" : null, + "ksql.udfs.enabled" : "true", + "ksql.udf.enable.security.manager" : "true", + "ksql.connect.worker.config" : "", + "ksql.nested.error.set.null" : "true", + "ksql.query.pull.thread.pool.size" : "50", + "ksql.persistent.prefix" : "query_", + "ksql.metastore.backup.location" : "", + "ksql.error.classifier.regex" : "", + "ksql.suppress.buffer.size.bytes" : "-1" + } +} \ No newline at end of file diff --git a/ksqldb-functional-tests/src/test/resources/historical_plans/topk-group-by_-_topk_long_-_PROTOBUF_NOSR/7.3.0_1651070247852/spec.json b/ksqldb-functional-tests/src/test/resources/historical_plans/topk-group-by_-_topk_long_-_PROTOBUF_NOSR/7.3.0_1651070247852/spec.json new file mode 100644 index 000000000000..79863635d7d4 --- /dev/null +++ b/ksqldb-functional-tests/src/test/resources/historical_plans/topk-group-by_-_topk_long_-_PROTOBUF_NOSR/7.3.0_1651070247852/spec.json @@ -0,0 +1,191 @@ +{ + "version" : "7.3.0", + "timestamp" : 1651070247852, + "path" : "query-validation-tests/topk-group-by.json", + "schemas" : { + "CTAS_S2_0.Aggregate.Aggregate.Materialize" : { + "schema" : "`ID` BIGINT KEY, `ID` BIGINT, `VALUE` BIGINT, `KSQL_AGG_VARIABLE_0` ARRAY", + "keyFormat" : { + "format" : "KAFKA" + }, + "valueFormat" : { + "format" : "PROTOBUF_NOSR" + } + }, + "CTAS_S2_0.Aggregate.Project" : { + "schema" : "`ID` BIGINT KEY, `TOPK` ARRAY", + "keyFormat" : { + "format" : "KAFKA" + }, + "valueFormat" : { + "format" : "PROTOBUF_NOSR" + } + }, + "CTAS_S2_0.KsqlTopic.Source" : { + "schema" : "`ID` BIGINT KEY, `NAME` STRING, `VALUE` BIGINT", + "keyFormat" : { + "format" : "KAFKA" + }, + "valueFormat" : { + "format" : "PROTOBUF_NOSR" + } + }, + "CTAS_S2_0.S2" : { + "schema" : "`ID` BIGINT KEY, `TOPK` ARRAY", + "keyFormat" : { + "format" : "KAFKA" + }, + "valueFormat" : { + "format" : "PROTOBUF_NOSR" + } + }, + "CTAS_S2_0.Aggregate.GroupBy" : { + "schema" : "`ID` BIGINT KEY, `ID` BIGINT, `VALUE` BIGINT, `KSQL_INTERNAL_COL_2` INTEGER", + "keyFormat" : { + "format" : "KAFKA" + }, + "valueFormat" : { + "format" : "PROTOBUF_NOSR" + } + } + }, + "testCase" : { + "name" : "topk long - PROTOBUF_NOSR", + "inputs" : [ { + "topic" : "test_topic", + "key" : 0, + "value" : { + "NAME" : "zero", + "key" : 0, + "value" : 2147483648 + } + }, { + "topic" : "test_topic", + "key" : 0, + "value" : { + "NAME" : "zero", + "key" : 0, + "value" : 100 + } + }, { + "topic" : "test_topic", + "key" : 0, + "value" : { + "NAME" : "zero", + "key" : 0, + "value" : 99 + } + }, { + "topic" : "test_topic", + "key" : 0, + "value" : { + "NAME" : "zero", + "key" : 0, + "value" : 7 + } + }, { + "topic" : "test_topic", + "key" : 0, + "value" : { + "NAME" : "zero", + "key" : 0, + "value" : 100 + } + } ], + "outputs" : [ { + "topic" : "S2", + "key" : 0, + "value" : { + "TOPK" : [ 2147483648 ] + } + }, { + "topic" : "S2", + "key" : 0, + "value" : { + "TOPK" : [ 2147483648, 100 ] + } + }, { + "topic" : "S2", + "key" : 0, + "value" : { + "TOPK" : [ 2147483648, 100, 99 ] + } + }, { + "topic" : "S2", + "key" : 0, + "value" : { + "TOPK" : [ 2147483648, 100, 99 ] + } + }, { + "topic" : "S2", + "key" : 0, + "value" : { + "TOPK" : [ 2147483648, 100, 100 ] + } + } ], + "topics" : [ { + "name" : "test_topic", + "replicas" : 1, + "numPartitions" : 4 + }, { + "name" : "S2", + "replicas" : 1, + "numPartitions" : 4 + } ], + "statements" : [ "CREATE STREAM TEST (ID BIGINT KEY, NAME varchar, VALUE bigint) WITH (kafka_topic='test_topic', value_format='PROTOBUF_NOSR');", "CREATE TABLE S2 as SELECT ID, topk(value, 3) as topk FROM test group by id;" ], + "post" : { + "sources" : [ { + "name" : "S2", + "type" : "TABLE", + "schema" : "`ID` BIGINT KEY, `TOPK` ARRAY", + "keyFormat" : { + "format" : "KAFKA" + }, + "valueFormat" : "PROTOBUF_NOSR", + "keyFeatures" : [ ], + "valueFeatures" : [ ], + "isSource" : false + }, { + "name" : "TEST", + "type" : "STREAM", + "schema" : "`ID` BIGINT KEY, `NAME` STRING, `VALUE` BIGINT", + "keyFormat" : { + "format" : "KAFKA" + }, + "valueFormat" : "PROTOBUF_NOSR", + "keyFeatures" : [ ], + "valueFeatures" : [ ], + "isSource" : false + } ], + "topics" : { + "topics" : [ { + "name" : "_confluent-ksql-some.ksql.service.idquery_CTAS_S2_0-Aggregate-Aggregate-Materialize-changelog", + "keyFormat" : { + "format" : "KAFKA" + }, + "valueFormat" : { + "format" : "PROTOBUF_NOSR" + } + }, { + "name" : "test_topic", + "keyFormat" : { + "format" : "KAFKA" + }, + "valueFormat" : { + "format" : "PROTOBUF_NOSR" + }, + "partitions" : 4 + }, { + "name" : "S2", + "keyFormat" : { + "format" : "KAFKA" + }, + "valueFormat" : { + "format" : "PROTOBUF_NOSR" + }, + "partitions" : 4 + } ] + } + } + } +} \ No newline at end of file diff --git a/ksqldb-functional-tests/src/test/resources/historical_plans/topk-group-by_-_topk_long_-_PROTOBUF_NOSR/7.3.0_1651070247852/topology b/ksqldb-functional-tests/src/test/resources/historical_plans/topk-group-by_-_topk_long_-_PROTOBUF_NOSR/7.3.0_1651070247852/topology new file mode 100644 index 000000000000..fcfc8fb28665 --- /dev/null +++ b/ksqldb-functional-tests/src/test/resources/historical_plans/topk-group-by_-_topk_long_-_PROTOBUF_NOSR/7.3.0_1651070247852/topology @@ -0,0 +1,25 @@ +Topologies: + Sub-topology: 0 + Source: KSTREAM-SOURCE-0000000000 (topics: [test_topic]) + --> KSTREAM-TRANSFORMVALUES-0000000001 + Processor: KSTREAM-TRANSFORMVALUES-0000000001 (stores: []) + --> Aggregate-Prepare + <-- KSTREAM-SOURCE-0000000000 + Processor: Aggregate-Prepare (stores: []) + --> KSTREAM-AGGREGATE-0000000003 + <-- KSTREAM-TRANSFORMVALUES-0000000001 + Processor: KSTREAM-AGGREGATE-0000000003 (stores: [Aggregate-Aggregate-Materialize]) + --> Aggregate-Aggregate-ToOutputSchema + <-- Aggregate-Prepare + Processor: Aggregate-Aggregate-ToOutputSchema (stores: []) + --> Aggregate-Project + <-- KSTREAM-AGGREGATE-0000000003 + Processor: Aggregate-Project (stores: []) + --> KTABLE-TOSTREAM-0000000006 + <-- Aggregate-Aggregate-ToOutputSchema + Processor: KTABLE-TOSTREAM-0000000006 (stores: []) + --> KSTREAM-SINK-0000000007 + <-- Aggregate-Project + Sink: KSTREAM-SINK-0000000007 (topic: S2) + <-- KTABLE-TOSTREAM-0000000006 + diff --git a/ksqldb-functional-tests/src/test/resources/historical_plans/topk-group-by_-_topk_string_-_PROTOBUF_NOSR/7.3.0_1651070254618/plan.json b/ksqldb-functional-tests/src/test/resources/historical_plans/topk-group-by_-_topk_string_-_PROTOBUF_NOSR/7.3.0_1651070254618/plan.json new file mode 100644 index 000000000000..72a5be0e333a --- /dev/null +++ b/ksqldb-functional-tests/src/test/resources/historical_plans/topk-group-by_-_topk_string_-_PROTOBUF_NOSR/7.3.0_1651070254618/plan.json @@ -0,0 +1,255 @@ +{ + "plan" : [ { + "@type" : "ksqlPlanV1", + "statementText" : "CREATE STREAM TEST (ID BIGINT KEY, NAME STRING, VALUE STRING) WITH (KAFKA_TOPIC='test_topic', KEY_FORMAT='KAFKA', VALUE_FORMAT='PROTOBUF_NOSR');", + "ddlCommand" : { + "@type" : "createStreamV1", + "sourceName" : "TEST", + "schema" : "`ID` BIGINT KEY, `NAME` STRING, `VALUE` STRING", + "timestampColumn" : null, + "topicName" : "test_topic", + "formats" : { + "keyFormat" : { + "format" : "KAFKA", + "properties" : { } + }, + "valueFormat" : { + "format" : "PROTOBUF_NOSR", + "properties" : { } + } + }, + "windowInfo" : null, + "orReplace" : false, + "isSource" : false + }, + "queryPlan" : null + }, { + "@type" : "ksqlPlanV1", + "statementText" : "CREATE TABLE S2 AS SELECT\n TEST.ID ID,\n TOPK(TEST.VALUE, 3) TOPK\nFROM TEST TEST\nGROUP BY TEST.ID\nEMIT CHANGES", + "ddlCommand" : { + "@type" : "createTableV1", + "sourceName" : "S2", + "schema" : "`ID` BIGINT KEY, `TOPK` ARRAY", + "timestampColumn" : null, + "topicName" : "S2", + "formats" : { + "keyFormat" : { + "format" : "KAFKA", + "properties" : { } + }, + "valueFormat" : { + "format" : "PROTOBUF_NOSR", + "properties" : { } + } + }, + "windowInfo" : null, + "orReplace" : false, + "isSource" : false + }, + "queryPlan" : { + "sources" : [ "TEST" ], + "sink" : "S2", + "physicalPlan" : { + "@type" : "tableSinkV1", + "properties" : { + "queryContext" : "S2" + }, + "source" : { + "@type" : "tableSelectV1", + "properties" : { + "queryContext" : "Aggregate/Project" + }, + "source" : { + "@type" : "streamAggregateV1", + "properties" : { + "queryContext" : "Aggregate/Aggregate" + }, + "source" : { + "@type" : "streamGroupByKeyV1", + "properties" : { + "queryContext" : "Aggregate/GroupBy" + }, + "source" : { + "@type" : "streamSelectV1", + "properties" : { + "queryContext" : "Aggregate/Prepare" + }, + "source" : { + "@type" : "streamSourceV1", + "properties" : { + "queryContext" : "KsqlTopic/Source" + }, + "topicName" : "test_topic", + "formats" : { + "keyFormat" : { + "format" : "KAFKA", + "properties" : { } + }, + "valueFormat" : { + "format" : "PROTOBUF_NOSR", + "properties" : { } + } + }, + "timestampColumn" : null, + "sourceSchema" : "`ID` BIGINT KEY, `NAME` STRING, `VALUE` STRING", + "pseudoColumnVersion" : 1 + }, + "keyColumnNames" : [ "ID" ], + "selectedKeys" : null, + "selectExpressions" : [ "ID AS ID", "VALUE AS VALUE", "3 AS KSQL_INTERNAL_COL_2" ] + }, + "internalFormats" : { + "keyFormat" : { + "format" : "KAFKA", + "properties" : { } + }, + "valueFormat" : { + "format" : "PROTOBUF_NOSR", + "properties" : { } + } + } + }, + "internalFormats" : { + "keyFormat" : { + "format" : "KAFKA", + "properties" : { } + }, + "valueFormat" : { + "format" : "PROTOBUF_NOSR", + "properties" : { } + } + }, + "nonAggregateColumns" : [ "ID", "VALUE" ], + "aggregationFunctions" : [ "TOPK(VALUE, 3)" ] + }, + "keyColumnNames" : [ "ID" ], + "selectExpressions" : [ "KSQL_AGG_VARIABLE_0 AS TOPK" ], + "internalFormats" : { + "keyFormat" : { + "format" : "KAFKA", + "properties" : { } + }, + "valueFormat" : { + "format" : "PROTOBUF_NOSR", + "properties" : { } + } + } + }, + "formats" : { + "keyFormat" : { + "format" : "KAFKA", + "properties" : { } + }, + "valueFormat" : { + "format" : "PROTOBUF_NOSR", + "properties" : { } + } + }, + "topicName" : "S2", + "timestampColumn" : null + }, + "queryId" : "CTAS_S2_0", + "runtimeId" : null + } + } ], + "configs" : { + "ksql.extension.dir" : "ext", + "ksql.streams.cache.max.bytes.buffering" : "0", + "metric.reporters" : "", + "ksql.query.status.running.threshold.seconds" : "300", + "ksql.connect.basic.auth.credentials.reload" : "false", + "ksql.output.topic.name.prefix" : "", + "ksql.query.pull.stream.enabled" : "true", + "ksql.query.push.v2.interpreter.enabled" : "true", + "ksql.queryanonymizer.logs_enabled" : "true", + "ksql.variable.substitution.enable" : "true", + "ksql.streams.shutdown.timeout.ms" : "300000", + "ksql.query.pull.max.allowed.offset.lag" : "9223372036854775807", + "ksql.query.pull.max.qps" : "2147483647", + "ksql.access.validator.enable" : "auto", + "ksql.streams.bootstrap.servers" : "localhost:0", + "ksql.query.pull.metrics.enabled" : "true", + "ksql.metrics.extension" : null, + "ksql.query.push.v2.alos.enabled" : "true", + "ksql.query.push.v2.max.hourly.bandwidth.megabytes" : "2147483647", + "ksql.query.pull.range.scan.enabled" : "true", + "ksql.transient.query.cleanup.service.initial.delay.seconds" : "600", + "ksql.hidden.topics" : "_confluent.*,__confluent.*,_schemas,__consumer_offsets,__transaction_state,connect-configs,connect-offsets,connect-status,connect-statuses", + "ksql.lambdas.enabled" : "true", + "ksql.source.table.materialization.enabled" : "true", + "ksql.query.pull.max.hourly.bandwidth.megabytes" : "2147483647", + "ksql.sink.window.change.log.additional.retention" : "1000000", + "ksql.query.persistent.active.limit" : "2147483647", + "ksql.persistence.wrap.single.values" : null, + "ksql.query.transient.max.bytes.buffering.total" : "-1", + "ksql.connect.basic.auth.credentials.source" : "NONE", + "ksql.schema.registry.url" : "schema_registry.url:0", + "ksql.properties.overrides.denylist" : "", + "ksql.service.id" : "some.ksql.service.id", + "ksql.query.push.v2.max.catchup.consumers" : "5", + "ksql.query.push.v2.enabled" : "false", + "ksql.transient.query.cleanup.service.enable" : "true", + "ksql.query.push.v2.metrics.enabled" : "true", + "ksql.rowpartition.rowoffset.enabled" : "true", + "ksql.streams.commit.interval.ms" : "2000", + "ksql.query.pull.table.scan.enabled" : "true", + "ksql.streams.auto.commit.interval.ms" : "0", + "ksql.streams.topology.optimization" : "all", + "ksql.endpoint.migrate.query" : "true", + "ksql.query.push.v2.registry.installed" : "false", + "ksql.streams.num.stream.threads" : "4", + "ksql.metrics.tags.custom" : "", + "ksql.query.push.v2.catchup.consumer.msg.window" : "50", + "ksql.runtime.feature.shared.enabled" : "false", + "ksql.udf.collect.metrics" : "false", + "ksql.new.query.planner.enabled" : "false", + "ksql.connect.request.headers.plugin" : null, + "ksql.security.extension.class" : null, + "ksql.transient.prefix" : "transient_", + "ksql.headers.columns.enabled" : "true", + "ksql.streams.default.deserialization.exception.handler" : "io.confluent.ksql.errors.LogMetricAndContinueExceptionHandler", + "ksql.connect.request.timeout.ms" : "5000", + "ksql.query.pull.enable.standby.reads" : "false", + "ksql.persistence.default.format.key" : "KAFKA", + "ksql.query.persistent.max.bytes.buffering.total" : "-1", + "ksql.query.error.max.queue.size" : "10", + "ksql.query.cleanup.shutdown.timeout.ms" : "30000", + "ksql.internal.topic.min.insync.replicas" : "1", + "ksql.internal.topic.replicas" : "1", + "ksql.insert.into.values.enabled" : "true", + "ksql.queryanonymizer.cluster_namespace" : null, + "ksql.create.or.replace.enabled" : "true", + "ksql.shared.runtimes.count" : "2", + "ksql.cast.strings.preserve.nulls" : "true", + "ksql.authorization.cache.max.entries" : "10000", + "ksql.pull.queries.enable" : "true", + "ksql.transient.query.cleanup.service.period.seconds" : "600", + "ksql.suppress.enabled" : "false", + "ksql.readonly.topics" : "_confluent.*,__confluent.*,_schemas,__consumer_offsets,__transaction_state,connect-configs,connect-offsets,connect-status,connect-statuses", + "ksql.connect.basic.auth.credentials.file" : "", + "ksql.authorization.cache.expiry.time.secs" : "30", + "ksql.query.retry.backoff.initial.ms" : "15000", + "ksql.query.pull.max.concurrent.requests" : "2147483647", + "ksql.streams.auto.offset.reset" : "earliest", + "ksql.connect.url" : "http://localhost:8083", + "ksql.query.push.v2.new.latest.delay.ms" : "5000", + "ksql.query.push.v2.latest.reset.age.ms" : "30000", + "ksql.streams.default.production.exception.handler" : "io.confluent.ksql.errors.ProductionExceptionHandlerUtil$LogAndFailProductionExceptionHandler", + "ksql.query.pull.interpreter.enabled" : "true", + "ksql.query.pull.limit.clause.enabled" : "true", + "ksql.query.pull.router.thread.pool.size" : "50", + "ksql.query.push.v2.continuation.tokens.enabled" : "false", + "ksql.query.retry.backoff.max.ms" : "900000", + "ksql.timestamp.throw.on.invalid" : "false", + "ksql.persistence.default.format.value" : null, + "ksql.udfs.enabled" : "true", + "ksql.udf.enable.security.manager" : "true", + "ksql.connect.worker.config" : "", + "ksql.nested.error.set.null" : "true", + "ksql.query.pull.thread.pool.size" : "50", + "ksql.persistent.prefix" : "query_", + "ksql.metastore.backup.location" : "", + "ksql.error.classifier.regex" : "", + "ksql.suppress.buffer.size.bytes" : "-1" + } +} \ No newline at end of file diff --git a/ksqldb-functional-tests/src/test/resources/historical_plans/topk-group-by_-_topk_string_-_PROTOBUF_NOSR/7.3.0_1651070254618/spec.json b/ksqldb-functional-tests/src/test/resources/historical_plans/topk-group-by_-_topk_string_-_PROTOBUF_NOSR/7.3.0_1651070254618/spec.json new file mode 100644 index 000000000000..554321b84e6a --- /dev/null +++ b/ksqldb-functional-tests/src/test/resources/historical_plans/topk-group-by_-_topk_string_-_PROTOBUF_NOSR/7.3.0_1651070254618/spec.json @@ -0,0 +1,191 @@ +{ + "version" : "7.3.0", + "timestamp" : 1651070254618, + "path" : "query-validation-tests/topk-group-by.json", + "schemas" : { + "CTAS_S2_0.Aggregate.Aggregate.Materialize" : { + "schema" : "`ID` BIGINT KEY, `ID` BIGINT, `VALUE` STRING, `KSQL_AGG_VARIABLE_0` ARRAY", + "keyFormat" : { + "format" : "KAFKA" + }, + "valueFormat" : { + "format" : "PROTOBUF_NOSR" + } + }, + "CTAS_S2_0.Aggregate.Project" : { + "schema" : "`ID` BIGINT KEY, `TOPK` ARRAY", + "keyFormat" : { + "format" : "KAFKA" + }, + "valueFormat" : { + "format" : "PROTOBUF_NOSR" + } + }, + "CTAS_S2_0.KsqlTopic.Source" : { + "schema" : "`ID` BIGINT KEY, `NAME` STRING, `VALUE` STRING", + "keyFormat" : { + "format" : "KAFKA" + }, + "valueFormat" : { + "format" : "PROTOBUF_NOSR" + } + }, + "CTAS_S2_0.S2" : { + "schema" : "`ID` BIGINT KEY, `TOPK` ARRAY", + "keyFormat" : { + "format" : "KAFKA" + }, + "valueFormat" : { + "format" : "PROTOBUF_NOSR" + } + }, + "CTAS_S2_0.Aggregate.GroupBy" : { + "schema" : "`ID` BIGINT KEY, `ID` BIGINT, `VALUE` STRING, `KSQL_INTERNAL_COL_2` INTEGER", + "keyFormat" : { + "format" : "KAFKA" + }, + "valueFormat" : { + "format" : "PROTOBUF_NOSR" + } + } + }, + "testCase" : { + "name" : "topk string - PROTOBUF_NOSR", + "inputs" : [ { + "topic" : "test_topic", + "key" : 0, + "value" : { + "NAME" : "zero", + "key" : 0, + "value" : "a" + } + }, { + "topic" : "test_topic", + "key" : 0, + "value" : { + "NAME" : "zero", + "key" : 0, + "value" : "b" + } + }, { + "topic" : "test_topic", + "key" : 0, + "value" : { + "NAME" : "zero", + "key" : 0, + "value" : "c" + } + }, { + "topic" : "test_topic", + "key" : 0, + "value" : { + "NAME" : "zero", + "key" : 0, + "value" : "b" + } + }, { + "topic" : "test_topic", + "key" : 0, + "value" : { + "NAME" : "zero", + "key" : 0, + "value" : "d" + } + } ], + "outputs" : [ { + "topic" : "S2", + "key" : 0, + "value" : { + "TOPK" : [ "a" ] + } + }, { + "topic" : "S2", + "key" : 0, + "value" : { + "TOPK" : [ "b", "a" ] + } + }, { + "topic" : "S2", + "key" : 0, + "value" : { + "TOPK" : [ "c", "b", "a" ] + } + }, { + "topic" : "S2", + "key" : 0, + "value" : { + "TOPK" : [ "c", "b", "b" ] + } + }, { + "topic" : "S2", + "key" : 0, + "value" : { + "TOPK" : [ "d", "c", "b" ] + } + } ], + "topics" : [ { + "name" : "test_topic", + "replicas" : 1, + "numPartitions" : 4 + }, { + "name" : "S2", + "replicas" : 1, + "numPartitions" : 4 + } ], + "statements" : [ "CREATE STREAM TEST (ID BIGINT KEY, NAME varchar, VALUE string) WITH (kafka_topic='test_topic', value_format='PROTOBUF_NOSR');", "CREATE TABLE S2 as SELECT ID, topk(value, 3) as topk FROM test group by id;" ], + "post" : { + "sources" : [ { + "name" : "S2", + "type" : "TABLE", + "schema" : "`ID` BIGINT KEY, `TOPK` ARRAY", + "keyFormat" : { + "format" : "KAFKA" + }, + "valueFormat" : "PROTOBUF_NOSR", + "keyFeatures" : [ ], + "valueFeatures" : [ ], + "isSource" : false + }, { + "name" : "TEST", + "type" : "STREAM", + "schema" : "`ID` BIGINT KEY, `NAME` STRING, `VALUE` STRING", + "keyFormat" : { + "format" : "KAFKA" + }, + "valueFormat" : "PROTOBUF_NOSR", + "keyFeatures" : [ ], + "valueFeatures" : [ ], + "isSource" : false + } ], + "topics" : { + "topics" : [ { + "name" : "_confluent-ksql-some.ksql.service.idquery_CTAS_S2_0-Aggregate-Aggregate-Materialize-changelog", + "keyFormat" : { + "format" : "KAFKA" + }, + "valueFormat" : { + "format" : "PROTOBUF_NOSR" + } + }, { + "name" : "test_topic", + "keyFormat" : { + "format" : "KAFKA" + }, + "valueFormat" : { + "format" : "PROTOBUF_NOSR" + }, + "partitions" : 4 + }, { + "name" : "S2", + "keyFormat" : { + "format" : "KAFKA" + }, + "valueFormat" : { + "format" : "PROTOBUF_NOSR" + }, + "partitions" : 4 + } ] + } + } + } +} \ No newline at end of file diff --git a/ksqldb-functional-tests/src/test/resources/historical_plans/topk-group-by_-_topk_string_-_PROTOBUF_NOSR/7.3.0_1651070254618/topology b/ksqldb-functional-tests/src/test/resources/historical_plans/topk-group-by_-_topk_string_-_PROTOBUF_NOSR/7.3.0_1651070254618/topology new file mode 100644 index 000000000000..fcfc8fb28665 --- /dev/null +++ b/ksqldb-functional-tests/src/test/resources/historical_plans/topk-group-by_-_topk_string_-_PROTOBUF_NOSR/7.3.0_1651070254618/topology @@ -0,0 +1,25 @@ +Topologies: + Sub-topology: 0 + Source: KSTREAM-SOURCE-0000000000 (topics: [test_topic]) + --> KSTREAM-TRANSFORMVALUES-0000000001 + Processor: KSTREAM-TRANSFORMVALUES-0000000001 (stores: []) + --> Aggregate-Prepare + <-- KSTREAM-SOURCE-0000000000 + Processor: Aggregate-Prepare (stores: []) + --> KSTREAM-AGGREGATE-0000000003 + <-- KSTREAM-TRANSFORMVALUES-0000000001 + Processor: KSTREAM-AGGREGATE-0000000003 (stores: [Aggregate-Aggregate-Materialize]) + --> Aggregate-Aggregate-ToOutputSchema + <-- Aggregate-Prepare + Processor: Aggregate-Aggregate-ToOutputSchema (stores: []) + --> Aggregate-Project + <-- KSTREAM-AGGREGATE-0000000003 + Processor: Aggregate-Project (stores: []) + --> KTABLE-TOSTREAM-0000000006 + <-- Aggregate-Aggregate-ToOutputSchema + Processor: KTABLE-TOSTREAM-0000000006 (stores: []) + --> KSTREAM-SINK-0000000007 + <-- Aggregate-Project + Sink: KSTREAM-SINK-0000000007 (topic: S2) + <-- KTABLE-TOSTREAM-0000000006 + diff --git a/ksqldb-functional-tests/src/test/resources/historical_plans/url_-_chain_a_call_to_URL_EXTRACT_PARAMETER_with_URL_DECODE_PARAM_-_PROTOBUF/7.3.0_1651070264604/plan.json b/ksqldb-functional-tests/src/test/resources/historical_plans/url_-_chain_a_call_to_URL_EXTRACT_PARAMETER_with_URL_DECODE_PARAM_-_PROTOBUF/7.3.0_1651070264604/plan.json new file mode 100644 index 000000000000..8339851ce9cc --- /dev/null +++ b/ksqldb-functional-tests/src/test/resources/historical_plans/url_-_chain_a_call_to_URL_EXTRACT_PARAMETER_with_URL_DECODE_PARAM_-_PROTOBUF/7.3.0_1651070264604/plan.json @@ -0,0 +1,211 @@ +{ + "plan" : [ { + "@type" : "ksqlPlanV1", + "statementText" : "CREATE STREAM TEST (K STRING KEY, URL STRING) WITH (KAFKA_TOPIC='test_topic', KEY_FORMAT='KAFKA', VALUE_FORMAT='PROTOBUF');", + "ddlCommand" : { + "@type" : "createStreamV1", + "sourceName" : "TEST", + "schema" : "`K` STRING KEY, `URL` STRING", + "timestampColumn" : null, + "topicName" : "test_topic", + "formats" : { + "keyFormat" : { + "format" : "KAFKA", + "properties" : { } + }, + "valueFormat" : { + "format" : "PROTOBUF", + "properties" : { + "unwrapPrimitives" : "true" + } + } + }, + "windowInfo" : null, + "orReplace" : false, + "isSource" : false + }, + "queryPlan" : null + }, { + "@type" : "ksqlPlanV1", + "statementText" : "CREATE STREAM OUTPUT AS SELECT\n TEST.K K,\n URL_DECODE_PARAM(URL_EXTRACT_PARAMETER(TEST.URL, 'two')) PARAM\nFROM TEST TEST\nEMIT CHANGES", + "ddlCommand" : { + "@type" : "createStreamV1", + "sourceName" : "OUTPUT", + "schema" : "`K` STRING KEY, `PARAM` STRING", + "timestampColumn" : null, + "topicName" : "OUTPUT", + "formats" : { + "keyFormat" : { + "format" : "KAFKA", + "properties" : { } + }, + "valueFormat" : { + "format" : "PROTOBUF", + "properties" : { + "unwrapPrimitives" : "true" + } + } + }, + "windowInfo" : null, + "orReplace" : false, + "isSource" : false + }, + "queryPlan" : { + "sources" : [ "TEST" ], + "sink" : "OUTPUT", + "physicalPlan" : { + "@type" : "streamSinkV1", + "properties" : { + "queryContext" : "OUTPUT" + }, + "source" : { + "@type" : "streamSelectV1", + "properties" : { + "queryContext" : "Project" + }, + "source" : { + "@type" : "streamSourceV1", + "properties" : { + "queryContext" : "KsqlTopic/Source" + }, + "topicName" : "test_topic", + "formats" : { + "keyFormat" : { + "format" : "KAFKA", + "properties" : { } + }, + "valueFormat" : { + "format" : "PROTOBUF", + "properties" : { + "unwrapPrimitives" : "true" + } + } + }, + "timestampColumn" : null, + "sourceSchema" : "`K` STRING KEY, `URL` STRING", + "pseudoColumnVersion" : 1 + }, + "keyColumnNames" : [ "K" ], + "selectedKeys" : null, + "selectExpressions" : [ "URL_DECODE_PARAM(URL_EXTRACT_PARAMETER(URL, 'two')) AS PARAM" ] + }, + "formats" : { + "keyFormat" : { + "format" : "KAFKA", + "properties" : { } + }, + "valueFormat" : { + "format" : "PROTOBUF", + "properties" : { + "unwrapPrimitives" : "true" + } + } + }, + "topicName" : "OUTPUT", + "timestampColumn" : null + }, + "queryId" : "CSAS_OUTPUT_0", + "runtimeId" : null + } + } ], + "configs" : { + "ksql.extension.dir" : "ext", + "ksql.streams.cache.max.bytes.buffering" : "0", + "metric.reporters" : "", + "ksql.query.status.running.threshold.seconds" : "300", + "ksql.connect.basic.auth.credentials.reload" : "false", + "ksql.output.topic.name.prefix" : "", + "ksql.query.pull.stream.enabled" : "true", + "ksql.query.push.v2.interpreter.enabled" : "true", + "ksql.queryanonymizer.logs_enabled" : "true", + "ksql.variable.substitution.enable" : "true", + "ksql.streams.shutdown.timeout.ms" : "300000", + "ksql.query.pull.max.allowed.offset.lag" : "9223372036854775807", + "ksql.query.pull.max.qps" : "2147483647", + "ksql.access.validator.enable" : "auto", + "ksql.streams.bootstrap.servers" : "localhost:0", + "ksql.query.pull.metrics.enabled" : "true", + "ksql.metrics.extension" : null, + "ksql.query.push.v2.alos.enabled" : "true", + "ksql.query.push.v2.max.hourly.bandwidth.megabytes" : "2147483647", + "ksql.query.pull.range.scan.enabled" : "true", + "ksql.transient.query.cleanup.service.initial.delay.seconds" : "600", + "ksql.hidden.topics" : "_confluent.*,__confluent.*,_schemas,__consumer_offsets,__transaction_state,connect-configs,connect-offsets,connect-status,connect-statuses", + "ksql.lambdas.enabled" : "true", + "ksql.source.table.materialization.enabled" : "true", + "ksql.query.pull.max.hourly.bandwidth.megabytes" : "2147483647", + "ksql.sink.window.change.log.additional.retention" : "1000000", + "ksql.query.persistent.active.limit" : "2147483647", + "ksql.persistence.wrap.single.values" : null, + "ksql.query.transient.max.bytes.buffering.total" : "-1", + "ksql.connect.basic.auth.credentials.source" : "NONE", + "ksql.schema.registry.url" : "schema_registry.url:0", + "ksql.properties.overrides.denylist" : "", + "ksql.service.id" : "some.ksql.service.id", + "ksql.query.push.v2.max.catchup.consumers" : "5", + "ksql.query.push.v2.enabled" : "false", + "ksql.transient.query.cleanup.service.enable" : "true", + "ksql.query.push.v2.metrics.enabled" : "true", + "ksql.rowpartition.rowoffset.enabled" : "true", + "ksql.streams.commit.interval.ms" : "2000", + "ksql.query.pull.table.scan.enabled" : "true", + "ksql.streams.auto.commit.interval.ms" : "0", + "ksql.streams.topology.optimization" : "all", + "ksql.endpoint.migrate.query" : "true", + "ksql.query.push.v2.registry.installed" : "false", + "ksql.streams.num.stream.threads" : "4", + "ksql.metrics.tags.custom" : "", + "ksql.query.push.v2.catchup.consumer.msg.window" : "50", + "ksql.runtime.feature.shared.enabled" : "false", + "ksql.udf.collect.metrics" : "false", + "ksql.new.query.planner.enabled" : "false", + "ksql.connect.request.headers.plugin" : null, + "ksql.security.extension.class" : null, + "ksql.transient.prefix" : "transient_", + "ksql.headers.columns.enabled" : "true", + "ksql.streams.default.deserialization.exception.handler" : "io.confluent.ksql.errors.LogMetricAndContinueExceptionHandler", + "ksql.connect.request.timeout.ms" : "5000", + "ksql.query.pull.enable.standby.reads" : "false", + "ksql.persistence.default.format.key" : "KAFKA", + "ksql.query.persistent.max.bytes.buffering.total" : "-1", + "ksql.query.error.max.queue.size" : "10", + "ksql.query.cleanup.shutdown.timeout.ms" : "30000", + "ksql.internal.topic.min.insync.replicas" : "1", + "ksql.internal.topic.replicas" : "1", + "ksql.insert.into.values.enabled" : "true", + "ksql.queryanonymizer.cluster_namespace" : null, + "ksql.create.or.replace.enabled" : "true", + "ksql.shared.runtimes.count" : "2", + "ksql.cast.strings.preserve.nulls" : "true", + "ksql.authorization.cache.max.entries" : "10000", + "ksql.pull.queries.enable" : "true", + "ksql.transient.query.cleanup.service.period.seconds" : "600", + "ksql.suppress.enabled" : "false", + "ksql.readonly.topics" : "_confluent.*,__confluent.*,_schemas,__consumer_offsets,__transaction_state,connect-configs,connect-offsets,connect-status,connect-statuses", + "ksql.connect.basic.auth.credentials.file" : "", + "ksql.authorization.cache.expiry.time.secs" : "30", + "ksql.query.retry.backoff.initial.ms" : "15000", + "ksql.query.pull.max.concurrent.requests" : "2147483647", + "ksql.streams.auto.offset.reset" : "earliest", + "ksql.connect.url" : "http://localhost:8083", + "ksql.query.push.v2.new.latest.delay.ms" : "5000", + "ksql.query.push.v2.latest.reset.age.ms" : "30000", + "ksql.streams.default.production.exception.handler" : "io.confluent.ksql.errors.ProductionExceptionHandlerUtil$LogAndFailProductionExceptionHandler", + "ksql.query.pull.interpreter.enabled" : "true", + "ksql.query.pull.limit.clause.enabled" : "true", + "ksql.query.pull.router.thread.pool.size" : "50", + "ksql.query.push.v2.continuation.tokens.enabled" : "false", + "ksql.query.retry.backoff.max.ms" : "900000", + "ksql.timestamp.throw.on.invalid" : "false", + "ksql.persistence.default.format.value" : null, + "ksql.udfs.enabled" : "true", + "ksql.udf.enable.security.manager" : "true", + "ksql.connect.worker.config" : "", + "ksql.nested.error.set.null" : "true", + "ksql.query.pull.thread.pool.size" : "50", + "ksql.persistent.prefix" : "query_", + "ksql.metastore.backup.location" : "", + "ksql.error.classifier.regex" : "", + "ksql.suppress.buffer.size.bytes" : "-1" + } +} \ No newline at end of file diff --git a/ksqldb-functional-tests/src/test/resources/historical_plans/url_-_chain_a_call_to_URL_EXTRACT_PARAMETER_with_URL_DECODE_PARAM_-_PROTOBUF/7.3.0_1651070264604/spec.json b/ksqldb-functional-tests/src/test/resources/historical_plans/url_-_chain_a_call_to_URL_EXTRACT_PARAMETER_with_URL_DECODE_PARAM_-_PROTOBUF/7.3.0_1651070264604/spec.json new file mode 100644 index 000000000000..f669b4b1803e --- /dev/null +++ b/ksqldb-functional-tests/src/test/resources/historical_plans/url_-_chain_a_call_to_URL_EXTRACT_PARAMETER_with_URL_DECODE_PARAM_-_PROTOBUF/7.3.0_1651070264604/spec.json @@ -0,0 +1,116 @@ +{ + "version" : "7.3.0", + "timestamp" : 1651070264604, + "path" : "query-validation-tests/url.json", + "schemas" : { + "CSAS_OUTPUT_0.KsqlTopic.Source" : { + "schema" : "`K` STRING KEY, `URL` STRING", + "keyFormat" : { + "format" : "KAFKA" + }, + "valueFormat" : { + "format" : "PROTOBUF", + "properties" : { + "unwrapPrimitives" : "true" + } + } + }, + "CSAS_OUTPUT_0.OUTPUT" : { + "schema" : "`K` STRING KEY, `PARAM` STRING", + "keyFormat" : { + "format" : "KAFKA" + }, + "valueFormat" : { + "format" : "PROTOBUF", + "properties" : { + "unwrapPrimitives" : "true" + } + } + } + }, + "testCase" : { + "name" : "chain a call to URL_EXTRACT_PARAMETER with URL_DECODE_PARAM - PROTOBUF", + "inputs" : [ { + "topic" : "test_topic", + "key" : "1", + "value" : { + "url" : "http://www.test.com/?one=a&two=url%20encoded" + }, + "timestamp" : 0 + } ], + "outputs" : [ { + "topic" : "OUTPUT", + "key" : "1", + "value" : { + "PARAM" : "url encoded" + }, + "timestamp" : 0 + } ], + "topics" : [ { + "name" : "OUTPUT", + "replicas" : 1, + "numPartitions" : 4 + }, { + "name" : "test_topic", + "valueSchema" : "syntax = \"proto3\";\n\nmessage ConnectDefault1 {\n string URL = 1;\n}\n", + "valueFormat" : "PROTOBUF", + "replicas" : 1, + "numPartitions" : 4 + } ], + "statements" : [ "CREATE STREAM TEST (K STRING KEY, url VARCHAR) WITH (kafka_topic='test_topic', value_format='PROTOBUF');", "CREATE STREAM OUTPUT AS SELECT K, URL_DECODE_PARAM(URL_EXTRACT_PARAMETER(url,'two')) as PARAM FROM TEST;" ], + "post" : { + "sources" : [ { + "name" : "OUTPUT", + "type" : "STREAM", + "schema" : "`K` STRING KEY, `PARAM` STRING", + "keyFormat" : { + "format" : "KAFKA" + }, + "valueFormat" : "PROTOBUF", + "keyFeatures" : [ ], + "valueFeatures" : [ ], + "isSource" : false + }, { + "name" : "TEST", + "type" : "STREAM", + "schema" : "`K` STRING KEY, `URL` STRING", + "keyFormat" : { + "format" : "KAFKA" + }, + "valueFormat" : "PROTOBUF", + "keyFeatures" : [ ], + "valueFeatures" : [ ], + "isSource" : false + } ], + "topics" : { + "topics" : [ { + "name" : "test_topic", + "keyFormat" : { + "format" : "KAFKA" + }, + "valueFormat" : { + "format" : "PROTOBUF", + "properties" : { + "unwrapPrimitives" : "true" + } + }, + "partitions" : 4, + "valueSchema" : "syntax = \"proto3\";\n\nmessage ConnectDefault1 {\n string URL = 1;\n}\n" + }, { + "name" : "OUTPUT", + "keyFormat" : { + "format" : "KAFKA" + }, + "valueFormat" : { + "format" : "PROTOBUF", + "properties" : { + "unwrapPrimitives" : "true" + } + }, + "partitions" : 4, + "valueSchema" : "syntax = \"proto3\";\n\nmessage ConnectDefault1 {\n string PARAM = 1;\n}\n" + } ] + } + } + } +} \ No newline at end of file diff --git a/ksqldb-functional-tests/src/test/resources/historical_plans/url_-_chain_a_call_to_URL_EXTRACT_PARAMETER_with_URL_DECODE_PARAM_-_PROTOBUF/7.3.0_1651070264604/topology b/ksqldb-functional-tests/src/test/resources/historical_plans/url_-_chain_a_call_to_URL_EXTRACT_PARAMETER_with_URL_DECODE_PARAM_-_PROTOBUF/7.3.0_1651070264604/topology new file mode 100644 index 000000000000..441a8f282644 --- /dev/null +++ b/ksqldb-functional-tests/src/test/resources/historical_plans/url_-_chain_a_call_to_URL_EXTRACT_PARAMETER_with_URL_DECODE_PARAM_-_PROTOBUF/7.3.0_1651070264604/topology @@ -0,0 +1,13 @@ +Topologies: + Sub-topology: 0 + Source: KSTREAM-SOURCE-0000000000 (topics: [test_topic]) + --> KSTREAM-TRANSFORMVALUES-0000000001 + Processor: KSTREAM-TRANSFORMVALUES-0000000001 (stores: []) + --> Project + <-- KSTREAM-SOURCE-0000000000 + Processor: Project (stores: []) + --> KSTREAM-SINK-0000000003 + <-- KSTREAM-TRANSFORMVALUES-0000000001 + Sink: KSTREAM-SINK-0000000003 (topic: OUTPUT) + <-- Project + diff --git a/ksqldb-functional-tests/src/test/resources/historical_plans/url_-_chain_a_call_to_URL_EXTRACT_PARAMETER_with_URL_DECODE_PARAM_-_PROTOBUF_NOSR/7.3.0_1651070264636/plan.json b/ksqldb-functional-tests/src/test/resources/historical_plans/url_-_chain_a_call_to_URL_EXTRACT_PARAMETER_with_URL_DECODE_PARAM_-_PROTOBUF_NOSR/7.3.0_1651070264636/plan.json new file mode 100644 index 000000000000..9dc8ed930946 --- /dev/null +++ b/ksqldb-functional-tests/src/test/resources/historical_plans/url_-_chain_a_call_to_URL_EXTRACT_PARAMETER_with_URL_DECODE_PARAM_-_PROTOBUF_NOSR/7.3.0_1651070264636/plan.json @@ -0,0 +1,203 @@ +{ + "plan" : [ { + "@type" : "ksqlPlanV1", + "statementText" : "CREATE STREAM TEST (K STRING KEY, URL STRING) WITH (KAFKA_TOPIC='test_topic', KEY_FORMAT='KAFKA', VALUE_FORMAT='PROTOBUF_NOSR');", + "ddlCommand" : { + "@type" : "createStreamV1", + "sourceName" : "TEST", + "schema" : "`K` STRING KEY, `URL` STRING", + "timestampColumn" : null, + "topicName" : "test_topic", + "formats" : { + "keyFormat" : { + "format" : "KAFKA", + "properties" : { } + }, + "valueFormat" : { + "format" : "PROTOBUF_NOSR", + "properties" : { } + } + }, + "windowInfo" : null, + "orReplace" : false, + "isSource" : false + }, + "queryPlan" : null + }, { + "@type" : "ksqlPlanV1", + "statementText" : "CREATE STREAM OUTPUT AS SELECT\n TEST.K K,\n URL_DECODE_PARAM(URL_EXTRACT_PARAMETER(TEST.URL, 'two')) PARAM\nFROM TEST TEST\nEMIT CHANGES", + "ddlCommand" : { + "@type" : "createStreamV1", + "sourceName" : "OUTPUT", + "schema" : "`K` STRING KEY, `PARAM` STRING", + "timestampColumn" : null, + "topicName" : "OUTPUT", + "formats" : { + "keyFormat" : { + "format" : "KAFKA", + "properties" : { } + }, + "valueFormat" : { + "format" : "PROTOBUF_NOSR", + "properties" : { } + } + }, + "windowInfo" : null, + "orReplace" : false, + "isSource" : false + }, + "queryPlan" : { + "sources" : [ "TEST" ], + "sink" : "OUTPUT", + "physicalPlan" : { + "@type" : "streamSinkV1", + "properties" : { + "queryContext" : "OUTPUT" + }, + "source" : { + "@type" : "streamSelectV1", + "properties" : { + "queryContext" : "Project" + }, + "source" : { + "@type" : "streamSourceV1", + "properties" : { + "queryContext" : "KsqlTopic/Source" + }, + "topicName" : "test_topic", + "formats" : { + "keyFormat" : { + "format" : "KAFKA", + "properties" : { } + }, + "valueFormat" : { + "format" : "PROTOBUF_NOSR", + "properties" : { } + } + }, + "timestampColumn" : null, + "sourceSchema" : "`K` STRING KEY, `URL` STRING", + "pseudoColumnVersion" : 1 + }, + "keyColumnNames" : [ "K" ], + "selectedKeys" : null, + "selectExpressions" : [ "URL_DECODE_PARAM(URL_EXTRACT_PARAMETER(URL, 'two')) AS PARAM" ] + }, + "formats" : { + "keyFormat" : { + "format" : "KAFKA", + "properties" : { } + }, + "valueFormat" : { + "format" : "PROTOBUF_NOSR", + "properties" : { } + } + }, + "topicName" : "OUTPUT", + "timestampColumn" : null + }, + "queryId" : "CSAS_OUTPUT_0", + "runtimeId" : null + } + } ], + "configs" : { + "ksql.extension.dir" : "ext", + "ksql.streams.cache.max.bytes.buffering" : "0", + "metric.reporters" : "", + "ksql.query.status.running.threshold.seconds" : "300", + "ksql.connect.basic.auth.credentials.reload" : "false", + "ksql.output.topic.name.prefix" : "", + "ksql.query.pull.stream.enabled" : "true", + "ksql.query.push.v2.interpreter.enabled" : "true", + "ksql.queryanonymizer.logs_enabled" : "true", + "ksql.variable.substitution.enable" : "true", + "ksql.streams.shutdown.timeout.ms" : "300000", + "ksql.query.pull.max.allowed.offset.lag" : "9223372036854775807", + "ksql.query.pull.max.qps" : "2147483647", + "ksql.access.validator.enable" : "auto", + "ksql.streams.bootstrap.servers" : "localhost:0", + "ksql.query.pull.metrics.enabled" : "true", + "ksql.metrics.extension" : null, + "ksql.query.push.v2.alos.enabled" : "true", + "ksql.query.push.v2.max.hourly.bandwidth.megabytes" : "2147483647", + "ksql.query.pull.range.scan.enabled" : "true", + "ksql.transient.query.cleanup.service.initial.delay.seconds" : "600", + "ksql.hidden.topics" : "_confluent.*,__confluent.*,_schemas,__consumer_offsets,__transaction_state,connect-configs,connect-offsets,connect-status,connect-statuses", + "ksql.lambdas.enabled" : "true", + "ksql.source.table.materialization.enabled" : "true", + "ksql.query.pull.max.hourly.bandwidth.megabytes" : "2147483647", + "ksql.sink.window.change.log.additional.retention" : "1000000", + "ksql.query.persistent.active.limit" : "2147483647", + "ksql.persistence.wrap.single.values" : null, + "ksql.query.transient.max.bytes.buffering.total" : "-1", + "ksql.connect.basic.auth.credentials.source" : "NONE", + "ksql.schema.registry.url" : "schema_registry.url:0", + "ksql.properties.overrides.denylist" : "", + "ksql.service.id" : "some.ksql.service.id", + "ksql.query.push.v2.max.catchup.consumers" : "5", + "ksql.query.push.v2.enabled" : "false", + "ksql.transient.query.cleanup.service.enable" : "true", + "ksql.query.push.v2.metrics.enabled" : "true", + "ksql.rowpartition.rowoffset.enabled" : "true", + "ksql.streams.commit.interval.ms" : "2000", + "ksql.query.pull.table.scan.enabled" : "true", + "ksql.streams.auto.commit.interval.ms" : "0", + "ksql.streams.topology.optimization" : "all", + "ksql.endpoint.migrate.query" : "true", + "ksql.query.push.v2.registry.installed" : "false", + "ksql.streams.num.stream.threads" : "4", + "ksql.metrics.tags.custom" : "", + "ksql.query.push.v2.catchup.consumer.msg.window" : "50", + "ksql.runtime.feature.shared.enabled" : "false", + "ksql.udf.collect.metrics" : "false", + "ksql.new.query.planner.enabled" : "false", + "ksql.connect.request.headers.plugin" : null, + "ksql.security.extension.class" : null, + "ksql.transient.prefix" : "transient_", + "ksql.headers.columns.enabled" : "true", + "ksql.streams.default.deserialization.exception.handler" : "io.confluent.ksql.errors.LogMetricAndContinueExceptionHandler", + "ksql.connect.request.timeout.ms" : "5000", + "ksql.query.pull.enable.standby.reads" : "false", + "ksql.persistence.default.format.key" : "KAFKA", + "ksql.query.persistent.max.bytes.buffering.total" : "-1", + "ksql.query.error.max.queue.size" : "10", + "ksql.query.cleanup.shutdown.timeout.ms" : "30000", + "ksql.internal.topic.min.insync.replicas" : "1", + "ksql.internal.topic.replicas" : "1", + "ksql.insert.into.values.enabled" : "true", + "ksql.queryanonymizer.cluster_namespace" : null, + "ksql.create.or.replace.enabled" : "true", + "ksql.shared.runtimes.count" : "2", + "ksql.cast.strings.preserve.nulls" : "true", + "ksql.authorization.cache.max.entries" : "10000", + "ksql.pull.queries.enable" : "true", + "ksql.transient.query.cleanup.service.period.seconds" : "600", + "ksql.suppress.enabled" : "false", + "ksql.readonly.topics" : "_confluent.*,__confluent.*,_schemas,__consumer_offsets,__transaction_state,connect-configs,connect-offsets,connect-status,connect-statuses", + "ksql.connect.basic.auth.credentials.file" : "", + "ksql.authorization.cache.expiry.time.secs" : "30", + "ksql.query.retry.backoff.initial.ms" : "15000", + "ksql.query.pull.max.concurrent.requests" : "2147483647", + "ksql.streams.auto.offset.reset" : "earliest", + "ksql.connect.url" : "http://localhost:8083", + "ksql.query.push.v2.new.latest.delay.ms" : "5000", + "ksql.query.push.v2.latest.reset.age.ms" : "30000", + "ksql.streams.default.production.exception.handler" : "io.confluent.ksql.errors.ProductionExceptionHandlerUtil$LogAndFailProductionExceptionHandler", + "ksql.query.pull.interpreter.enabled" : "true", + "ksql.query.pull.limit.clause.enabled" : "true", + "ksql.query.pull.router.thread.pool.size" : "50", + "ksql.query.push.v2.continuation.tokens.enabled" : "false", + "ksql.query.retry.backoff.max.ms" : "900000", + "ksql.timestamp.throw.on.invalid" : "false", + "ksql.persistence.default.format.value" : null, + "ksql.udfs.enabled" : "true", + "ksql.udf.enable.security.manager" : "true", + "ksql.connect.worker.config" : "", + "ksql.nested.error.set.null" : "true", + "ksql.query.pull.thread.pool.size" : "50", + "ksql.persistent.prefix" : "query_", + "ksql.metastore.backup.location" : "", + "ksql.error.classifier.regex" : "", + "ksql.suppress.buffer.size.bytes" : "-1" + } +} \ No newline at end of file diff --git a/ksqldb-functional-tests/src/test/resources/historical_plans/url_-_chain_a_call_to_URL_EXTRACT_PARAMETER_with_URL_DECODE_PARAM_-_PROTOBUF_NOSR/7.3.0_1651070264636/spec.json b/ksqldb-functional-tests/src/test/resources/historical_plans/url_-_chain_a_call_to_URL_EXTRACT_PARAMETER_with_URL_DECODE_PARAM_-_PROTOBUF_NOSR/7.3.0_1651070264636/spec.json new file mode 100644 index 000000000000..8ad7e2605b3d --- /dev/null +++ b/ksqldb-functional-tests/src/test/resources/historical_plans/url_-_chain_a_call_to_URL_EXTRACT_PARAMETER_with_URL_DECODE_PARAM_-_PROTOBUF_NOSR/7.3.0_1651070264636/spec.json @@ -0,0 +1,100 @@ +{ + "version" : "7.3.0", + "timestamp" : 1651070264636, + "path" : "query-validation-tests/url.json", + "schemas" : { + "CSAS_OUTPUT_0.KsqlTopic.Source" : { + "schema" : "`K` STRING KEY, `URL` STRING", + "keyFormat" : { + "format" : "KAFKA" + }, + "valueFormat" : { + "format" : "PROTOBUF_NOSR" + } + }, + "CSAS_OUTPUT_0.OUTPUT" : { + "schema" : "`K` STRING KEY, `PARAM` STRING", + "keyFormat" : { + "format" : "KAFKA" + }, + "valueFormat" : { + "format" : "PROTOBUF_NOSR" + } + } + }, + "testCase" : { + "name" : "chain a call to URL_EXTRACT_PARAMETER with URL_DECODE_PARAM - PROTOBUF_NOSR", + "inputs" : [ { + "topic" : "test_topic", + "key" : "1", + "value" : { + "url" : "http://www.test.com/?one=a&two=url%20encoded" + }, + "timestamp" : 0 + } ], + "outputs" : [ { + "topic" : "OUTPUT", + "key" : "1", + "value" : { + "PARAM" : "url encoded" + }, + "timestamp" : 0 + } ], + "topics" : [ { + "name" : "OUTPUT", + "replicas" : 1, + "numPartitions" : 4 + }, { + "name" : "test_topic", + "replicas" : 1, + "numPartitions" : 4 + } ], + "statements" : [ "CREATE STREAM TEST (K STRING KEY, url VARCHAR) WITH (kafka_topic='test_topic', value_format='PROTOBUF_NOSR');", "CREATE STREAM OUTPUT AS SELECT K, URL_DECODE_PARAM(URL_EXTRACT_PARAMETER(url,'two')) as PARAM FROM TEST;" ], + "post" : { + "sources" : [ { + "name" : "OUTPUT", + "type" : "STREAM", + "schema" : "`K` STRING KEY, `PARAM` STRING", + "keyFormat" : { + "format" : "KAFKA" + }, + "valueFormat" : "PROTOBUF_NOSR", + "keyFeatures" : [ ], + "valueFeatures" : [ ], + "isSource" : false + }, { + "name" : "TEST", + "type" : "STREAM", + "schema" : "`K` STRING KEY, `URL` STRING", + "keyFormat" : { + "format" : "KAFKA" + }, + "valueFormat" : "PROTOBUF_NOSR", + "keyFeatures" : [ ], + "valueFeatures" : [ ], + "isSource" : false + } ], + "topics" : { + "topics" : [ { + "name" : "test_topic", + "keyFormat" : { + "format" : "KAFKA" + }, + "valueFormat" : { + "format" : "PROTOBUF_NOSR" + }, + "partitions" : 4 + }, { + "name" : "OUTPUT", + "keyFormat" : { + "format" : "KAFKA" + }, + "valueFormat" : { + "format" : "PROTOBUF_NOSR" + }, + "partitions" : 4 + } ] + } + } + } +} \ No newline at end of file diff --git a/ksqldb-functional-tests/src/test/resources/historical_plans/url_-_chain_a_call_to_URL_EXTRACT_PARAMETER_with_URL_DECODE_PARAM_-_PROTOBUF_NOSR/7.3.0_1651070264636/topology b/ksqldb-functional-tests/src/test/resources/historical_plans/url_-_chain_a_call_to_URL_EXTRACT_PARAMETER_with_URL_DECODE_PARAM_-_PROTOBUF_NOSR/7.3.0_1651070264636/topology new file mode 100644 index 000000000000..441a8f282644 --- /dev/null +++ b/ksqldb-functional-tests/src/test/resources/historical_plans/url_-_chain_a_call_to_URL_EXTRACT_PARAMETER_with_URL_DECODE_PARAM_-_PROTOBUF_NOSR/7.3.0_1651070264636/topology @@ -0,0 +1,13 @@ +Topologies: + Sub-topology: 0 + Source: KSTREAM-SOURCE-0000000000 (topics: [test_topic]) + --> KSTREAM-TRANSFORMVALUES-0000000001 + Processor: KSTREAM-TRANSFORMVALUES-0000000001 (stores: []) + --> Project + <-- KSTREAM-SOURCE-0000000000 + Processor: Project (stores: []) + --> KSTREAM-SINK-0000000003 + <-- KSTREAM-TRANSFORMVALUES-0000000001 + Sink: KSTREAM-SINK-0000000003 (topic: OUTPUT) + <-- Project + diff --git a/ksqldb-functional-tests/src/test/resources/historical_plans/url_-_decode_a_url_parameter_using_DECODE_URL_PARAM_-_PROTOBUF/7.3.0_1651070263696/plan.json b/ksqldb-functional-tests/src/test/resources/historical_plans/url_-_decode_a_url_parameter_using_DECODE_URL_PARAM_-_PROTOBUF/7.3.0_1651070263696/plan.json new file mode 100644 index 000000000000..0365747c68d1 --- /dev/null +++ b/ksqldb-functional-tests/src/test/resources/historical_plans/url_-_decode_a_url_parameter_using_DECODE_URL_PARAM_-_PROTOBUF/7.3.0_1651070263696/plan.json @@ -0,0 +1,211 @@ +{ + "plan" : [ { + "@type" : "ksqlPlanV1", + "statementText" : "CREATE STREAM TEST (K STRING KEY, URL STRING) WITH (KAFKA_TOPIC='test_topic', KEY_FORMAT='KAFKA', VALUE_FORMAT='PROTOBUF');", + "ddlCommand" : { + "@type" : "createStreamV1", + "sourceName" : "TEST", + "schema" : "`K` STRING KEY, `URL` STRING", + "timestampColumn" : null, + "topicName" : "test_topic", + "formats" : { + "keyFormat" : { + "format" : "KAFKA", + "properties" : { } + }, + "valueFormat" : { + "format" : "PROTOBUF", + "properties" : { + "unwrapPrimitives" : "true" + } + } + }, + "windowInfo" : null, + "orReplace" : false, + "isSource" : false + }, + "queryPlan" : null + }, { + "@type" : "ksqlPlanV1", + "statementText" : "CREATE STREAM OUTPUT AS SELECT\n TEST.K K,\n URL_DECODE_PARAM(TEST.URL) DECODED\nFROM TEST TEST\nEMIT CHANGES", + "ddlCommand" : { + "@type" : "createStreamV1", + "sourceName" : "OUTPUT", + "schema" : "`K` STRING KEY, `DECODED` STRING", + "timestampColumn" : null, + "topicName" : "OUTPUT", + "formats" : { + "keyFormat" : { + "format" : "KAFKA", + "properties" : { } + }, + "valueFormat" : { + "format" : "PROTOBUF", + "properties" : { + "unwrapPrimitives" : "true" + } + } + }, + "windowInfo" : null, + "orReplace" : false, + "isSource" : false + }, + "queryPlan" : { + "sources" : [ "TEST" ], + "sink" : "OUTPUT", + "physicalPlan" : { + "@type" : "streamSinkV1", + "properties" : { + "queryContext" : "OUTPUT" + }, + "source" : { + "@type" : "streamSelectV1", + "properties" : { + "queryContext" : "Project" + }, + "source" : { + "@type" : "streamSourceV1", + "properties" : { + "queryContext" : "KsqlTopic/Source" + }, + "topicName" : "test_topic", + "formats" : { + "keyFormat" : { + "format" : "KAFKA", + "properties" : { } + }, + "valueFormat" : { + "format" : "PROTOBUF", + "properties" : { + "unwrapPrimitives" : "true" + } + } + }, + "timestampColumn" : null, + "sourceSchema" : "`K` STRING KEY, `URL` STRING", + "pseudoColumnVersion" : 1 + }, + "keyColumnNames" : [ "K" ], + "selectedKeys" : null, + "selectExpressions" : [ "URL_DECODE_PARAM(URL) AS DECODED" ] + }, + "formats" : { + "keyFormat" : { + "format" : "KAFKA", + "properties" : { } + }, + "valueFormat" : { + "format" : "PROTOBUF", + "properties" : { + "unwrapPrimitives" : "true" + } + } + }, + "topicName" : "OUTPUT", + "timestampColumn" : null + }, + "queryId" : "CSAS_OUTPUT_0", + "runtimeId" : null + } + } ], + "configs" : { + "ksql.extension.dir" : "ext", + "ksql.streams.cache.max.bytes.buffering" : "0", + "metric.reporters" : "", + "ksql.query.status.running.threshold.seconds" : "300", + "ksql.connect.basic.auth.credentials.reload" : "false", + "ksql.output.topic.name.prefix" : "", + "ksql.query.pull.stream.enabled" : "true", + "ksql.query.push.v2.interpreter.enabled" : "true", + "ksql.queryanonymizer.logs_enabled" : "true", + "ksql.variable.substitution.enable" : "true", + "ksql.streams.shutdown.timeout.ms" : "300000", + "ksql.query.pull.max.allowed.offset.lag" : "9223372036854775807", + "ksql.query.pull.max.qps" : "2147483647", + "ksql.access.validator.enable" : "auto", + "ksql.streams.bootstrap.servers" : "localhost:0", + "ksql.query.pull.metrics.enabled" : "true", + "ksql.metrics.extension" : null, + "ksql.query.push.v2.alos.enabled" : "true", + "ksql.query.push.v2.max.hourly.bandwidth.megabytes" : "2147483647", + "ksql.query.pull.range.scan.enabled" : "true", + "ksql.transient.query.cleanup.service.initial.delay.seconds" : "600", + "ksql.hidden.topics" : "_confluent.*,__confluent.*,_schemas,__consumer_offsets,__transaction_state,connect-configs,connect-offsets,connect-status,connect-statuses", + "ksql.lambdas.enabled" : "true", + "ksql.source.table.materialization.enabled" : "true", + "ksql.query.pull.max.hourly.bandwidth.megabytes" : "2147483647", + "ksql.sink.window.change.log.additional.retention" : "1000000", + "ksql.query.persistent.active.limit" : "2147483647", + "ksql.persistence.wrap.single.values" : null, + "ksql.query.transient.max.bytes.buffering.total" : "-1", + "ksql.connect.basic.auth.credentials.source" : "NONE", + "ksql.schema.registry.url" : "schema_registry.url:0", + "ksql.properties.overrides.denylist" : "", + "ksql.service.id" : "some.ksql.service.id", + "ksql.query.push.v2.max.catchup.consumers" : "5", + "ksql.query.push.v2.enabled" : "false", + "ksql.transient.query.cleanup.service.enable" : "true", + "ksql.query.push.v2.metrics.enabled" : "true", + "ksql.rowpartition.rowoffset.enabled" : "true", + "ksql.streams.commit.interval.ms" : "2000", + "ksql.query.pull.table.scan.enabled" : "true", + "ksql.streams.auto.commit.interval.ms" : "0", + "ksql.streams.topology.optimization" : "all", + "ksql.endpoint.migrate.query" : "true", + "ksql.query.push.v2.registry.installed" : "false", + "ksql.streams.num.stream.threads" : "4", + "ksql.metrics.tags.custom" : "", + "ksql.query.push.v2.catchup.consumer.msg.window" : "50", + "ksql.runtime.feature.shared.enabled" : "false", + "ksql.udf.collect.metrics" : "false", + "ksql.new.query.planner.enabled" : "false", + "ksql.connect.request.headers.plugin" : null, + "ksql.security.extension.class" : null, + "ksql.transient.prefix" : "transient_", + "ksql.headers.columns.enabled" : "true", + "ksql.streams.default.deserialization.exception.handler" : "io.confluent.ksql.errors.LogMetricAndContinueExceptionHandler", + "ksql.connect.request.timeout.ms" : "5000", + "ksql.query.pull.enable.standby.reads" : "false", + "ksql.persistence.default.format.key" : "KAFKA", + "ksql.query.persistent.max.bytes.buffering.total" : "-1", + "ksql.query.error.max.queue.size" : "10", + "ksql.query.cleanup.shutdown.timeout.ms" : "30000", + "ksql.internal.topic.min.insync.replicas" : "1", + "ksql.internal.topic.replicas" : "1", + "ksql.insert.into.values.enabled" : "true", + "ksql.queryanonymizer.cluster_namespace" : null, + "ksql.create.or.replace.enabled" : "true", + "ksql.shared.runtimes.count" : "2", + "ksql.cast.strings.preserve.nulls" : "true", + "ksql.authorization.cache.max.entries" : "10000", + "ksql.pull.queries.enable" : "true", + "ksql.transient.query.cleanup.service.period.seconds" : "600", + "ksql.suppress.enabled" : "false", + "ksql.readonly.topics" : "_confluent.*,__confluent.*,_schemas,__consumer_offsets,__transaction_state,connect-configs,connect-offsets,connect-status,connect-statuses", + "ksql.connect.basic.auth.credentials.file" : "", + "ksql.authorization.cache.expiry.time.secs" : "30", + "ksql.query.retry.backoff.initial.ms" : "15000", + "ksql.query.pull.max.concurrent.requests" : "2147483647", + "ksql.streams.auto.offset.reset" : "earliest", + "ksql.connect.url" : "http://localhost:8083", + "ksql.query.push.v2.new.latest.delay.ms" : "5000", + "ksql.query.push.v2.latest.reset.age.ms" : "30000", + "ksql.streams.default.production.exception.handler" : "io.confluent.ksql.errors.ProductionExceptionHandlerUtil$LogAndFailProductionExceptionHandler", + "ksql.query.pull.interpreter.enabled" : "true", + "ksql.query.pull.limit.clause.enabled" : "true", + "ksql.query.pull.router.thread.pool.size" : "50", + "ksql.query.push.v2.continuation.tokens.enabled" : "false", + "ksql.query.retry.backoff.max.ms" : "900000", + "ksql.timestamp.throw.on.invalid" : "false", + "ksql.persistence.default.format.value" : null, + "ksql.udfs.enabled" : "true", + "ksql.udf.enable.security.manager" : "true", + "ksql.connect.worker.config" : "", + "ksql.nested.error.set.null" : "true", + "ksql.query.pull.thread.pool.size" : "50", + "ksql.persistent.prefix" : "query_", + "ksql.metastore.backup.location" : "", + "ksql.error.classifier.regex" : "", + "ksql.suppress.buffer.size.bytes" : "-1" + } +} \ No newline at end of file diff --git a/ksqldb-functional-tests/src/test/resources/historical_plans/url_-_decode_a_url_parameter_using_DECODE_URL_PARAM_-_PROTOBUF/7.3.0_1651070263696/spec.json b/ksqldb-functional-tests/src/test/resources/historical_plans/url_-_decode_a_url_parameter_using_DECODE_URL_PARAM_-_PROTOBUF/7.3.0_1651070263696/spec.json new file mode 100644 index 000000000000..96dd9187ee43 --- /dev/null +++ b/ksqldb-functional-tests/src/test/resources/historical_plans/url_-_decode_a_url_parameter_using_DECODE_URL_PARAM_-_PROTOBUF/7.3.0_1651070263696/spec.json @@ -0,0 +1,144 @@ +{ + "version" : "7.3.0", + "timestamp" : 1651070263696, + "path" : "query-validation-tests/url.json", + "schemas" : { + "CSAS_OUTPUT_0.KsqlTopic.Source" : { + "schema" : "`K` STRING KEY, `URL` STRING", + "keyFormat" : { + "format" : "KAFKA" + }, + "valueFormat" : { + "format" : "PROTOBUF", + "properties" : { + "unwrapPrimitives" : "true" + } + } + }, + "CSAS_OUTPUT_0.OUTPUT" : { + "schema" : "`K` STRING KEY, `DECODED` STRING", + "keyFormat" : { + "format" : "KAFKA" + }, + "valueFormat" : { + "format" : "PROTOBUF", + "properties" : { + "unwrapPrimitives" : "true" + } + } + } + }, + "testCase" : { + "name" : "decode a url parameter using DECODE_URL_PARAM - PROTOBUF", + "inputs" : [ { + "topic" : "test_topic", + "key" : "1", + "value" : { + "url" : "%3Ffoo+%24bar" + }, + "timestamp" : 0 + }, { + "topic" : "test_topic", + "key" : "2", + "value" : { + "url" : "hello%26world" + }, + "timestamp" : 1 + }, { + "topic" : "test_topic", + "key" : "3", + "value" : { + "url" : "nothing" + }, + "timestamp" : 2 + } ], + "outputs" : [ { + "topic" : "OUTPUT", + "key" : "1", + "value" : { + "DECODED" : "?foo $bar" + }, + "timestamp" : 0 + }, { + "topic" : "OUTPUT", + "key" : "2", + "value" : { + "DECODED" : "hello&world" + }, + "timestamp" : 1 + }, { + "topic" : "OUTPUT", + "key" : "3", + "value" : { + "DECODED" : "nothing" + }, + "timestamp" : 2 + } ], + "topics" : [ { + "name" : "OUTPUT", + "replicas" : 1, + "numPartitions" : 4 + }, { + "name" : "test_topic", + "valueSchema" : "syntax = \"proto3\";\n\nmessage ConnectDefault1 {\n string URL = 1;\n}\n", + "valueFormat" : "PROTOBUF", + "replicas" : 1, + "numPartitions" : 4 + } ], + "statements" : [ "CREATE STREAM TEST (K STRING KEY, url VARCHAR) WITH (kafka_topic='test_topic', value_format='PROTOBUF');", "CREATE STREAM OUTPUT AS SELECT K, URL_DECODE_PARAM(url) as DECODED FROM TEST;" ], + "post" : { + "sources" : [ { + "name" : "OUTPUT", + "type" : "STREAM", + "schema" : "`K` STRING KEY, `DECODED` STRING", + "keyFormat" : { + "format" : "KAFKA" + }, + "valueFormat" : "PROTOBUF", + "keyFeatures" : [ ], + "valueFeatures" : [ ], + "isSource" : false + }, { + "name" : "TEST", + "type" : "STREAM", + "schema" : "`K` STRING KEY, `URL` STRING", + "keyFormat" : { + "format" : "KAFKA" + }, + "valueFormat" : "PROTOBUF", + "keyFeatures" : [ ], + "valueFeatures" : [ ], + "isSource" : false + } ], + "topics" : { + "topics" : [ { + "name" : "test_topic", + "keyFormat" : { + "format" : "KAFKA" + }, + "valueFormat" : { + "format" : "PROTOBUF", + "properties" : { + "unwrapPrimitives" : "true" + } + }, + "partitions" : 4, + "valueSchema" : "syntax = \"proto3\";\n\nmessage ConnectDefault1 {\n string URL = 1;\n}\n" + }, { + "name" : "OUTPUT", + "keyFormat" : { + "format" : "KAFKA" + }, + "valueFormat" : { + "format" : "PROTOBUF", + "properties" : { + "unwrapPrimitives" : "true" + } + }, + "partitions" : 4, + "valueSchema" : "syntax = \"proto3\";\n\nmessage ConnectDefault1 {\n string DECODED = 1;\n}\n" + } ] + } + } + } +} \ No newline at end of file diff --git a/ksqldb-functional-tests/src/test/resources/historical_plans/url_-_decode_a_url_parameter_using_DECODE_URL_PARAM_-_PROTOBUF/7.3.0_1651070263696/topology b/ksqldb-functional-tests/src/test/resources/historical_plans/url_-_decode_a_url_parameter_using_DECODE_URL_PARAM_-_PROTOBUF/7.3.0_1651070263696/topology new file mode 100644 index 000000000000..441a8f282644 --- /dev/null +++ b/ksqldb-functional-tests/src/test/resources/historical_plans/url_-_decode_a_url_parameter_using_DECODE_URL_PARAM_-_PROTOBUF/7.3.0_1651070263696/topology @@ -0,0 +1,13 @@ +Topologies: + Sub-topology: 0 + Source: KSTREAM-SOURCE-0000000000 (topics: [test_topic]) + --> KSTREAM-TRANSFORMVALUES-0000000001 + Processor: KSTREAM-TRANSFORMVALUES-0000000001 (stores: []) + --> Project + <-- KSTREAM-SOURCE-0000000000 + Processor: Project (stores: []) + --> KSTREAM-SINK-0000000003 + <-- KSTREAM-TRANSFORMVALUES-0000000001 + Sink: KSTREAM-SINK-0000000003 (topic: OUTPUT) + <-- Project + diff --git a/ksqldb-functional-tests/src/test/resources/historical_plans/url_-_decode_a_url_parameter_using_DECODE_URL_PARAM_-_PROTOBUF_NOSR/7.3.0_1651070263719/plan.json b/ksqldb-functional-tests/src/test/resources/historical_plans/url_-_decode_a_url_parameter_using_DECODE_URL_PARAM_-_PROTOBUF_NOSR/7.3.0_1651070263719/plan.json new file mode 100644 index 000000000000..6694546e0021 --- /dev/null +++ b/ksqldb-functional-tests/src/test/resources/historical_plans/url_-_decode_a_url_parameter_using_DECODE_URL_PARAM_-_PROTOBUF_NOSR/7.3.0_1651070263719/plan.json @@ -0,0 +1,203 @@ +{ + "plan" : [ { + "@type" : "ksqlPlanV1", + "statementText" : "CREATE STREAM TEST (K STRING KEY, URL STRING) WITH (KAFKA_TOPIC='test_topic', KEY_FORMAT='KAFKA', VALUE_FORMAT='PROTOBUF_NOSR');", + "ddlCommand" : { + "@type" : "createStreamV1", + "sourceName" : "TEST", + "schema" : "`K` STRING KEY, `URL` STRING", + "timestampColumn" : null, + "topicName" : "test_topic", + "formats" : { + "keyFormat" : { + "format" : "KAFKA", + "properties" : { } + }, + "valueFormat" : { + "format" : "PROTOBUF_NOSR", + "properties" : { } + } + }, + "windowInfo" : null, + "orReplace" : false, + "isSource" : false + }, + "queryPlan" : null + }, { + "@type" : "ksqlPlanV1", + "statementText" : "CREATE STREAM OUTPUT AS SELECT\n TEST.K K,\n URL_DECODE_PARAM(TEST.URL) DECODED\nFROM TEST TEST\nEMIT CHANGES", + "ddlCommand" : { + "@type" : "createStreamV1", + "sourceName" : "OUTPUT", + "schema" : "`K` STRING KEY, `DECODED` STRING", + "timestampColumn" : null, + "topicName" : "OUTPUT", + "formats" : { + "keyFormat" : { + "format" : "KAFKA", + "properties" : { } + }, + "valueFormat" : { + "format" : "PROTOBUF_NOSR", + "properties" : { } + } + }, + "windowInfo" : null, + "orReplace" : false, + "isSource" : false + }, + "queryPlan" : { + "sources" : [ "TEST" ], + "sink" : "OUTPUT", + "physicalPlan" : { + "@type" : "streamSinkV1", + "properties" : { + "queryContext" : "OUTPUT" + }, + "source" : { + "@type" : "streamSelectV1", + "properties" : { + "queryContext" : "Project" + }, + "source" : { + "@type" : "streamSourceV1", + "properties" : { + "queryContext" : "KsqlTopic/Source" + }, + "topicName" : "test_topic", + "formats" : { + "keyFormat" : { + "format" : "KAFKA", + "properties" : { } + }, + "valueFormat" : { + "format" : "PROTOBUF_NOSR", + "properties" : { } + } + }, + "timestampColumn" : null, + "sourceSchema" : "`K` STRING KEY, `URL` STRING", + "pseudoColumnVersion" : 1 + }, + "keyColumnNames" : [ "K" ], + "selectedKeys" : null, + "selectExpressions" : [ "URL_DECODE_PARAM(URL) AS DECODED" ] + }, + "formats" : { + "keyFormat" : { + "format" : "KAFKA", + "properties" : { } + }, + "valueFormat" : { + "format" : "PROTOBUF_NOSR", + "properties" : { } + } + }, + "topicName" : "OUTPUT", + "timestampColumn" : null + }, + "queryId" : "CSAS_OUTPUT_0", + "runtimeId" : null + } + } ], + "configs" : { + "ksql.extension.dir" : "ext", + "ksql.streams.cache.max.bytes.buffering" : "0", + "metric.reporters" : "", + "ksql.query.status.running.threshold.seconds" : "300", + "ksql.connect.basic.auth.credentials.reload" : "false", + "ksql.output.topic.name.prefix" : "", + "ksql.query.pull.stream.enabled" : "true", + "ksql.query.push.v2.interpreter.enabled" : "true", + "ksql.queryanonymizer.logs_enabled" : "true", + "ksql.variable.substitution.enable" : "true", + "ksql.streams.shutdown.timeout.ms" : "300000", + "ksql.query.pull.max.allowed.offset.lag" : "9223372036854775807", + "ksql.query.pull.max.qps" : "2147483647", + "ksql.access.validator.enable" : "auto", + "ksql.streams.bootstrap.servers" : "localhost:0", + "ksql.query.pull.metrics.enabled" : "true", + "ksql.metrics.extension" : null, + "ksql.query.push.v2.alos.enabled" : "true", + "ksql.query.push.v2.max.hourly.bandwidth.megabytes" : "2147483647", + "ksql.query.pull.range.scan.enabled" : "true", + "ksql.transient.query.cleanup.service.initial.delay.seconds" : "600", + "ksql.hidden.topics" : "_confluent.*,__confluent.*,_schemas,__consumer_offsets,__transaction_state,connect-configs,connect-offsets,connect-status,connect-statuses", + "ksql.lambdas.enabled" : "true", + "ksql.source.table.materialization.enabled" : "true", + "ksql.query.pull.max.hourly.bandwidth.megabytes" : "2147483647", + "ksql.sink.window.change.log.additional.retention" : "1000000", + "ksql.query.persistent.active.limit" : "2147483647", + "ksql.persistence.wrap.single.values" : null, + "ksql.query.transient.max.bytes.buffering.total" : "-1", + "ksql.connect.basic.auth.credentials.source" : "NONE", + "ksql.schema.registry.url" : "schema_registry.url:0", + "ksql.properties.overrides.denylist" : "", + "ksql.service.id" : "some.ksql.service.id", + "ksql.query.push.v2.max.catchup.consumers" : "5", + "ksql.query.push.v2.enabled" : "false", + "ksql.transient.query.cleanup.service.enable" : "true", + "ksql.query.push.v2.metrics.enabled" : "true", + "ksql.rowpartition.rowoffset.enabled" : "true", + "ksql.streams.commit.interval.ms" : "2000", + "ksql.query.pull.table.scan.enabled" : "true", + "ksql.streams.auto.commit.interval.ms" : "0", + "ksql.streams.topology.optimization" : "all", + "ksql.endpoint.migrate.query" : "true", + "ksql.query.push.v2.registry.installed" : "false", + "ksql.streams.num.stream.threads" : "4", + "ksql.metrics.tags.custom" : "", + "ksql.query.push.v2.catchup.consumer.msg.window" : "50", + "ksql.runtime.feature.shared.enabled" : "false", + "ksql.udf.collect.metrics" : "false", + "ksql.new.query.planner.enabled" : "false", + "ksql.connect.request.headers.plugin" : null, + "ksql.security.extension.class" : null, + "ksql.transient.prefix" : "transient_", + "ksql.headers.columns.enabled" : "true", + "ksql.streams.default.deserialization.exception.handler" : "io.confluent.ksql.errors.LogMetricAndContinueExceptionHandler", + "ksql.connect.request.timeout.ms" : "5000", + "ksql.query.pull.enable.standby.reads" : "false", + "ksql.persistence.default.format.key" : "KAFKA", + "ksql.query.persistent.max.bytes.buffering.total" : "-1", + "ksql.query.error.max.queue.size" : "10", + "ksql.query.cleanup.shutdown.timeout.ms" : "30000", + "ksql.internal.topic.min.insync.replicas" : "1", + "ksql.internal.topic.replicas" : "1", + "ksql.insert.into.values.enabled" : "true", + "ksql.queryanonymizer.cluster_namespace" : null, + "ksql.create.or.replace.enabled" : "true", + "ksql.shared.runtimes.count" : "2", + "ksql.cast.strings.preserve.nulls" : "true", + "ksql.authorization.cache.max.entries" : "10000", + "ksql.pull.queries.enable" : "true", + "ksql.transient.query.cleanup.service.period.seconds" : "600", + "ksql.suppress.enabled" : "false", + "ksql.readonly.topics" : "_confluent.*,__confluent.*,_schemas,__consumer_offsets,__transaction_state,connect-configs,connect-offsets,connect-status,connect-statuses", + "ksql.connect.basic.auth.credentials.file" : "", + "ksql.authorization.cache.expiry.time.secs" : "30", + "ksql.query.retry.backoff.initial.ms" : "15000", + "ksql.query.pull.max.concurrent.requests" : "2147483647", + "ksql.streams.auto.offset.reset" : "earliest", + "ksql.connect.url" : "http://localhost:8083", + "ksql.query.push.v2.new.latest.delay.ms" : "5000", + "ksql.query.push.v2.latest.reset.age.ms" : "30000", + "ksql.streams.default.production.exception.handler" : "io.confluent.ksql.errors.ProductionExceptionHandlerUtil$LogAndFailProductionExceptionHandler", + "ksql.query.pull.interpreter.enabled" : "true", + "ksql.query.pull.limit.clause.enabled" : "true", + "ksql.query.pull.router.thread.pool.size" : "50", + "ksql.query.push.v2.continuation.tokens.enabled" : "false", + "ksql.query.retry.backoff.max.ms" : "900000", + "ksql.timestamp.throw.on.invalid" : "false", + "ksql.persistence.default.format.value" : null, + "ksql.udfs.enabled" : "true", + "ksql.udf.enable.security.manager" : "true", + "ksql.connect.worker.config" : "", + "ksql.nested.error.set.null" : "true", + "ksql.query.pull.thread.pool.size" : "50", + "ksql.persistent.prefix" : "query_", + "ksql.metastore.backup.location" : "", + "ksql.error.classifier.regex" : "", + "ksql.suppress.buffer.size.bytes" : "-1" + } +} \ No newline at end of file diff --git a/ksqldb-functional-tests/src/test/resources/historical_plans/url_-_decode_a_url_parameter_using_DECODE_URL_PARAM_-_PROTOBUF_NOSR/7.3.0_1651070263719/spec.json b/ksqldb-functional-tests/src/test/resources/historical_plans/url_-_decode_a_url_parameter_using_DECODE_URL_PARAM_-_PROTOBUF_NOSR/7.3.0_1651070263719/spec.json new file mode 100644 index 000000000000..67f9230e1c91 --- /dev/null +++ b/ksqldb-functional-tests/src/test/resources/historical_plans/url_-_decode_a_url_parameter_using_DECODE_URL_PARAM_-_PROTOBUF_NOSR/7.3.0_1651070263719/spec.json @@ -0,0 +1,128 @@ +{ + "version" : "7.3.0", + "timestamp" : 1651070263719, + "path" : "query-validation-tests/url.json", + "schemas" : { + "CSAS_OUTPUT_0.KsqlTopic.Source" : { + "schema" : "`K` STRING KEY, `URL` STRING", + "keyFormat" : { + "format" : "KAFKA" + }, + "valueFormat" : { + "format" : "PROTOBUF_NOSR" + } + }, + "CSAS_OUTPUT_0.OUTPUT" : { + "schema" : "`K` STRING KEY, `DECODED` STRING", + "keyFormat" : { + "format" : "KAFKA" + }, + "valueFormat" : { + "format" : "PROTOBUF_NOSR" + } + } + }, + "testCase" : { + "name" : "decode a url parameter using DECODE_URL_PARAM - PROTOBUF_NOSR", + "inputs" : [ { + "topic" : "test_topic", + "key" : "1", + "value" : { + "url" : "%3Ffoo+%24bar" + }, + "timestamp" : 0 + }, { + "topic" : "test_topic", + "key" : "2", + "value" : { + "url" : "hello%26world" + }, + "timestamp" : 1 + }, { + "topic" : "test_topic", + "key" : "3", + "value" : { + "url" : "nothing" + }, + "timestamp" : 2 + } ], + "outputs" : [ { + "topic" : "OUTPUT", + "key" : "1", + "value" : { + "DECODED" : "?foo $bar" + }, + "timestamp" : 0 + }, { + "topic" : "OUTPUT", + "key" : "2", + "value" : { + "DECODED" : "hello&world" + }, + "timestamp" : 1 + }, { + "topic" : "OUTPUT", + "key" : "3", + "value" : { + "DECODED" : "nothing" + }, + "timestamp" : 2 + } ], + "topics" : [ { + "name" : "OUTPUT", + "replicas" : 1, + "numPartitions" : 4 + }, { + "name" : "test_topic", + "replicas" : 1, + "numPartitions" : 4 + } ], + "statements" : [ "CREATE STREAM TEST (K STRING KEY, url VARCHAR) WITH (kafka_topic='test_topic', value_format='PROTOBUF_NOSR');", "CREATE STREAM OUTPUT AS SELECT K, URL_DECODE_PARAM(url) as DECODED FROM TEST;" ], + "post" : { + "sources" : [ { + "name" : "OUTPUT", + "type" : "STREAM", + "schema" : "`K` STRING KEY, `DECODED` STRING", + "keyFormat" : { + "format" : "KAFKA" + }, + "valueFormat" : "PROTOBUF_NOSR", + "keyFeatures" : [ ], + "valueFeatures" : [ ], + "isSource" : false + }, { + "name" : "TEST", + "type" : "STREAM", + "schema" : "`K` STRING KEY, `URL` STRING", + "keyFormat" : { + "format" : "KAFKA" + }, + "valueFormat" : "PROTOBUF_NOSR", + "keyFeatures" : [ ], + "valueFeatures" : [ ], + "isSource" : false + } ], + "topics" : { + "topics" : [ { + "name" : "test_topic", + "keyFormat" : { + "format" : "KAFKA" + }, + "valueFormat" : { + "format" : "PROTOBUF_NOSR" + }, + "partitions" : 4 + }, { + "name" : "OUTPUT", + "keyFormat" : { + "format" : "KAFKA" + }, + "valueFormat" : { + "format" : "PROTOBUF_NOSR" + }, + "partitions" : 4 + } ] + } + } + } +} \ No newline at end of file diff --git a/ksqldb-functional-tests/src/test/resources/historical_plans/url_-_decode_a_url_parameter_using_DECODE_URL_PARAM_-_PROTOBUF_NOSR/7.3.0_1651070263719/topology b/ksqldb-functional-tests/src/test/resources/historical_plans/url_-_decode_a_url_parameter_using_DECODE_URL_PARAM_-_PROTOBUF_NOSR/7.3.0_1651070263719/topology new file mode 100644 index 000000000000..441a8f282644 --- /dev/null +++ b/ksqldb-functional-tests/src/test/resources/historical_plans/url_-_decode_a_url_parameter_using_DECODE_URL_PARAM_-_PROTOBUF_NOSR/7.3.0_1651070263719/topology @@ -0,0 +1,13 @@ +Topologies: + Sub-topology: 0 + Source: KSTREAM-SOURCE-0000000000 (topics: [test_topic]) + --> KSTREAM-TRANSFORMVALUES-0000000001 + Processor: KSTREAM-TRANSFORMVALUES-0000000001 (stores: []) + --> Project + <-- KSTREAM-SOURCE-0000000000 + Processor: Project (stores: []) + --> KSTREAM-SINK-0000000003 + <-- KSTREAM-TRANSFORMVALUES-0000000001 + Sink: KSTREAM-SINK-0000000003 (topic: OUTPUT) + <-- Project + diff --git a/ksqldb-functional-tests/src/test/resources/historical_plans/url_-_encode_a_url_parameter_using_ENCODE_URL_PARAM_-_PROTOBUF/7.3.0_1651070263625/plan.json b/ksqldb-functional-tests/src/test/resources/historical_plans/url_-_encode_a_url_parameter_using_ENCODE_URL_PARAM_-_PROTOBUF/7.3.0_1651070263625/plan.json new file mode 100644 index 000000000000..770fbe96ddf2 --- /dev/null +++ b/ksqldb-functional-tests/src/test/resources/historical_plans/url_-_encode_a_url_parameter_using_ENCODE_URL_PARAM_-_PROTOBUF/7.3.0_1651070263625/plan.json @@ -0,0 +1,211 @@ +{ + "plan" : [ { + "@type" : "ksqlPlanV1", + "statementText" : "CREATE STREAM TEST (K STRING KEY, URL STRING) WITH (KAFKA_TOPIC='test_topic', KEY_FORMAT='KAFKA', VALUE_FORMAT='PROTOBUF');", + "ddlCommand" : { + "@type" : "createStreamV1", + "sourceName" : "TEST", + "schema" : "`K` STRING KEY, `URL` STRING", + "timestampColumn" : null, + "topicName" : "test_topic", + "formats" : { + "keyFormat" : { + "format" : "KAFKA", + "properties" : { } + }, + "valueFormat" : { + "format" : "PROTOBUF", + "properties" : { + "unwrapPrimitives" : "true" + } + } + }, + "windowInfo" : null, + "orReplace" : false, + "isSource" : false + }, + "queryPlan" : null + }, { + "@type" : "ksqlPlanV1", + "statementText" : "CREATE STREAM OUTPUT AS SELECT\n TEST.K K,\n URL_ENCODE_PARAM(TEST.URL) ENCODED\nFROM TEST TEST\nEMIT CHANGES", + "ddlCommand" : { + "@type" : "createStreamV1", + "sourceName" : "OUTPUT", + "schema" : "`K` STRING KEY, `ENCODED` STRING", + "timestampColumn" : null, + "topicName" : "OUTPUT", + "formats" : { + "keyFormat" : { + "format" : "KAFKA", + "properties" : { } + }, + "valueFormat" : { + "format" : "PROTOBUF", + "properties" : { + "unwrapPrimitives" : "true" + } + } + }, + "windowInfo" : null, + "orReplace" : false, + "isSource" : false + }, + "queryPlan" : { + "sources" : [ "TEST" ], + "sink" : "OUTPUT", + "physicalPlan" : { + "@type" : "streamSinkV1", + "properties" : { + "queryContext" : "OUTPUT" + }, + "source" : { + "@type" : "streamSelectV1", + "properties" : { + "queryContext" : "Project" + }, + "source" : { + "@type" : "streamSourceV1", + "properties" : { + "queryContext" : "KsqlTopic/Source" + }, + "topicName" : "test_topic", + "formats" : { + "keyFormat" : { + "format" : "KAFKA", + "properties" : { } + }, + "valueFormat" : { + "format" : "PROTOBUF", + "properties" : { + "unwrapPrimitives" : "true" + } + } + }, + "timestampColumn" : null, + "sourceSchema" : "`K` STRING KEY, `URL` STRING", + "pseudoColumnVersion" : 1 + }, + "keyColumnNames" : [ "K" ], + "selectedKeys" : null, + "selectExpressions" : [ "URL_ENCODE_PARAM(URL) AS ENCODED" ] + }, + "formats" : { + "keyFormat" : { + "format" : "KAFKA", + "properties" : { } + }, + "valueFormat" : { + "format" : "PROTOBUF", + "properties" : { + "unwrapPrimitives" : "true" + } + } + }, + "topicName" : "OUTPUT", + "timestampColumn" : null + }, + "queryId" : "CSAS_OUTPUT_0", + "runtimeId" : null + } + } ], + "configs" : { + "ksql.extension.dir" : "ext", + "ksql.streams.cache.max.bytes.buffering" : "0", + "metric.reporters" : "", + "ksql.query.status.running.threshold.seconds" : "300", + "ksql.connect.basic.auth.credentials.reload" : "false", + "ksql.output.topic.name.prefix" : "", + "ksql.query.pull.stream.enabled" : "true", + "ksql.query.push.v2.interpreter.enabled" : "true", + "ksql.queryanonymizer.logs_enabled" : "true", + "ksql.variable.substitution.enable" : "true", + "ksql.streams.shutdown.timeout.ms" : "300000", + "ksql.query.pull.max.allowed.offset.lag" : "9223372036854775807", + "ksql.query.pull.max.qps" : "2147483647", + "ksql.access.validator.enable" : "auto", + "ksql.streams.bootstrap.servers" : "localhost:0", + "ksql.query.pull.metrics.enabled" : "true", + "ksql.metrics.extension" : null, + "ksql.query.push.v2.alos.enabled" : "true", + "ksql.query.push.v2.max.hourly.bandwidth.megabytes" : "2147483647", + "ksql.query.pull.range.scan.enabled" : "true", + "ksql.transient.query.cleanup.service.initial.delay.seconds" : "600", + "ksql.hidden.topics" : "_confluent.*,__confluent.*,_schemas,__consumer_offsets,__transaction_state,connect-configs,connect-offsets,connect-status,connect-statuses", + "ksql.lambdas.enabled" : "true", + "ksql.source.table.materialization.enabled" : "true", + "ksql.query.pull.max.hourly.bandwidth.megabytes" : "2147483647", + "ksql.sink.window.change.log.additional.retention" : "1000000", + "ksql.query.persistent.active.limit" : "2147483647", + "ksql.persistence.wrap.single.values" : null, + "ksql.query.transient.max.bytes.buffering.total" : "-1", + "ksql.connect.basic.auth.credentials.source" : "NONE", + "ksql.schema.registry.url" : "schema_registry.url:0", + "ksql.properties.overrides.denylist" : "", + "ksql.service.id" : "some.ksql.service.id", + "ksql.query.push.v2.max.catchup.consumers" : "5", + "ksql.query.push.v2.enabled" : "false", + "ksql.transient.query.cleanup.service.enable" : "true", + "ksql.query.push.v2.metrics.enabled" : "true", + "ksql.rowpartition.rowoffset.enabled" : "true", + "ksql.streams.commit.interval.ms" : "2000", + "ksql.query.pull.table.scan.enabled" : "true", + "ksql.streams.auto.commit.interval.ms" : "0", + "ksql.streams.topology.optimization" : "all", + "ksql.endpoint.migrate.query" : "true", + "ksql.query.push.v2.registry.installed" : "false", + "ksql.streams.num.stream.threads" : "4", + "ksql.metrics.tags.custom" : "", + "ksql.query.push.v2.catchup.consumer.msg.window" : "50", + "ksql.runtime.feature.shared.enabled" : "false", + "ksql.udf.collect.metrics" : "false", + "ksql.new.query.planner.enabled" : "false", + "ksql.connect.request.headers.plugin" : null, + "ksql.security.extension.class" : null, + "ksql.transient.prefix" : "transient_", + "ksql.headers.columns.enabled" : "true", + "ksql.streams.default.deserialization.exception.handler" : "io.confluent.ksql.errors.LogMetricAndContinueExceptionHandler", + "ksql.connect.request.timeout.ms" : "5000", + "ksql.query.pull.enable.standby.reads" : "false", + "ksql.persistence.default.format.key" : "KAFKA", + "ksql.query.persistent.max.bytes.buffering.total" : "-1", + "ksql.query.error.max.queue.size" : "10", + "ksql.query.cleanup.shutdown.timeout.ms" : "30000", + "ksql.internal.topic.min.insync.replicas" : "1", + "ksql.internal.topic.replicas" : "1", + "ksql.insert.into.values.enabled" : "true", + "ksql.queryanonymizer.cluster_namespace" : null, + "ksql.create.or.replace.enabled" : "true", + "ksql.shared.runtimes.count" : "2", + "ksql.cast.strings.preserve.nulls" : "true", + "ksql.authorization.cache.max.entries" : "10000", + "ksql.pull.queries.enable" : "true", + "ksql.transient.query.cleanup.service.period.seconds" : "600", + "ksql.suppress.enabled" : "false", + "ksql.readonly.topics" : "_confluent.*,__confluent.*,_schemas,__consumer_offsets,__transaction_state,connect-configs,connect-offsets,connect-status,connect-statuses", + "ksql.connect.basic.auth.credentials.file" : "", + "ksql.authorization.cache.expiry.time.secs" : "30", + "ksql.query.retry.backoff.initial.ms" : "15000", + "ksql.query.pull.max.concurrent.requests" : "2147483647", + "ksql.streams.auto.offset.reset" : "earliest", + "ksql.connect.url" : "http://localhost:8083", + "ksql.query.push.v2.new.latest.delay.ms" : "5000", + "ksql.query.push.v2.latest.reset.age.ms" : "30000", + "ksql.streams.default.production.exception.handler" : "io.confluent.ksql.errors.ProductionExceptionHandlerUtil$LogAndFailProductionExceptionHandler", + "ksql.query.pull.interpreter.enabled" : "true", + "ksql.query.pull.limit.clause.enabled" : "true", + "ksql.query.pull.router.thread.pool.size" : "50", + "ksql.query.push.v2.continuation.tokens.enabled" : "false", + "ksql.query.retry.backoff.max.ms" : "900000", + "ksql.timestamp.throw.on.invalid" : "false", + "ksql.persistence.default.format.value" : null, + "ksql.udfs.enabled" : "true", + "ksql.udf.enable.security.manager" : "true", + "ksql.connect.worker.config" : "", + "ksql.nested.error.set.null" : "true", + "ksql.query.pull.thread.pool.size" : "50", + "ksql.persistent.prefix" : "query_", + "ksql.metastore.backup.location" : "", + "ksql.error.classifier.regex" : "", + "ksql.suppress.buffer.size.bytes" : "-1" + } +} \ No newline at end of file diff --git a/ksqldb-functional-tests/src/test/resources/historical_plans/url_-_encode_a_url_parameter_using_ENCODE_URL_PARAM_-_PROTOBUF/7.3.0_1651070263625/spec.json b/ksqldb-functional-tests/src/test/resources/historical_plans/url_-_encode_a_url_parameter_using_ENCODE_URL_PARAM_-_PROTOBUF/7.3.0_1651070263625/spec.json new file mode 100644 index 000000000000..501610ffce67 --- /dev/null +++ b/ksqldb-functional-tests/src/test/resources/historical_plans/url_-_encode_a_url_parameter_using_ENCODE_URL_PARAM_-_PROTOBUF/7.3.0_1651070263625/spec.json @@ -0,0 +1,144 @@ +{ + "version" : "7.3.0", + "timestamp" : 1651070263625, + "path" : "query-validation-tests/url.json", + "schemas" : { + "CSAS_OUTPUT_0.KsqlTopic.Source" : { + "schema" : "`K` STRING KEY, `URL` STRING", + "keyFormat" : { + "format" : "KAFKA" + }, + "valueFormat" : { + "format" : "PROTOBUF", + "properties" : { + "unwrapPrimitives" : "true" + } + } + }, + "CSAS_OUTPUT_0.OUTPUT" : { + "schema" : "`K` STRING KEY, `ENCODED` STRING", + "keyFormat" : { + "format" : "KAFKA" + }, + "valueFormat" : { + "format" : "PROTOBUF", + "properties" : { + "unwrapPrimitives" : "true" + } + } + } + }, + "testCase" : { + "name" : "encode a url parameter using ENCODE_URL_PARAM - PROTOBUF", + "inputs" : [ { + "topic" : "test_topic", + "key" : "1", + "value" : { + "url" : "?foo $bar" + }, + "timestamp" : 0 + }, { + "topic" : "test_topic", + "key" : "2", + "value" : { + "url" : "hello&world" + }, + "timestamp" : 1 + }, { + "topic" : "test_topic", + "key" : "3", + "value" : { + "url" : "nothing" + }, + "timestamp" : 2 + } ], + "outputs" : [ { + "topic" : "OUTPUT", + "key" : "1", + "value" : { + "ENCODED" : "%3Ffoo+%24bar" + }, + "timestamp" : 0 + }, { + "topic" : "OUTPUT", + "key" : "2", + "value" : { + "ENCODED" : "hello%26world" + }, + "timestamp" : 1 + }, { + "topic" : "OUTPUT", + "key" : "3", + "value" : { + "ENCODED" : "nothing" + }, + "timestamp" : 2 + } ], + "topics" : [ { + "name" : "OUTPUT", + "replicas" : 1, + "numPartitions" : 4 + }, { + "name" : "test_topic", + "valueSchema" : "syntax = \"proto3\";\n\nmessage ConnectDefault1 {\n string URL = 1;\n}\n", + "valueFormat" : "PROTOBUF", + "replicas" : 1, + "numPartitions" : 4 + } ], + "statements" : [ "CREATE STREAM TEST (K STRING KEY, url VARCHAR) WITH (kafka_topic='test_topic', value_format='PROTOBUF');", "CREATE STREAM OUTPUT AS SELECT K, URL_ENCODE_PARAM(url) as ENCODED FROM TEST;" ], + "post" : { + "sources" : [ { + "name" : "OUTPUT", + "type" : "STREAM", + "schema" : "`K` STRING KEY, `ENCODED` STRING", + "keyFormat" : { + "format" : "KAFKA" + }, + "valueFormat" : "PROTOBUF", + "keyFeatures" : [ ], + "valueFeatures" : [ ], + "isSource" : false + }, { + "name" : "TEST", + "type" : "STREAM", + "schema" : "`K` STRING KEY, `URL` STRING", + "keyFormat" : { + "format" : "KAFKA" + }, + "valueFormat" : "PROTOBUF", + "keyFeatures" : [ ], + "valueFeatures" : [ ], + "isSource" : false + } ], + "topics" : { + "topics" : [ { + "name" : "test_topic", + "keyFormat" : { + "format" : "KAFKA" + }, + "valueFormat" : { + "format" : "PROTOBUF", + "properties" : { + "unwrapPrimitives" : "true" + } + }, + "partitions" : 4, + "valueSchema" : "syntax = \"proto3\";\n\nmessage ConnectDefault1 {\n string URL = 1;\n}\n" + }, { + "name" : "OUTPUT", + "keyFormat" : { + "format" : "KAFKA" + }, + "valueFormat" : { + "format" : "PROTOBUF", + "properties" : { + "unwrapPrimitives" : "true" + } + }, + "partitions" : 4, + "valueSchema" : "syntax = \"proto3\";\n\nmessage ConnectDefault1 {\n string ENCODED = 1;\n}\n" + } ] + } + } + } +} \ No newline at end of file diff --git a/ksqldb-functional-tests/src/test/resources/historical_plans/url_-_encode_a_url_parameter_using_ENCODE_URL_PARAM_-_PROTOBUF/7.3.0_1651070263625/topology b/ksqldb-functional-tests/src/test/resources/historical_plans/url_-_encode_a_url_parameter_using_ENCODE_URL_PARAM_-_PROTOBUF/7.3.0_1651070263625/topology new file mode 100644 index 000000000000..441a8f282644 --- /dev/null +++ b/ksqldb-functional-tests/src/test/resources/historical_plans/url_-_encode_a_url_parameter_using_ENCODE_URL_PARAM_-_PROTOBUF/7.3.0_1651070263625/topology @@ -0,0 +1,13 @@ +Topologies: + Sub-topology: 0 + Source: KSTREAM-SOURCE-0000000000 (topics: [test_topic]) + --> KSTREAM-TRANSFORMVALUES-0000000001 + Processor: KSTREAM-TRANSFORMVALUES-0000000001 (stores: []) + --> Project + <-- KSTREAM-SOURCE-0000000000 + Processor: Project (stores: []) + --> KSTREAM-SINK-0000000003 + <-- KSTREAM-TRANSFORMVALUES-0000000001 + Sink: KSTREAM-SINK-0000000003 (topic: OUTPUT) + <-- Project + diff --git a/ksqldb-functional-tests/src/test/resources/historical_plans/url_-_encode_a_url_parameter_using_ENCODE_URL_PARAM_-_PROTOBUF_NOSR/7.3.0_1651070263647/plan.json b/ksqldb-functional-tests/src/test/resources/historical_plans/url_-_encode_a_url_parameter_using_ENCODE_URL_PARAM_-_PROTOBUF_NOSR/7.3.0_1651070263647/plan.json new file mode 100644 index 000000000000..a7b64b79ae9e --- /dev/null +++ b/ksqldb-functional-tests/src/test/resources/historical_plans/url_-_encode_a_url_parameter_using_ENCODE_URL_PARAM_-_PROTOBUF_NOSR/7.3.0_1651070263647/plan.json @@ -0,0 +1,203 @@ +{ + "plan" : [ { + "@type" : "ksqlPlanV1", + "statementText" : "CREATE STREAM TEST (K STRING KEY, URL STRING) WITH (KAFKA_TOPIC='test_topic', KEY_FORMAT='KAFKA', VALUE_FORMAT='PROTOBUF_NOSR');", + "ddlCommand" : { + "@type" : "createStreamV1", + "sourceName" : "TEST", + "schema" : "`K` STRING KEY, `URL` STRING", + "timestampColumn" : null, + "topicName" : "test_topic", + "formats" : { + "keyFormat" : { + "format" : "KAFKA", + "properties" : { } + }, + "valueFormat" : { + "format" : "PROTOBUF_NOSR", + "properties" : { } + } + }, + "windowInfo" : null, + "orReplace" : false, + "isSource" : false + }, + "queryPlan" : null + }, { + "@type" : "ksqlPlanV1", + "statementText" : "CREATE STREAM OUTPUT AS SELECT\n TEST.K K,\n URL_ENCODE_PARAM(TEST.URL) ENCODED\nFROM TEST TEST\nEMIT CHANGES", + "ddlCommand" : { + "@type" : "createStreamV1", + "sourceName" : "OUTPUT", + "schema" : "`K` STRING KEY, `ENCODED` STRING", + "timestampColumn" : null, + "topicName" : "OUTPUT", + "formats" : { + "keyFormat" : { + "format" : "KAFKA", + "properties" : { } + }, + "valueFormat" : { + "format" : "PROTOBUF_NOSR", + "properties" : { } + } + }, + "windowInfo" : null, + "orReplace" : false, + "isSource" : false + }, + "queryPlan" : { + "sources" : [ "TEST" ], + "sink" : "OUTPUT", + "physicalPlan" : { + "@type" : "streamSinkV1", + "properties" : { + "queryContext" : "OUTPUT" + }, + "source" : { + "@type" : "streamSelectV1", + "properties" : { + "queryContext" : "Project" + }, + "source" : { + "@type" : "streamSourceV1", + "properties" : { + "queryContext" : "KsqlTopic/Source" + }, + "topicName" : "test_topic", + "formats" : { + "keyFormat" : { + "format" : "KAFKA", + "properties" : { } + }, + "valueFormat" : { + "format" : "PROTOBUF_NOSR", + "properties" : { } + } + }, + "timestampColumn" : null, + "sourceSchema" : "`K` STRING KEY, `URL` STRING", + "pseudoColumnVersion" : 1 + }, + "keyColumnNames" : [ "K" ], + "selectedKeys" : null, + "selectExpressions" : [ "URL_ENCODE_PARAM(URL) AS ENCODED" ] + }, + "formats" : { + "keyFormat" : { + "format" : "KAFKA", + "properties" : { } + }, + "valueFormat" : { + "format" : "PROTOBUF_NOSR", + "properties" : { } + } + }, + "topicName" : "OUTPUT", + "timestampColumn" : null + }, + "queryId" : "CSAS_OUTPUT_0", + "runtimeId" : null + } + } ], + "configs" : { + "ksql.extension.dir" : "ext", + "ksql.streams.cache.max.bytes.buffering" : "0", + "metric.reporters" : "", + "ksql.query.status.running.threshold.seconds" : "300", + "ksql.connect.basic.auth.credentials.reload" : "false", + "ksql.output.topic.name.prefix" : "", + "ksql.query.pull.stream.enabled" : "true", + "ksql.query.push.v2.interpreter.enabled" : "true", + "ksql.queryanonymizer.logs_enabled" : "true", + "ksql.variable.substitution.enable" : "true", + "ksql.streams.shutdown.timeout.ms" : "300000", + "ksql.query.pull.max.allowed.offset.lag" : "9223372036854775807", + "ksql.query.pull.max.qps" : "2147483647", + "ksql.access.validator.enable" : "auto", + "ksql.streams.bootstrap.servers" : "localhost:0", + "ksql.query.pull.metrics.enabled" : "true", + "ksql.metrics.extension" : null, + "ksql.query.push.v2.alos.enabled" : "true", + "ksql.query.push.v2.max.hourly.bandwidth.megabytes" : "2147483647", + "ksql.query.pull.range.scan.enabled" : "true", + "ksql.transient.query.cleanup.service.initial.delay.seconds" : "600", + "ksql.hidden.topics" : "_confluent.*,__confluent.*,_schemas,__consumer_offsets,__transaction_state,connect-configs,connect-offsets,connect-status,connect-statuses", + "ksql.lambdas.enabled" : "true", + "ksql.source.table.materialization.enabled" : "true", + "ksql.query.pull.max.hourly.bandwidth.megabytes" : "2147483647", + "ksql.sink.window.change.log.additional.retention" : "1000000", + "ksql.query.persistent.active.limit" : "2147483647", + "ksql.persistence.wrap.single.values" : null, + "ksql.query.transient.max.bytes.buffering.total" : "-1", + "ksql.connect.basic.auth.credentials.source" : "NONE", + "ksql.schema.registry.url" : "schema_registry.url:0", + "ksql.properties.overrides.denylist" : "", + "ksql.service.id" : "some.ksql.service.id", + "ksql.query.push.v2.max.catchup.consumers" : "5", + "ksql.query.push.v2.enabled" : "false", + "ksql.transient.query.cleanup.service.enable" : "true", + "ksql.query.push.v2.metrics.enabled" : "true", + "ksql.rowpartition.rowoffset.enabled" : "true", + "ksql.streams.commit.interval.ms" : "2000", + "ksql.query.pull.table.scan.enabled" : "true", + "ksql.streams.auto.commit.interval.ms" : "0", + "ksql.streams.topology.optimization" : "all", + "ksql.endpoint.migrate.query" : "true", + "ksql.query.push.v2.registry.installed" : "false", + "ksql.streams.num.stream.threads" : "4", + "ksql.metrics.tags.custom" : "", + "ksql.query.push.v2.catchup.consumer.msg.window" : "50", + "ksql.runtime.feature.shared.enabled" : "false", + "ksql.udf.collect.metrics" : "false", + "ksql.new.query.planner.enabled" : "false", + "ksql.connect.request.headers.plugin" : null, + "ksql.security.extension.class" : null, + "ksql.transient.prefix" : "transient_", + "ksql.headers.columns.enabled" : "true", + "ksql.streams.default.deserialization.exception.handler" : "io.confluent.ksql.errors.LogMetricAndContinueExceptionHandler", + "ksql.connect.request.timeout.ms" : "5000", + "ksql.query.pull.enable.standby.reads" : "false", + "ksql.persistence.default.format.key" : "KAFKA", + "ksql.query.persistent.max.bytes.buffering.total" : "-1", + "ksql.query.error.max.queue.size" : "10", + "ksql.query.cleanup.shutdown.timeout.ms" : "30000", + "ksql.internal.topic.min.insync.replicas" : "1", + "ksql.internal.topic.replicas" : "1", + "ksql.insert.into.values.enabled" : "true", + "ksql.queryanonymizer.cluster_namespace" : null, + "ksql.create.or.replace.enabled" : "true", + "ksql.shared.runtimes.count" : "2", + "ksql.cast.strings.preserve.nulls" : "true", + "ksql.authorization.cache.max.entries" : "10000", + "ksql.pull.queries.enable" : "true", + "ksql.transient.query.cleanup.service.period.seconds" : "600", + "ksql.suppress.enabled" : "false", + "ksql.readonly.topics" : "_confluent.*,__confluent.*,_schemas,__consumer_offsets,__transaction_state,connect-configs,connect-offsets,connect-status,connect-statuses", + "ksql.connect.basic.auth.credentials.file" : "", + "ksql.authorization.cache.expiry.time.secs" : "30", + "ksql.query.retry.backoff.initial.ms" : "15000", + "ksql.query.pull.max.concurrent.requests" : "2147483647", + "ksql.streams.auto.offset.reset" : "earliest", + "ksql.connect.url" : "http://localhost:8083", + "ksql.query.push.v2.new.latest.delay.ms" : "5000", + "ksql.query.push.v2.latest.reset.age.ms" : "30000", + "ksql.streams.default.production.exception.handler" : "io.confluent.ksql.errors.ProductionExceptionHandlerUtil$LogAndFailProductionExceptionHandler", + "ksql.query.pull.interpreter.enabled" : "true", + "ksql.query.pull.limit.clause.enabled" : "true", + "ksql.query.pull.router.thread.pool.size" : "50", + "ksql.query.push.v2.continuation.tokens.enabled" : "false", + "ksql.query.retry.backoff.max.ms" : "900000", + "ksql.timestamp.throw.on.invalid" : "false", + "ksql.persistence.default.format.value" : null, + "ksql.udfs.enabled" : "true", + "ksql.udf.enable.security.manager" : "true", + "ksql.connect.worker.config" : "", + "ksql.nested.error.set.null" : "true", + "ksql.query.pull.thread.pool.size" : "50", + "ksql.persistent.prefix" : "query_", + "ksql.metastore.backup.location" : "", + "ksql.error.classifier.regex" : "", + "ksql.suppress.buffer.size.bytes" : "-1" + } +} \ No newline at end of file diff --git a/ksqldb-functional-tests/src/test/resources/historical_plans/url_-_encode_a_url_parameter_using_ENCODE_URL_PARAM_-_PROTOBUF_NOSR/7.3.0_1651070263647/spec.json b/ksqldb-functional-tests/src/test/resources/historical_plans/url_-_encode_a_url_parameter_using_ENCODE_URL_PARAM_-_PROTOBUF_NOSR/7.3.0_1651070263647/spec.json new file mode 100644 index 000000000000..11fd6bb9ace5 --- /dev/null +++ b/ksqldb-functional-tests/src/test/resources/historical_plans/url_-_encode_a_url_parameter_using_ENCODE_URL_PARAM_-_PROTOBUF_NOSR/7.3.0_1651070263647/spec.json @@ -0,0 +1,128 @@ +{ + "version" : "7.3.0", + "timestamp" : 1651070263647, + "path" : "query-validation-tests/url.json", + "schemas" : { + "CSAS_OUTPUT_0.KsqlTopic.Source" : { + "schema" : "`K` STRING KEY, `URL` STRING", + "keyFormat" : { + "format" : "KAFKA" + }, + "valueFormat" : { + "format" : "PROTOBUF_NOSR" + } + }, + "CSAS_OUTPUT_0.OUTPUT" : { + "schema" : "`K` STRING KEY, `ENCODED` STRING", + "keyFormat" : { + "format" : "KAFKA" + }, + "valueFormat" : { + "format" : "PROTOBUF_NOSR" + } + } + }, + "testCase" : { + "name" : "encode a url parameter using ENCODE_URL_PARAM - PROTOBUF_NOSR", + "inputs" : [ { + "topic" : "test_topic", + "key" : "1", + "value" : { + "url" : "?foo $bar" + }, + "timestamp" : 0 + }, { + "topic" : "test_topic", + "key" : "2", + "value" : { + "url" : "hello&world" + }, + "timestamp" : 1 + }, { + "topic" : "test_topic", + "key" : "3", + "value" : { + "url" : "nothing" + }, + "timestamp" : 2 + } ], + "outputs" : [ { + "topic" : "OUTPUT", + "key" : "1", + "value" : { + "ENCODED" : "%3Ffoo+%24bar" + }, + "timestamp" : 0 + }, { + "topic" : "OUTPUT", + "key" : "2", + "value" : { + "ENCODED" : "hello%26world" + }, + "timestamp" : 1 + }, { + "topic" : "OUTPUT", + "key" : "3", + "value" : { + "ENCODED" : "nothing" + }, + "timestamp" : 2 + } ], + "topics" : [ { + "name" : "OUTPUT", + "replicas" : 1, + "numPartitions" : 4 + }, { + "name" : "test_topic", + "replicas" : 1, + "numPartitions" : 4 + } ], + "statements" : [ "CREATE STREAM TEST (K STRING KEY, url VARCHAR) WITH (kafka_topic='test_topic', value_format='PROTOBUF_NOSR');", "CREATE STREAM OUTPUT AS SELECT K, URL_ENCODE_PARAM(url) as ENCODED FROM TEST;" ], + "post" : { + "sources" : [ { + "name" : "OUTPUT", + "type" : "STREAM", + "schema" : "`K` STRING KEY, `ENCODED` STRING", + "keyFormat" : { + "format" : "KAFKA" + }, + "valueFormat" : "PROTOBUF_NOSR", + "keyFeatures" : [ ], + "valueFeatures" : [ ], + "isSource" : false + }, { + "name" : "TEST", + "type" : "STREAM", + "schema" : "`K` STRING KEY, `URL` STRING", + "keyFormat" : { + "format" : "KAFKA" + }, + "valueFormat" : "PROTOBUF_NOSR", + "keyFeatures" : [ ], + "valueFeatures" : [ ], + "isSource" : false + } ], + "topics" : { + "topics" : [ { + "name" : "test_topic", + "keyFormat" : { + "format" : "KAFKA" + }, + "valueFormat" : { + "format" : "PROTOBUF_NOSR" + }, + "partitions" : 4 + }, { + "name" : "OUTPUT", + "keyFormat" : { + "format" : "KAFKA" + }, + "valueFormat" : { + "format" : "PROTOBUF_NOSR" + }, + "partitions" : 4 + } ] + } + } + } +} \ No newline at end of file diff --git a/ksqldb-functional-tests/src/test/resources/historical_plans/url_-_encode_a_url_parameter_using_ENCODE_URL_PARAM_-_PROTOBUF_NOSR/7.3.0_1651070263647/topology b/ksqldb-functional-tests/src/test/resources/historical_plans/url_-_encode_a_url_parameter_using_ENCODE_URL_PARAM_-_PROTOBUF_NOSR/7.3.0_1651070263647/topology new file mode 100644 index 000000000000..441a8f282644 --- /dev/null +++ b/ksqldb-functional-tests/src/test/resources/historical_plans/url_-_encode_a_url_parameter_using_ENCODE_URL_PARAM_-_PROTOBUF_NOSR/7.3.0_1651070263647/topology @@ -0,0 +1,13 @@ +Topologies: + Sub-topology: 0 + Source: KSTREAM-SOURCE-0000000000 (topics: [test_topic]) + --> KSTREAM-TRANSFORMVALUES-0000000001 + Processor: KSTREAM-TRANSFORMVALUES-0000000001 (stores: []) + --> Project + <-- KSTREAM-SOURCE-0000000000 + Processor: Project (stores: []) + --> KSTREAM-SINK-0000000003 + <-- KSTREAM-TRANSFORMVALUES-0000000001 + Sink: KSTREAM-SINK-0000000003 (topic: OUTPUT) + <-- Project + diff --git a/ksqldb-functional-tests/src/test/resources/historical_plans/url_-_extract_a_fragment_from_a_URL_using_URL_EXTRACT_FRAGMENT_-_AVRO/7.3.0_1651070263763/plan.json b/ksqldb-functional-tests/src/test/resources/historical_plans/url_-_extract_a_fragment_from_a_URL_using_URL_EXTRACT_FRAGMENT_-_AVRO/7.3.0_1651070263763/plan.json new file mode 100644 index 000000000000..a6df24f5aa79 --- /dev/null +++ b/ksqldb-functional-tests/src/test/resources/historical_plans/url_-_extract_a_fragment_from_a_URL_using_URL_EXTRACT_FRAGMENT_-_AVRO/7.3.0_1651070263763/plan.json @@ -0,0 +1,203 @@ +{ + "plan" : [ { + "@type" : "ksqlPlanV1", + "statementText" : "CREATE STREAM TEST (K STRING KEY, URL STRING) WITH (KAFKA_TOPIC='test_topic', KEY_FORMAT='KAFKA', VALUE_FORMAT='AVRO');", + "ddlCommand" : { + "@type" : "createStreamV1", + "sourceName" : "TEST", + "schema" : "`K` STRING KEY, `URL` STRING", + "timestampColumn" : null, + "topicName" : "test_topic", + "formats" : { + "keyFormat" : { + "format" : "KAFKA", + "properties" : { } + }, + "valueFormat" : { + "format" : "AVRO", + "properties" : { } + } + }, + "windowInfo" : null, + "orReplace" : false, + "isSource" : false + }, + "queryPlan" : null + }, { + "@type" : "ksqlPlanV1", + "statementText" : "CREATE STREAM OUTPUT AS SELECT\n TEST.K K,\n URL_EXTRACT_FRAGMENT(TEST.URL) FRAGMENT\nFROM TEST TEST\nEMIT CHANGES", + "ddlCommand" : { + "@type" : "createStreamV1", + "sourceName" : "OUTPUT", + "schema" : "`K` STRING KEY, `FRAGMENT` STRING", + "timestampColumn" : null, + "topicName" : "OUTPUT", + "formats" : { + "keyFormat" : { + "format" : "KAFKA", + "properties" : { } + }, + "valueFormat" : { + "format" : "AVRO", + "properties" : { } + } + }, + "windowInfo" : null, + "orReplace" : false, + "isSource" : false + }, + "queryPlan" : { + "sources" : [ "TEST" ], + "sink" : "OUTPUT", + "physicalPlan" : { + "@type" : "streamSinkV1", + "properties" : { + "queryContext" : "OUTPUT" + }, + "source" : { + "@type" : "streamSelectV1", + "properties" : { + "queryContext" : "Project" + }, + "source" : { + "@type" : "streamSourceV1", + "properties" : { + "queryContext" : "KsqlTopic/Source" + }, + "topicName" : "test_topic", + "formats" : { + "keyFormat" : { + "format" : "KAFKA", + "properties" : { } + }, + "valueFormat" : { + "format" : "AVRO", + "properties" : { } + } + }, + "timestampColumn" : null, + "sourceSchema" : "`K` STRING KEY, `URL` STRING", + "pseudoColumnVersion" : 1 + }, + "keyColumnNames" : [ "K" ], + "selectedKeys" : null, + "selectExpressions" : [ "URL_EXTRACT_FRAGMENT(URL) AS FRAGMENT" ] + }, + "formats" : { + "keyFormat" : { + "format" : "KAFKA", + "properties" : { } + }, + "valueFormat" : { + "format" : "AVRO", + "properties" : { } + } + }, + "topicName" : "OUTPUT", + "timestampColumn" : null + }, + "queryId" : "CSAS_OUTPUT_0", + "runtimeId" : null + } + } ], + "configs" : { + "ksql.extension.dir" : "ext", + "ksql.streams.cache.max.bytes.buffering" : "0", + "metric.reporters" : "", + "ksql.query.status.running.threshold.seconds" : "300", + "ksql.connect.basic.auth.credentials.reload" : "false", + "ksql.output.topic.name.prefix" : "", + "ksql.query.pull.stream.enabled" : "true", + "ksql.query.push.v2.interpreter.enabled" : "true", + "ksql.queryanonymizer.logs_enabled" : "true", + "ksql.variable.substitution.enable" : "true", + "ksql.streams.shutdown.timeout.ms" : "300000", + "ksql.query.pull.max.allowed.offset.lag" : "9223372036854775807", + "ksql.query.pull.max.qps" : "2147483647", + "ksql.access.validator.enable" : "auto", + "ksql.streams.bootstrap.servers" : "localhost:0", + "ksql.query.pull.metrics.enabled" : "true", + "ksql.metrics.extension" : null, + "ksql.query.push.v2.alos.enabled" : "true", + "ksql.query.push.v2.max.hourly.bandwidth.megabytes" : "2147483647", + "ksql.query.pull.range.scan.enabled" : "true", + "ksql.transient.query.cleanup.service.initial.delay.seconds" : "600", + "ksql.hidden.topics" : "_confluent.*,__confluent.*,_schemas,__consumer_offsets,__transaction_state,connect-configs,connect-offsets,connect-status,connect-statuses", + "ksql.lambdas.enabled" : "true", + "ksql.source.table.materialization.enabled" : "true", + "ksql.query.pull.max.hourly.bandwidth.megabytes" : "2147483647", + "ksql.sink.window.change.log.additional.retention" : "1000000", + "ksql.query.persistent.active.limit" : "2147483647", + "ksql.persistence.wrap.single.values" : null, + "ksql.query.transient.max.bytes.buffering.total" : "-1", + "ksql.connect.basic.auth.credentials.source" : "NONE", + "ksql.schema.registry.url" : "schema_registry.url:0", + "ksql.properties.overrides.denylist" : "", + "ksql.service.id" : "some.ksql.service.id", + "ksql.query.push.v2.max.catchup.consumers" : "5", + "ksql.query.push.v2.enabled" : "false", + "ksql.transient.query.cleanup.service.enable" : "true", + "ksql.query.push.v2.metrics.enabled" : "true", + "ksql.rowpartition.rowoffset.enabled" : "true", + "ksql.streams.commit.interval.ms" : "2000", + "ksql.query.pull.table.scan.enabled" : "true", + "ksql.streams.auto.commit.interval.ms" : "0", + "ksql.streams.topology.optimization" : "all", + "ksql.endpoint.migrate.query" : "true", + "ksql.query.push.v2.registry.installed" : "false", + "ksql.streams.num.stream.threads" : "4", + "ksql.metrics.tags.custom" : "", + "ksql.query.push.v2.catchup.consumer.msg.window" : "50", + "ksql.runtime.feature.shared.enabled" : "false", + "ksql.udf.collect.metrics" : "false", + "ksql.new.query.planner.enabled" : "false", + "ksql.connect.request.headers.plugin" : null, + "ksql.security.extension.class" : null, + "ksql.transient.prefix" : "transient_", + "ksql.headers.columns.enabled" : "true", + "ksql.streams.default.deserialization.exception.handler" : "io.confluent.ksql.errors.LogMetricAndContinueExceptionHandler", + "ksql.connect.request.timeout.ms" : "5000", + "ksql.query.pull.enable.standby.reads" : "false", + "ksql.persistence.default.format.key" : "KAFKA", + "ksql.query.persistent.max.bytes.buffering.total" : "-1", + "ksql.query.error.max.queue.size" : "10", + "ksql.query.cleanup.shutdown.timeout.ms" : "30000", + "ksql.internal.topic.min.insync.replicas" : "1", + "ksql.internal.topic.replicas" : "1", + "ksql.insert.into.values.enabled" : "true", + "ksql.queryanonymizer.cluster_namespace" : null, + "ksql.create.or.replace.enabled" : "true", + "ksql.shared.runtimes.count" : "2", + "ksql.cast.strings.preserve.nulls" : "true", + "ksql.authorization.cache.max.entries" : "10000", + "ksql.pull.queries.enable" : "true", + "ksql.transient.query.cleanup.service.period.seconds" : "600", + "ksql.suppress.enabled" : "false", + "ksql.readonly.topics" : "_confluent.*,__confluent.*,_schemas,__consumer_offsets,__transaction_state,connect-configs,connect-offsets,connect-status,connect-statuses", + "ksql.connect.basic.auth.credentials.file" : "", + "ksql.authorization.cache.expiry.time.secs" : "30", + "ksql.query.retry.backoff.initial.ms" : "15000", + "ksql.query.pull.max.concurrent.requests" : "2147483647", + "ksql.streams.auto.offset.reset" : "earliest", + "ksql.connect.url" : "http://localhost:8083", + "ksql.query.push.v2.new.latest.delay.ms" : "5000", + "ksql.query.push.v2.latest.reset.age.ms" : "30000", + "ksql.streams.default.production.exception.handler" : "io.confluent.ksql.errors.ProductionExceptionHandlerUtil$LogAndFailProductionExceptionHandler", + "ksql.query.pull.interpreter.enabled" : "true", + "ksql.query.pull.limit.clause.enabled" : "true", + "ksql.query.pull.router.thread.pool.size" : "50", + "ksql.query.push.v2.continuation.tokens.enabled" : "false", + "ksql.query.retry.backoff.max.ms" : "900000", + "ksql.timestamp.throw.on.invalid" : "false", + "ksql.persistence.default.format.value" : null, + "ksql.udfs.enabled" : "true", + "ksql.udf.enable.security.manager" : "true", + "ksql.connect.worker.config" : "", + "ksql.nested.error.set.null" : "true", + "ksql.query.pull.thread.pool.size" : "50", + "ksql.persistent.prefix" : "query_", + "ksql.metastore.backup.location" : "", + "ksql.error.classifier.regex" : "", + "ksql.suppress.buffer.size.bytes" : "-1" + } +} \ No newline at end of file diff --git a/ksqldb-functional-tests/src/test/resources/historical_plans/url_-_extract_a_fragment_from_a_URL_using_URL_EXTRACT_FRAGMENT_-_AVRO/7.3.0_1651070263763/spec.json b/ksqldb-functional-tests/src/test/resources/historical_plans/url_-_extract_a_fragment_from_a_URL_using_URL_EXTRACT_FRAGMENT_-_AVRO/7.3.0_1651070263763/spec.json new file mode 100644 index 000000000000..08d3c4548b71 --- /dev/null +++ b/ksqldb-functional-tests/src/test/resources/historical_plans/url_-_extract_a_fragment_from_a_URL_using_URL_EXTRACT_FRAGMENT_-_AVRO/7.3.0_1651070263763/spec.json @@ -0,0 +1,147 @@ +{ + "version" : "7.3.0", + "timestamp" : 1651070263763, + "path" : "query-validation-tests/url.json", + "schemas" : { + "CSAS_OUTPUT_0.KsqlTopic.Source" : { + "schema" : "`K` STRING KEY, `URL` STRING", + "keyFormat" : { + "format" : "KAFKA" + }, + "valueFormat" : { + "format" : "AVRO" + } + }, + "CSAS_OUTPUT_0.OUTPUT" : { + "schema" : "`K` STRING KEY, `FRAGMENT` STRING", + "keyFormat" : { + "format" : "KAFKA" + }, + "valueFormat" : { + "format" : "AVRO" + } + } + }, + "testCase" : { + "name" : "extract a fragment from a URL using URL_EXTRACT_FRAGMENT - AVRO", + "inputs" : [ { + "topic" : "test_topic", + "key" : "1", + "value" : { + "url" : "http://www.test.com/?query#fragment" + }, + "timestamp" : 0 + }, { + "topic" : "test_topic", + "key" : "2", + "value" : { + "url" : "http://www.nofragment.com" + }, + "timestamp" : 1 + } ], + "outputs" : [ { + "topic" : "OUTPUT", + "key" : "1", + "value" : { + "FRAGMENT" : "fragment" + }, + "timestamp" : 0 + }, { + "topic" : "OUTPUT", + "key" : "2", + "value" : { + "FRAGMENT" : null + }, + "timestamp" : 1 + } ], + "topics" : [ { + "name" : "OUTPUT", + "replicas" : 1, + "numPartitions" : 4 + }, { + "name" : "test_topic", + "valueSchema" : { + "type" : "record", + "name" : "KsqlDataSourceSchema", + "namespace" : "io.confluent.ksql.avro_schemas", + "fields" : [ { + "name" : "URL", + "type" : [ "null", "string" ], + "default" : null + } ], + "connect.name" : "io.confluent.ksql.avro_schemas.KsqlDataSourceSchema" + }, + "valueFormat" : "AVRO", + "replicas" : 1, + "numPartitions" : 4 + } ], + "statements" : [ "CREATE STREAM TEST (K STRING KEY, url VARCHAR) WITH (kafka_topic='test_topic', value_format='AVRO');", "CREATE STREAM OUTPUT AS SELECT K, URL_EXTRACT_FRAGMENT(url) as FRAGMENT FROM TEST;" ], + "post" : { + "sources" : [ { + "name" : "OUTPUT", + "type" : "STREAM", + "schema" : "`K` STRING KEY, `FRAGMENT` STRING", + "keyFormat" : { + "format" : "KAFKA" + }, + "valueFormat" : "AVRO", + "keyFeatures" : [ ], + "valueFeatures" : [ ], + "isSource" : false + }, { + "name" : "TEST", + "type" : "STREAM", + "schema" : "`K` STRING KEY, `URL` STRING", + "keyFormat" : { + "format" : "KAFKA" + }, + "valueFormat" : "AVRO", + "keyFeatures" : [ ], + "valueFeatures" : [ ], + "isSource" : false + } ], + "topics" : { + "topics" : [ { + "name" : "test_topic", + "keyFormat" : { + "format" : "KAFKA" + }, + "valueFormat" : { + "format" : "AVRO" + }, + "partitions" : 4, + "valueSchema" : { + "type" : "record", + "name" : "KsqlDataSourceSchema", + "namespace" : "io.confluent.ksql.avro_schemas", + "fields" : [ { + "name" : "URL", + "type" : [ "null", "string" ], + "default" : null + } ], + "connect.name" : "io.confluent.ksql.avro_schemas.KsqlDataSourceSchema" + } + }, { + "name" : "OUTPUT", + "keyFormat" : { + "format" : "KAFKA" + }, + "valueFormat" : { + "format" : "AVRO" + }, + "partitions" : 4, + "valueSchema" : { + "type" : "record", + "name" : "KsqlDataSourceSchema", + "namespace" : "io.confluent.ksql.avro_schemas", + "fields" : [ { + "name" : "FRAGMENT", + "type" : [ "null", "string" ], + "default" : null + } ] + } + } ] + } + } + } +} \ No newline at end of file diff --git a/ksqldb-functional-tests/src/test/resources/historical_plans/url_-_extract_a_fragment_from_a_URL_using_URL_EXTRACT_FRAGMENT_-_AVRO/7.3.0_1651070263763/topology b/ksqldb-functional-tests/src/test/resources/historical_plans/url_-_extract_a_fragment_from_a_URL_using_URL_EXTRACT_FRAGMENT_-_AVRO/7.3.0_1651070263763/topology new file mode 100644 index 000000000000..441a8f282644 --- /dev/null +++ b/ksqldb-functional-tests/src/test/resources/historical_plans/url_-_extract_a_fragment_from_a_URL_using_URL_EXTRACT_FRAGMENT_-_AVRO/7.3.0_1651070263763/topology @@ -0,0 +1,13 @@ +Topologies: + Sub-topology: 0 + Source: KSTREAM-SOURCE-0000000000 (topics: [test_topic]) + --> KSTREAM-TRANSFORMVALUES-0000000001 + Processor: KSTREAM-TRANSFORMVALUES-0000000001 (stores: []) + --> Project + <-- KSTREAM-SOURCE-0000000000 + Processor: Project (stores: []) + --> KSTREAM-SINK-0000000003 + <-- KSTREAM-TRANSFORMVALUES-0000000001 + Sink: KSTREAM-SINK-0000000003 (topic: OUTPUT) + <-- Project + diff --git a/ksqldb-functional-tests/src/test/resources/historical_plans/url_-_extract_a_fragment_from_a_URL_using_URL_EXTRACT_FRAGMENT_-_PROTOBUFs_-_PROTOBUF/7.3.0_1651070263786/plan.json b/ksqldb-functional-tests/src/test/resources/historical_plans/url_-_extract_a_fragment_from_a_URL_using_URL_EXTRACT_FRAGMENT_-_PROTOBUFs_-_PROTOBUF/7.3.0_1651070263786/plan.json new file mode 100644 index 000000000000..ce6859fecb1a --- /dev/null +++ b/ksqldb-functional-tests/src/test/resources/historical_plans/url_-_extract_a_fragment_from_a_URL_using_URL_EXTRACT_FRAGMENT_-_PROTOBUFs_-_PROTOBUF/7.3.0_1651070263786/plan.json @@ -0,0 +1,211 @@ +{ + "plan" : [ { + "@type" : "ksqlPlanV1", + "statementText" : "CREATE STREAM TEST (K STRING KEY, URL STRING) WITH (KAFKA_TOPIC='test_topic', KEY_FORMAT='KAFKA', VALUE_FORMAT='PROTOBUF');", + "ddlCommand" : { + "@type" : "createStreamV1", + "sourceName" : "TEST", + "schema" : "`K` STRING KEY, `URL` STRING", + "timestampColumn" : null, + "topicName" : "test_topic", + "formats" : { + "keyFormat" : { + "format" : "KAFKA", + "properties" : { } + }, + "valueFormat" : { + "format" : "PROTOBUF", + "properties" : { + "unwrapPrimitives" : "true" + } + } + }, + "windowInfo" : null, + "orReplace" : false, + "isSource" : false + }, + "queryPlan" : null + }, { + "@type" : "ksqlPlanV1", + "statementText" : "CREATE STREAM OUTPUT AS SELECT\n TEST.K K,\n URL_EXTRACT_FRAGMENT(TEST.URL) FRAGMENT\nFROM TEST TEST\nEMIT CHANGES", + "ddlCommand" : { + "@type" : "createStreamV1", + "sourceName" : "OUTPUT", + "schema" : "`K` STRING KEY, `FRAGMENT` STRING", + "timestampColumn" : null, + "topicName" : "OUTPUT", + "formats" : { + "keyFormat" : { + "format" : "KAFKA", + "properties" : { } + }, + "valueFormat" : { + "format" : "PROTOBUF", + "properties" : { + "unwrapPrimitives" : "true" + } + } + }, + "windowInfo" : null, + "orReplace" : false, + "isSource" : false + }, + "queryPlan" : { + "sources" : [ "TEST" ], + "sink" : "OUTPUT", + "physicalPlan" : { + "@type" : "streamSinkV1", + "properties" : { + "queryContext" : "OUTPUT" + }, + "source" : { + "@type" : "streamSelectV1", + "properties" : { + "queryContext" : "Project" + }, + "source" : { + "@type" : "streamSourceV1", + "properties" : { + "queryContext" : "KsqlTopic/Source" + }, + "topicName" : "test_topic", + "formats" : { + "keyFormat" : { + "format" : "KAFKA", + "properties" : { } + }, + "valueFormat" : { + "format" : "PROTOBUF", + "properties" : { + "unwrapPrimitives" : "true" + } + } + }, + "timestampColumn" : null, + "sourceSchema" : "`K` STRING KEY, `URL` STRING", + "pseudoColumnVersion" : 1 + }, + "keyColumnNames" : [ "K" ], + "selectedKeys" : null, + "selectExpressions" : [ "URL_EXTRACT_FRAGMENT(URL) AS FRAGMENT" ] + }, + "formats" : { + "keyFormat" : { + "format" : "KAFKA", + "properties" : { } + }, + "valueFormat" : { + "format" : "PROTOBUF", + "properties" : { + "unwrapPrimitives" : "true" + } + } + }, + "topicName" : "OUTPUT", + "timestampColumn" : null + }, + "queryId" : "CSAS_OUTPUT_0", + "runtimeId" : null + } + } ], + "configs" : { + "ksql.extension.dir" : "ext", + "ksql.streams.cache.max.bytes.buffering" : "0", + "metric.reporters" : "", + "ksql.query.status.running.threshold.seconds" : "300", + "ksql.connect.basic.auth.credentials.reload" : "false", + "ksql.output.topic.name.prefix" : "", + "ksql.query.pull.stream.enabled" : "true", + "ksql.query.push.v2.interpreter.enabled" : "true", + "ksql.queryanonymizer.logs_enabled" : "true", + "ksql.variable.substitution.enable" : "true", + "ksql.streams.shutdown.timeout.ms" : "300000", + "ksql.query.pull.max.allowed.offset.lag" : "9223372036854775807", + "ksql.query.pull.max.qps" : "2147483647", + "ksql.access.validator.enable" : "auto", + "ksql.streams.bootstrap.servers" : "localhost:0", + "ksql.query.pull.metrics.enabled" : "true", + "ksql.metrics.extension" : null, + "ksql.query.push.v2.alos.enabled" : "true", + "ksql.query.push.v2.max.hourly.bandwidth.megabytes" : "2147483647", + "ksql.query.pull.range.scan.enabled" : "true", + "ksql.transient.query.cleanup.service.initial.delay.seconds" : "600", + "ksql.hidden.topics" : "_confluent.*,__confluent.*,_schemas,__consumer_offsets,__transaction_state,connect-configs,connect-offsets,connect-status,connect-statuses", + "ksql.lambdas.enabled" : "true", + "ksql.source.table.materialization.enabled" : "true", + "ksql.query.pull.max.hourly.bandwidth.megabytes" : "2147483647", + "ksql.sink.window.change.log.additional.retention" : "1000000", + "ksql.query.persistent.active.limit" : "2147483647", + "ksql.persistence.wrap.single.values" : null, + "ksql.query.transient.max.bytes.buffering.total" : "-1", + "ksql.connect.basic.auth.credentials.source" : "NONE", + "ksql.schema.registry.url" : "schema_registry.url:0", + "ksql.properties.overrides.denylist" : "", + "ksql.service.id" : "some.ksql.service.id", + "ksql.query.push.v2.max.catchup.consumers" : "5", + "ksql.query.push.v2.enabled" : "false", + "ksql.transient.query.cleanup.service.enable" : "true", + "ksql.query.push.v2.metrics.enabled" : "true", + "ksql.rowpartition.rowoffset.enabled" : "true", + "ksql.streams.commit.interval.ms" : "2000", + "ksql.query.pull.table.scan.enabled" : "true", + "ksql.streams.auto.commit.interval.ms" : "0", + "ksql.streams.topology.optimization" : "all", + "ksql.endpoint.migrate.query" : "true", + "ksql.query.push.v2.registry.installed" : "false", + "ksql.streams.num.stream.threads" : "4", + "ksql.metrics.tags.custom" : "", + "ksql.query.push.v2.catchup.consumer.msg.window" : "50", + "ksql.runtime.feature.shared.enabled" : "false", + "ksql.udf.collect.metrics" : "false", + "ksql.new.query.planner.enabled" : "false", + "ksql.connect.request.headers.plugin" : null, + "ksql.security.extension.class" : null, + "ksql.transient.prefix" : "transient_", + "ksql.headers.columns.enabled" : "true", + "ksql.streams.default.deserialization.exception.handler" : "io.confluent.ksql.errors.LogMetricAndContinueExceptionHandler", + "ksql.connect.request.timeout.ms" : "5000", + "ksql.query.pull.enable.standby.reads" : "false", + "ksql.persistence.default.format.key" : "KAFKA", + "ksql.query.persistent.max.bytes.buffering.total" : "-1", + "ksql.query.error.max.queue.size" : "10", + "ksql.query.cleanup.shutdown.timeout.ms" : "30000", + "ksql.internal.topic.min.insync.replicas" : "1", + "ksql.internal.topic.replicas" : "1", + "ksql.insert.into.values.enabled" : "true", + "ksql.queryanonymizer.cluster_namespace" : null, + "ksql.create.or.replace.enabled" : "true", + "ksql.shared.runtimes.count" : "2", + "ksql.cast.strings.preserve.nulls" : "true", + "ksql.authorization.cache.max.entries" : "10000", + "ksql.pull.queries.enable" : "true", + "ksql.transient.query.cleanup.service.period.seconds" : "600", + "ksql.suppress.enabled" : "false", + "ksql.readonly.topics" : "_confluent.*,__confluent.*,_schemas,__consumer_offsets,__transaction_state,connect-configs,connect-offsets,connect-status,connect-statuses", + "ksql.connect.basic.auth.credentials.file" : "", + "ksql.authorization.cache.expiry.time.secs" : "30", + "ksql.query.retry.backoff.initial.ms" : "15000", + "ksql.query.pull.max.concurrent.requests" : "2147483647", + "ksql.streams.auto.offset.reset" : "earliest", + "ksql.connect.url" : "http://localhost:8083", + "ksql.query.push.v2.new.latest.delay.ms" : "5000", + "ksql.query.push.v2.latest.reset.age.ms" : "30000", + "ksql.streams.default.production.exception.handler" : "io.confluent.ksql.errors.ProductionExceptionHandlerUtil$LogAndFailProductionExceptionHandler", + "ksql.query.pull.interpreter.enabled" : "true", + "ksql.query.pull.limit.clause.enabled" : "true", + "ksql.query.pull.router.thread.pool.size" : "50", + "ksql.query.push.v2.continuation.tokens.enabled" : "false", + "ksql.query.retry.backoff.max.ms" : "900000", + "ksql.timestamp.throw.on.invalid" : "false", + "ksql.persistence.default.format.value" : null, + "ksql.udfs.enabled" : "true", + "ksql.udf.enable.security.manager" : "true", + "ksql.connect.worker.config" : "", + "ksql.nested.error.set.null" : "true", + "ksql.query.pull.thread.pool.size" : "50", + "ksql.persistent.prefix" : "query_", + "ksql.metastore.backup.location" : "", + "ksql.error.classifier.regex" : "", + "ksql.suppress.buffer.size.bytes" : "-1" + } +} \ No newline at end of file diff --git a/ksqldb-functional-tests/src/test/resources/historical_plans/url_-_extract_a_fragment_from_a_URL_using_URL_EXTRACT_FRAGMENT_-_PROTOBUFs_-_PROTOBUF/7.3.0_1651070263786/spec.json b/ksqldb-functional-tests/src/test/resources/historical_plans/url_-_extract_a_fragment_from_a_URL_using_URL_EXTRACT_FRAGMENT_-_PROTOBUFs_-_PROTOBUF/7.3.0_1651070263786/spec.json new file mode 100644 index 000000000000..bd966d94ad44 --- /dev/null +++ b/ksqldb-functional-tests/src/test/resources/historical_plans/url_-_extract_a_fragment_from_a_URL_using_URL_EXTRACT_FRAGMENT_-_PROTOBUFs_-_PROTOBUF/7.3.0_1651070263786/spec.json @@ -0,0 +1,130 @@ +{ + "version" : "7.3.0", + "timestamp" : 1651070263786, + "path" : "query-validation-tests/url.json", + "schemas" : { + "CSAS_OUTPUT_0.KsqlTopic.Source" : { + "schema" : "`K` STRING KEY, `URL` STRING", + "keyFormat" : { + "format" : "KAFKA" + }, + "valueFormat" : { + "format" : "PROTOBUF", + "properties" : { + "unwrapPrimitives" : "true" + } + } + }, + "CSAS_OUTPUT_0.OUTPUT" : { + "schema" : "`K` STRING KEY, `FRAGMENT` STRING", + "keyFormat" : { + "format" : "KAFKA" + }, + "valueFormat" : { + "format" : "PROTOBUF", + "properties" : { + "unwrapPrimitives" : "true" + } + } + } + }, + "testCase" : { + "name" : "extract a fragment from a URL using URL_EXTRACT_FRAGMENT - PROTOBUFs - PROTOBUF", + "inputs" : [ { + "topic" : "test_topic", + "key" : "1", + "value" : { + "url" : "http://www.test.com/?query#fragment" + }, + "timestamp" : 0 + }, { + "topic" : "test_topic", + "key" : "2", + "value" : { + "url" : "http://www.nofragment.com" + }, + "timestamp" : 1 + } ], + "outputs" : [ { + "topic" : "OUTPUT", + "key" : "1", + "value" : { + "FRAGMENT" : "fragment" + }, + "timestamp" : 0 + }, { + "topic" : "OUTPUT", + "key" : "2", + "value" : { + "FRAGMENT" : "" + }, + "timestamp" : 1 + } ], + "topics" : [ { + "name" : "OUTPUT", + "replicas" : 1, + "numPartitions" : 4 + }, { + "name" : "test_topic", + "valueSchema" : "syntax = \"proto3\";\n\nmessage ConnectDefault1 {\n string URL = 1;\n}\n", + "valueFormat" : "PROTOBUF", + "replicas" : 1, + "numPartitions" : 4 + } ], + "statements" : [ "CREATE STREAM TEST (K STRING KEY, url VARCHAR) WITH (kafka_topic='test_topic', value_format='PROTOBUF');", "CREATE STREAM OUTPUT AS SELECT K, URL_EXTRACT_FRAGMENT(url) as FRAGMENT FROM TEST;" ], + "post" : { + "sources" : [ { + "name" : "OUTPUT", + "type" : "STREAM", + "schema" : "`K` STRING KEY, `FRAGMENT` STRING", + "keyFormat" : { + "format" : "KAFKA" + }, + "valueFormat" : "PROTOBUF", + "keyFeatures" : [ ], + "valueFeatures" : [ ], + "isSource" : false + }, { + "name" : "TEST", + "type" : "STREAM", + "schema" : "`K` STRING KEY, `URL` STRING", + "keyFormat" : { + "format" : "KAFKA" + }, + "valueFormat" : "PROTOBUF", + "keyFeatures" : [ ], + "valueFeatures" : [ ], + "isSource" : false + } ], + "topics" : { + "topics" : [ { + "name" : "test_topic", + "keyFormat" : { + "format" : "KAFKA" + }, + "valueFormat" : { + "format" : "PROTOBUF", + "properties" : { + "unwrapPrimitives" : "true" + } + }, + "partitions" : 4, + "valueSchema" : "syntax = \"proto3\";\n\nmessage ConnectDefault1 {\n string URL = 1;\n}\n" + }, { + "name" : "OUTPUT", + "keyFormat" : { + "format" : "KAFKA" + }, + "valueFormat" : { + "format" : "PROTOBUF", + "properties" : { + "unwrapPrimitives" : "true" + } + }, + "partitions" : 4, + "valueSchema" : "syntax = \"proto3\";\n\nmessage ConnectDefault1 {\n string FRAGMENT = 1;\n}\n" + } ] + } + } + } +} \ No newline at end of file diff --git a/ksqldb-functional-tests/src/test/resources/historical_plans/url_-_extract_a_fragment_from_a_URL_using_URL_EXTRACT_FRAGMENT_-_PROTOBUFs_-_PROTOBUF/7.3.0_1651070263786/topology b/ksqldb-functional-tests/src/test/resources/historical_plans/url_-_extract_a_fragment_from_a_URL_using_URL_EXTRACT_FRAGMENT_-_PROTOBUFs_-_PROTOBUF/7.3.0_1651070263786/topology new file mode 100644 index 000000000000..441a8f282644 --- /dev/null +++ b/ksqldb-functional-tests/src/test/resources/historical_plans/url_-_extract_a_fragment_from_a_URL_using_URL_EXTRACT_FRAGMENT_-_PROTOBUFs_-_PROTOBUF/7.3.0_1651070263786/topology @@ -0,0 +1,13 @@ +Topologies: + Sub-topology: 0 + Source: KSTREAM-SOURCE-0000000000 (topics: [test_topic]) + --> KSTREAM-TRANSFORMVALUES-0000000001 + Processor: KSTREAM-TRANSFORMVALUES-0000000001 (stores: []) + --> Project + <-- KSTREAM-SOURCE-0000000000 + Processor: Project (stores: []) + --> KSTREAM-SINK-0000000003 + <-- KSTREAM-TRANSFORMVALUES-0000000001 + Sink: KSTREAM-SINK-0000000003 (topic: OUTPUT) + <-- Project + diff --git a/ksqldb-functional-tests/src/test/resources/historical_plans/url_-_extract_a_fragment_from_a_URL_using_URL_EXTRACT_FRAGMENT_-_PROTOBUFs_-_PROTOBUF_NOSR/7.3.0_1651070263812/plan.json b/ksqldb-functional-tests/src/test/resources/historical_plans/url_-_extract_a_fragment_from_a_URL_using_URL_EXTRACT_FRAGMENT_-_PROTOBUFs_-_PROTOBUF_NOSR/7.3.0_1651070263812/plan.json new file mode 100644 index 000000000000..185f8063b27b --- /dev/null +++ b/ksqldb-functional-tests/src/test/resources/historical_plans/url_-_extract_a_fragment_from_a_URL_using_URL_EXTRACT_FRAGMENT_-_PROTOBUFs_-_PROTOBUF_NOSR/7.3.0_1651070263812/plan.json @@ -0,0 +1,203 @@ +{ + "plan" : [ { + "@type" : "ksqlPlanV1", + "statementText" : "CREATE STREAM TEST (K STRING KEY, URL STRING) WITH (KAFKA_TOPIC='test_topic', KEY_FORMAT='KAFKA', VALUE_FORMAT='PROTOBUF_NOSR');", + "ddlCommand" : { + "@type" : "createStreamV1", + "sourceName" : "TEST", + "schema" : "`K` STRING KEY, `URL` STRING", + "timestampColumn" : null, + "topicName" : "test_topic", + "formats" : { + "keyFormat" : { + "format" : "KAFKA", + "properties" : { } + }, + "valueFormat" : { + "format" : "PROTOBUF_NOSR", + "properties" : { } + } + }, + "windowInfo" : null, + "orReplace" : false, + "isSource" : false + }, + "queryPlan" : null + }, { + "@type" : "ksqlPlanV1", + "statementText" : "CREATE STREAM OUTPUT AS SELECT\n TEST.K K,\n URL_EXTRACT_FRAGMENT(TEST.URL) FRAGMENT\nFROM TEST TEST\nEMIT CHANGES", + "ddlCommand" : { + "@type" : "createStreamV1", + "sourceName" : "OUTPUT", + "schema" : "`K` STRING KEY, `FRAGMENT` STRING", + "timestampColumn" : null, + "topicName" : "OUTPUT", + "formats" : { + "keyFormat" : { + "format" : "KAFKA", + "properties" : { } + }, + "valueFormat" : { + "format" : "PROTOBUF_NOSR", + "properties" : { } + } + }, + "windowInfo" : null, + "orReplace" : false, + "isSource" : false + }, + "queryPlan" : { + "sources" : [ "TEST" ], + "sink" : "OUTPUT", + "physicalPlan" : { + "@type" : "streamSinkV1", + "properties" : { + "queryContext" : "OUTPUT" + }, + "source" : { + "@type" : "streamSelectV1", + "properties" : { + "queryContext" : "Project" + }, + "source" : { + "@type" : "streamSourceV1", + "properties" : { + "queryContext" : "KsqlTopic/Source" + }, + "topicName" : "test_topic", + "formats" : { + "keyFormat" : { + "format" : "KAFKA", + "properties" : { } + }, + "valueFormat" : { + "format" : "PROTOBUF_NOSR", + "properties" : { } + } + }, + "timestampColumn" : null, + "sourceSchema" : "`K` STRING KEY, `URL` STRING", + "pseudoColumnVersion" : 1 + }, + "keyColumnNames" : [ "K" ], + "selectedKeys" : null, + "selectExpressions" : [ "URL_EXTRACT_FRAGMENT(URL) AS FRAGMENT" ] + }, + "formats" : { + "keyFormat" : { + "format" : "KAFKA", + "properties" : { } + }, + "valueFormat" : { + "format" : "PROTOBUF_NOSR", + "properties" : { } + } + }, + "topicName" : "OUTPUT", + "timestampColumn" : null + }, + "queryId" : "CSAS_OUTPUT_0", + "runtimeId" : null + } + } ], + "configs" : { + "ksql.extension.dir" : "ext", + "ksql.streams.cache.max.bytes.buffering" : "0", + "metric.reporters" : "", + "ksql.query.status.running.threshold.seconds" : "300", + "ksql.connect.basic.auth.credentials.reload" : "false", + "ksql.output.topic.name.prefix" : "", + "ksql.query.pull.stream.enabled" : "true", + "ksql.query.push.v2.interpreter.enabled" : "true", + "ksql.queryanonymizer.logs_enabled" : "true", + "ksql.variable.substitution.enable" : "true", + "ksql.streams.shutdown.timeout.ms" : "300000", + "ksql.query.pull.max.allowed.offset.lag" : "9223372036854775807", + "ksql.query.pull.max.qps" : "2147483647", + "ksql.access.validator.enable" : "auto", + "ksql.streams.bootstrap.servers" : "localhost:0", + "ksql.query.pull.metrics.enabled" : "true", + "ksql.metrics.extension" : null, + "ksql.query.push.v2.alos.enabled" : "true", + "ksql.query.push.v2.max.hourly.bandwidth.megabytes" : "2147483647", + "ksql.query.pull.range.scan.enabled" : "true", + "ksql.transient.query.cleanup.service.initial.delay.seconds" : "600", + "ksql.hidden.topics" : "_confluent.*,__confluent.*,_schemas,__consumer_offsets,__transaction_state,connect-configs,connect-offsets,connect-status,connect-statuses", + "ksql.lambdas.enabled" : "true", + "ksql.source.table.materialization.enabled" : "true", + "ksql.query.pull.max.hourly.bandwidth.megabytes" : "2147483647", + "ksql.sink.window.change.log.additional.retention" : "1000000", + "ksql.query.persistent.active.limit" : "2147483647", + "ksql.persistence.wrap.single.values" : null, + "ksql.query.transient.max.bytes.buffering.total" : "-1", + "ksql.connect.basic.auth.credentials.source" : "NONE", + "ksql.schema.registry.url" : "schema_registry.url:0", + "ksql.properties.overrides.denylist" : "", + "ksql.service.id" : "some.ksql.service.id", + "ksql.query.push.v2.max.catchup.consumers" : "5", + "ksql.query.push.v2.enabled" : "false", + "ksql.transient.query.cleanup.service.enable" : "true", + "ksql.query.push.v2.metrics.enabled" : "true", + "ksql.rowpartition.rowoffset.enabled" : "true", + "ksql.streams.commit.interval.ms" : "2000", + "ksql.query.pull.table.scan.enabled" : "true", + "ksql.streams.auto.commit.interval.ms" : "0", + "ksql.streams.topology.optimization" : "all", + "ksql.endpoint.migrate.query" : "true", + "ksql.query.push.v2.registry.installed" : "false", + "ksql.streams.num.stream.threads" : "4", + "ksql.metrics.tags.custom" : "", + "ksql.query.push.v2.catchup.consumer.msg.window" : "50", + "ksql.runtime.feature.shared.enabled" : "false", + "ksql.udf.collect.metrics" : "false", + "ksql.new.query.planner.enabled" : "false", + "ksql.connect.request.headers.plugin" : null, + "ksql.security.extension.class" : null, + "ksql.transient.prefix" : "transient_", + "ksql.headers.columns.enabled" : "true", + "ksql.streams.default.deserialization.exception.handler" : "io.confluent.ksql.errors.LogMetricAndContinueExceptionHandler", + "ksql.connect.request.timeout.ms" : "5000", + "ksql.query.pull.enable.standby.reads" : "false", + "ksql.persistence.default.format.key" : "KAFKA", + "ksql.query.persistent.max.bytes.buffering.total" : "-1", + "ksql.query.error.max.queue.size" : "10", + "ksql.query.cleanup.shutdown.timeout.ms" : "30000", + "ksql.internal.topic.min.insync.replicas" : "1", + "ksql.internal.topic.replicas" : "1", + "ksql.insert.into.values.enabled" : "true", + "ksql.queryanonymizer.cluster_namespace" : null, + "ksql.create.or.replace.enabled" : "true", + "ksql.shared.runtimes.count" : "2", + "ksql.cast.strings.preserve.nulls" : "true", + "ksql.authorization.cache.max.entries" : "10000", + "ksql.pull.queries.enable" : "true", + "ksql.transient.query.cleanup.service.period.seconds" : "600", + "ksql.suppress.enabled" : "false", + "ksql.readonly.topics" : "_confluent.*,__confluent.*,_schemas,__consumer_offsets,__transaction_state,connect-configs,connect-offsets,connect-status,connect-statuses", + "ksql.connect.basic.auth.credentials.file" : "", + "ksql.authorization.cache.expiry.time.secs" : "30", + "ksql.query.retry.backoff.initial.ms" : "15000", + "ksql.query.pull.max.concurrent.requests" : "2147483647", + "ksql.streams.auto.offset.reset" : "earliest", + "ksql.connect.url" : "http://localhost:8083", + "ksql.query.push.v2.new.latest.delay.ms" : "5000", + "ksql.query.push.v2.latest.reset.age.ms" : "30000", + "ksql.streams.default.production.exception.handler" : "io.confluent.ksql.errors.ProductionExceptionHandlerUtil$LogAndFailProductionExceptionHandler", + "ksql.query.pull.interpreter.enabled" : "true", + "ksql.query.pull.limit.clause.enabled" : "true", + "ksql.query.pull.router.thread.pool.size" : "50", + "ksql.query.push.v2.continuation.tokens.enabled" : "false", + "ksql.query.retry.backoff.max.ms" : "900000", + "ksql.timestamp.throw.on.invalid" : "false", + "ksql.persistence.default.format.value" : null, + "ksql.udfs.enabled" : "true", + "ksql.udf.enable.security.manager" : "true", + "ksql.connect.worker.config" : "", + "ksql.nested.error.set.null" : "true", + "ksql.query.pull.thread.pool.size" : "50", + "ksql.persistent.prefix" : "query_", + "ksql.metastore.backup.location" : "", + "ksql.error.classifier.regex" : "", + "ksql.suppress.buffer.size.bytes" : "-1" + } +} \ No newline at end of file diff --git a/ksqldb-functional-tests/src/test/resources/historical_plans/url_-_extract_a_fragment_from_a_URL_using_URL_EXTRACT_FRAGMENT_-_PROTOBUFs_-_PROTOBUF_NOSR/7.3.0_1651070263812/spec.json b/ksqldb-functional-tests/src/test/resources/historical_plans/url_-_extract_a_fragment_from_a_URL_using_URL_EXTRACT_FRAGMENT_-_PROTOBUFs_-_PROTOBUF_NOSR/7.3.0_1651070263812/spec.json new file mode 100644 index 000000000000..8a0b863ab913 --- /dev/null +++ b/ksqldb-functional-tests/src/test/resources/historical_plans/url_-_extract_a_fragment_from_a_URL_using_URL_EXTRACT_FRAGMENT_-_PROTOBUFs_-_PROTOBUF_NOSR/7.3.0_1651070263812/spec.json @@ -0,0 +1,114 @@ +{ + "version" : "7.3.0", + "timestamp" : 1651070263812, + "path" : "query-validation-tests/url.json", + "schemas" : { + "CSAS_OUTPUT_0.KsqlTopic.Source" : { + "schema" : "`K` STRING KEY, `URL` STRING", + "keyFormat" : { + "format" : "KAFKA" + }, + "valueFormat" : { + "format" : "PROTOBUF_NOSR" + } + }, + "CSAS_OUTPUT_0.OUTPUT" : { + "schema" : "`K` STRING KEY, `FRAGMENT` STRING", + "keyFormat" : { + "format" : "KAFKA" + }, + "valueFormat" : { + "format" : "PROTOBUF_NOSR" + } + } + }, + "testCase" : { + "name" : "extract a fragment from a URL using URL_EXTRACT_FRAGMENT - PROTOBUFs - PROTOBUF_NOSR", + "inputs" : [ { + "topic" : "test_topic", + "key" : "1", + "value" : { + "url" : "http://www.test.com/?query#fragment" + }, + "timestamp" : 0 + }, { + "topic" : "test_topic", + "key" : "2", + "value" : { + "url" : "http://www.nofragment.com" + }, + "timestamp" : 1 + } ], + "outputs" : [ { + "topic" : "OUTPUT", + "key" : "1", + "value" : { + "FRAGMENT" : "fragment" + }, + "timestamp" : 0 + }, { + "topic" : "OUTPUT", + "key" : "2", + "value" : { + "FRAGMENT" : "" + }, + "timestamp" : 1 + } ], + "topics" : [ { + "name" : "OUTPUT", + "replicas" : 1, + "numPartitions" : 4 + }, { + "name" : "test_topic", + "replicas" : 1, + "numPartitions" : 4 + } ], + "statements" : [ "CREATE STREAM TEST (K STRING KEY, url VARCHAR) WITH (kafka_topic='test_topic', value_format='PROTOBUF_NOSR');", "CREATE STREAM OUTPUT AS SELECT K, URL_EXTRACT_FRAGMENT(url) as FRAGMENT FROM TEST;" ], + "post" : { + "sources" : [ { + "name" : "OUTPUT", + "type" : "STREAM", + "schema" : "`K` STRING KEY, `FRAGMENT` STRING", + "keyFormat" : { + "format" : "KAFKA" + }, + "valueFormat" : "PROTOBUF_NOSR", + "keyFeatures" : [ ], + "valueFeatures" : [ ], + "isSource" : false + }, { + "name" : "TEST", + "type" : "STREAM", + "schema" : "`K` STRING KEY, `URL` STRING", + "keyFormat" : { + "format" : "KAFKA" + }, + "valueFormat" : "PROTOBUF_NOSR", + "keyFeatures" : [ ], + "valueFeatures" : [ ], + "isSource" : false + } ], + "topics" : { + "topics" : [ { + "name" : "test_topic", + "keyFormat" : { + "format" : "KAFKA" + }, + "valueFormat" : { + "format" : "PROTOBUF_NOSR" + }, + "partitions" : 4 + }, { + "name" : "OUTPUT", + "keyFormat" : { + "format" : "KAFKA" + }, + "valueFormat" : { + "format" : "PROTOBUF_NOSR" + }, + "partitions" : 4 + } ] + } + } + } +} \ No newline at end of file diff --git a/ksqldb-functional-tests/src/test/resources/historical_plans/url_-_extract_a_fragment_from_a_URL_using_URL_EXTRACT_FRAGMENT_-_PROTOBUFs_-_PROTOBUF_NOSR/7.3.0_1651070263812/topology b/ksqldb-functional-tests/src/test/resources/historical_plans/url_-_extract_a_fragment_from_a_URL_using_URL_EXTRACT_FRAGMENT_-_PROTOBUFs_-_PROTOBUF_NOSR/7.3.0_1651070263812/topology new file mode 100644 index 000000000000..441a8f282644 --- /dev/null +++ b/ksqldb-functional-tests/src/test/resources/historical_plans/url_-_extract_a_fragment_from_a_URL_using_URL_EXTRACT_FRAGMENT_-_PROTOBUFs_-_PROTOBUF_NOSR/7.3.0_1651070263812/topology @@ -0,0 +1,13 @@ +Topologies: + Sub-topology: 0 + Source: KSTREAM-SOURCE-0000000000 (topics: [test_topic]) + --> KSTREAM-TRANSFORMVALUES-0000000001 + Processor: KSTREAM-TRANSFORMVALUES-0000000001 (stores: []) + --> Project + <-- KSTREAM-SOURCE-0000000000 + Processor: Project (stores: []) + --> KSTREAM-SINK-0000000003 + <-- KSTREAM-TRANSFORMVALUES-0000000001 + Sink: KSTREAM-SINK-0000000003 (topic: OUTPUT) + <-- Project + diff --git a/ksqldb-functional-tests/src/test/resources/historical_plans/url_-_extract_a_host_from_a_URL_using_URL_EXTRACT_HOST_-_PROTOBUF/7.3.0_1651070263865/plan.json b/ksqldb-functional-tests/src/test/resources/historical_plans/url_-_extract_a_host_from_a_URL_using_URL_EXTRACT_HOST_-_PROTOBUF/7.3.0_1651070263865/plan.json new file mode 100644 index 000000000000..d3afd20e522d --- /dev/null +++ b/ksqldb-functional-tests/src/test/resources/historical_plans/url_-_extract_a_host_from_a_URL_using_URL_EXTRACT_HOST_-_PROTOBUF/7.3.0_1651070263865/plan.json @@ -0,0 +1,211 @@ +{ + "plan" : [ { + "@type" : "ksqlPlanV1", + "statementText" : "CREATE STREAM TEST (K STRING KEY, URL STRING) WITH (KAFKA_TOPIC='test_topic', KEY_FORMAT='KAFKA', VALUE_FORMAT='PROTOBUF');", + "ddlCommand" : { + "@type" : "createStreamV1", + "sourceName" : "TEST", + "schema" : "`K` STRING KEY, `URL` STRING", + "timestampColumn" : null, + "topicName" : "test_topic", + "formats" : { + "keyFormat" : { + "format" : "KAFKA", + "properties" : { } + }, + "valueFormat" : { + "format" : "PROTOBUF", + "properties" : { + "unwrapPrimitives" : "true" + } + } + }, + "windowInfo" : null, + "orReplace" : false, + "isSource" : false + }, + "queryPlan" : null + }, { + "@type" : "ksqlPlanV1", + "statementText" : "CREATE STREAM OUTPUT AS SELECT\n TEST.K K,\n URL_EXTRACT_HOST(TEST.URL) HOST\nFROM TEST TEST\nEMIT CHANGES", + "ddlCommand" : { + "@type" : "createStreamV1", + "sourceName" : "OUTPUT", + "schema" : "`K` STRING KEY, `HOST` STRING", + "timestampColumn" : null, + "topicName" : "OUTPUT", + "formats" : { + "keyFormat" : { + "format" : "KAFKA", + "properties" : { } + }, + "valueFormat" : { + "format" : "PROTOBUF", + "properties" : { + "unwrapPrimitives" : "true" + } + } + }, + "windowInfo" : null, + "orReplace" : false, + "isSource" : false + }, + "queryPlan" : { + "sources" : [ "TEST" ], + "sink" : "OUTPUT", + "physicalPlan" : { + "@type" : "streamSinkV1", + "properties" : { + "queryContext" : "OUTPUT" + }, + "source" : { + "@type" : "streamSelectV1", + "properties" : { + "queryContext" : "Project" + }, + "source" : { + "@type" : "streamSourceV1", + "properties" : { + "queryContext" : "KsqlTopic/Source" + }, + "topicName" : "test_topic", + "formats" : { + "keyFormat" : { + "format" : "KAFKA", + "properties" : { } + }, + "valueFormat" : { + "format" : "PROTOBUF", + "properties" : { + "unwrapPrimitives" : "true" + } + } + }, + "timestampColumn" : null, + "sourceSchema" : "`K` STRING KEY, `URL` STRING", + "pseudoColumnVersion" : 1 + }, + "keyColumnNames" : [ "K" ], + "selectedKeys" : null, + "selectExpressions" : [ "URL_EXTRACT_HOST(URL) AS HOST" ] + }, + "formats" : { + "keyFormat" : { + "format" : "KAFKA", + "properties" : { } + }, + "valueFormat" : { + "format" : "PROTOBUF", + "properties" : { + "unwrapPrimitives" : "true" + } + } + }, + "topicName" : "OUTPUT", + "timestampColumn" : null + }, + "queryId" : "CSAS_OUTPUT_0", + "runtimeId" : null + } + } ], + "configs" : { + "ksql.extension.dir" : "ext", + "ksql.streams.cache.max.bytes.buffering" : "0", + "metric.reporters" : "", + "ksql.query.status.running.threshold.seconds" : "300", + "ksql.connect.basic.auth.credentials.reload" : "false", + "ksql.output.topic.name.prefix" : "", + "ksql.query.pull.stream.enabled" : "true", + "ksql.query.push.v2.interpreter.enabled" : "true", + "ksql.queryanonymizer.logs_enabled" : "true", + "ksql.variable.substitution.enable" : "true", + "ksql.streams.shutdown.timeout.ms" : "300000", + "ksql.query.pull.max.allowed.offset.lag" : "9223372036854775807", + "ksql.query.pull.max.qps" : "2147483647", + "ksql.access.validator.enable" : "auto", + "ksql.streams.bootstrap.servers" : "localhost:0", + "ksql.query.pull.metrics.enabled" : "true", + "ksql.metrics.extension" : null, + "ksql.query.push.v2.alos.enabled" : "true", + "ksql.query.push.v2.max.hourly.bandwidth.megabytes" : "2147483647", + "ksql.query.pull.range.scan.enabled" : "true", + "ksql.transient.query.cleanup.service.initial.delay.seconds" : "600", + "ksql.hidden.topics" : "_confluent.*,__confluent.*,_schemas,__consumer_offsets,__transaction_state,connect-configs,connect-offsets,connect-status,connect-statuses", + "ksql.lambdas.enabled" : "true", + "ksql.source.table.materialization.enabled" : "true", + "ksql.query.pull.max.hourly.bandwidth.megabytes" : "2147483647", + "ksql.sink.window.change.log.additional.retention" : "1000000", + "ksql.query.persistent.active.limit" : "2147483647", + "ksql.persistence.wrap.single.values" : null, + "ksql.query.transient.max.bytes.buffering.total" : "-1", + "ksql.connect.basic.auth.credentials.source" : "NONE", + "ksql.schema.registry.url" : "schema_registry.url:0", + "ksql.properties.overrides.denylist" : "", + "ksql.service.id" : "some.ksql.service.id", + "ksql.query.push.v2.max.catchup.consumers" : "5", + "ksql.query.push.v2.enabled" : "false", + "ksql.transient.query.cleanup.service.enable" : "true", + "ksql.query.push.v2.metrics.enabled" : "true", + "ksql.rowpartition.rowoffset.enabled" : "true", + "ksql.streams.commit.interval.ms" : "2000", + "ksql.query.pull.table.scan.enabled" : "true", + "ksql.streams.auto.commit.interval.ms" : "0", + "ksql.streams.topology.optimization" : "all", + "ksql.endpoint.migrate.query" : "true", + "ksql.query.push.v2.registry.installed" : "false", + "ksql.streams.num.stream.threads" : "4", + "ksql.metrics.tags.custom" : "", + "ksql.query.push.v2.catchup.consumer.msg.window" : "50", + "ksql.runtime.feature.shared.enabled" : "false", + "ksql.udf.collect.metrics" : "false", + "ksql.new.query.planner.enabled" : "false", + "ksql.connect.request.headers.plugin" : null, + "ksql.security.extension.class" : null, + "ksql.transient.prefix" : "transient_", + "ksql.headers.columns.enabled" : "true", + "ksql.streams.default.deserialization.exception.handler" : "io.confluent.ksql.errors.LogMetricAndContinueExceptionHandler", + "ksql.connect.request.timeout.ms" : "5000", + "ksql.query.pull.enable.standby.reads" : "false", + "ksql.persistence.default.format.key" : "KAFKA", + "ksql.query.persistent.max.bytes.buffering.total" : "-1", + "ksql.query.error.max.queue.size" : "10", + "ksql.query.cleanup.shutdown.timeout.ms" : "30000", + "ksql.internal.topic.min.insync.replicas" : "1", + "ksql.internal.topic.replicas" : "1", + "ksql.insert.into.values.enabled" : "true", + "ksql.queryanonymizer.cluster_namespace" : null, + "ksql.create.or.replace.enabled" : "true", + "ksql.shared.runtimes.count" : "2", + "ksql.cast.strings.preserve.nulls" : "true", + "ksql.authorization.cache.max.entries" : "10000", + "ksql.pull.queries.enable" : "true", + "ksql.transient.query.cleanup.service.period.seconds" : "600", + "ksql.suppress.enabled" : "false", + "ksql.readonly.topics" : "_confluent.*,__confluent.*,_schemas,__consumer_offsets,__transaction_state,connect-configs,connect-offsets,connect-status,connect-statuses", + "ksql.connect.basic.auth.credentials.file" : "", + "ksql.authorization.cache.expiry.time.secs" : "30", + "ksql.query.retry.backoff.initial.ms" : "15000", + "ksql.query.pull.max.concurrent.requests" : "2147483647", + "ksql.streams.auto.offset.reset" : "earliest", + "ksql.connect.url" : "http://localhost:8083", + "ksql.query.push.v2.new.latest.delay.ms" : "5000", + "ksql.query.push.v2.latest.reset.age.ms" : "30000", + "ksql.streams.default.production.exception.handler" : "io.confluent.ksql.errors.ProductionExceptionHandlerUtil$LogAndFailProductionExceptionHandler", + "ksql.query.pull.interpreter.enabled" : "true", + "ksql.query.pull.limit.clause.enabled" : "true", + "ksql.query.pull.router.thread.pool.size" : "50", + "ksql.query.push.v2.continuation.tokens.enabled" : "false", + "ksql.query.retry.backoff.max.ms" : "900000", + "ksql.timestamp.throw.on.invalid" : "false", + "ksql.persistence.default.format.value" : null, + "ksql.udfs.enabled" : "true", + "ksql.udf.enable.security.manager" : "true", + "ksql.connect.worker.config" : "", + "ksql.nested.error.set.null" : "true", + "ksql.query.pull.thread.pool.size" : "50", + "ksql.persistent.prefix" : "query_", + "ksql.metastore.backup.location" : "", + "ksql.error.classifier.regex" : "", + "ksql.suppress.buffer.size.bytes" : "-1" + } +} \ No newline at end of file diff --git a/ksqldb-functional-tests/src/test/resources/historical_plans/url_-_extract_a_host_from_a_URL_using_URL_EXTRACT_HOST_-_PROTOBUF/7.3.0_1651070263865/spec.json b/ksqldb-functional-tests/src/test/resources/historical_plans/url_-_extract_a_host_from_a_URL_using_URL_EXTRACT_HOST_-_PROTOBUF/7.3.0_1651070263865/spec.json new file mode 100644 index 000000000000..7a09d78e492e --- /dev/null +++ b/ksqldb-functional-tests/src/test/resources/historical_plans/url_-_extract_a_host_from_a_URL_using_URL_EXTRACT_HOST_-_PROTOBUF/7.3.0_1651070263865/spec.json @@ -0,0 +1,130 @@ +{ + "version" : "7.3.0", + "timestamp" : 1651070263865, + "path" : "query-validation-tests/url.json", + "schemas" : { + "CSAS_OUTPUT_0.KsqlTopic.Source" : { + "schema" : "`K` STRING KEY, `URL` STRING", + "keyFormat" : { + "format" : "KAFKA" + }, + "valueFormat" : { + "format" : "PROTOBUF", + "properties" : { + "unwrapPrimitives" : "true" + } + } + }, + "CSAS_OUTPUT_0.OUTPUT" : { + "schema" : "`K` STRING KEY, `HOST` STRING", + "keyFormat" : { + "format" : "KAFKA" + }, + "valueFormat" : { + "format" : "PROTOBUF", + "properties" : { + "unwrapPrimitives" : "true" + } + } + } + }, + "testCase" : { + "name" : "extract a host from a URL using URL_EXTRACT_HOST - PROTOBUF", + "inputs" : [ { + "topic" : "test_topic", + "key" : "1", + "value" : { + "url" : "http://www.test.com/?query#fragment" + }, + "timestamp" : 0 + }, { + "topic" : "test_topic", + "key" : "2", + "value" : { + "url" : "http://test@confluent.io:8080" + }, + "timestamp" : 1 + } ], + "outputs" : [ { + "topic" : "OUTPUT", + "key" : "1", + "value" : { + "HOST" : "www.test.com" + }, + "timestamp" : 0 + }, { + "topic" : "OUTPUT", + "key" : "2", + "value" : { + "HOST" : "confluent.io" + }, + "timestamp" : 1 + } ], + "topics" : [ { + "name" : "OUTPUT", + "replicas" : 1, + "numPartitions" : 4 + }, { + "name" : "test_topic", + "valueSchema" : "syntax = \"proto3\";\n\nmessage ConnectDefault1 {\n string URL = 1;\n}\n", + "valueFormat" : "PROTOBUF", + "replicas" : 1, + "numPartitions" : 4 + } ], + "statements" : [ "CREATE STREAM TEST (K STRING KEY, url VARCHAR) WITH (kafka_topic='test_topic', value_format='PROTOBUF');", "CREATE STREAM OUTPUT AS SELECT K, URL_EXTRACT_HOST(url) as HOST FROM TEST;" ], + "post" : { + "sources" : [ { + "name" : "OUTPUT", + "type" : "STREAM", + "schema" : "`K` STRING KEY, `HOST` STRING", + "keyFormat" : { + "format" : "KAFKA" + }, + "valueFormat" : "PROTOBUF", + "keyFeatures" : [ ], + "valueFeatures" : [ ], + "isSource" : false + }, { + "name" : "TEST", + "type" : "STREAM", + "schema" : "`K` STRING KEY, `URL` STRING", + "keyFormat" : { + "format" : "KAFKA" + }, + "valueFormat" : "PROTOBUF", + "keyFeatures" : [ ], + "valueFeatures" : [ ], + "isSource" : false + } ], + "topics" : { + "topics" : [ { + "name" : "test_topic", + "keyFormat" : { + "format" : "KAFKA" + }, + "valueFormat" : { + "format" : "PROTOBUF", + "properties" : { + "unwrapPrimitives" : "true" + } + }, + "partitions" : 4, + "valueSchema" : "syntax = \"proto3\";\n\nmessage ConnectDefault1 {\n string URL = 1;\n}\n" + }, { + "name" : "OUTPUT", + "keyFormat" : { + "format" : "KAFKA" + }, + "valueFormat" : { + "format" : "PROTOBUF", + "properties" : { + "unwrapPrimitives" : "true" + } + }, + "partitions" : 4, + "valueSchema" : "syntax = \"proto3\";\n\nmessage ConnectDefault1 {\n string HOST = 1;\n}\n" + } ] + } + } + } +} \ No newline at end of file diff --git a/ksqldb-functional-tests/src/test/resources/historical_plans/url_-_extract_a_host_from_a_URL_using_URL_EXTRACT_HOST_-_PROTOBUF/7.3.0_1651070263865/topology b/ksqldb-functional-tests/src/test/resources/historical_plans/url_-_extract_a_host_from_a_URL_using_URL_EXTRACT_HOST_-_PROTOBUF/7.3.0_1651070263865/topology new file mode 100644 index 000000000000..441a8f282644 --- /dev/null +++ b/ksqldb-functional-tests/src/test/resources/historical_plans/url_-_extract_a_host_from_a_URL_using_URL_EXTRACT_HOST_-_PROTOBUF/7.3.0_1651070263865/topology @@ -0,0 +1,13 @@ +Topologies: + Sub-topology: 0 + Source: KSTREAM-SOURCE-0000000000 (topics: [test_topic]) + --> KSTREAM-TRANSFORMVALUES-0000000001 + Processor: KSTREAM-TRANSFORMVALUES-0000000001 (stores: []) + --> Project + <-- KSTREAM-SOURCE-0000000000 + Processor: Project (stores: []) + --> KSTREAM-SINK-0000000003 + <-- KSTREAM-TRANSFORMVALUES-0000000001 + Sink: KSTREAM-SINK-0000000003 (topic: OUTPUT) + <-- Project + diff --git a/ksqldb-functional-tests/src/test/resources/historical_plans/url_-_extract_a_host_from_a_URL_using_URL_EXTRACT_HOST_-_PROTOBUF_NOSR/7.3.0_1651070263890/plan.json b/ksqldb-functional-tests/src/test/resources/historical_plans/url_-_extract_a_host_from_a_URL_using_URL_EXTRACT_HOST_-_PROTOBUF_NOSR/7.3.0_1651070263890/plan.json new file mode 100644 index 000000000000..835004ce50d2 --- /dev/null +++ b/ksqldb-functional-tests/src/test/resources/historical_plans/url_-_extract_a_host_from_a_URL_using_URL_EXTRACT_HOST_-_PROTOBUF_NOSR/7.3.0_1651070263890/plan.json @@ -0,0 +1,203 @@ +{ + "plan" : [ { + "@type" : "ksqlPlanV1", + "statementText" : "CREATE STREAM TEST (K STRING KEY, URL STRING) WITH (KAFKA_TOPIC='test_topic', KEY_FORMAT='KAFKA', VALUE_FORMAT='PROTOBUF_NOSR');", + "ddlCommand" : { + "@type" : "createStreamV1", + "sourceName" : "TEST", + "schema" : "`K` STRING KEY, `URL` STRING", + "timestampColumn" : null, + "topicName" : "test_topic", + "formats" : { + "keyFormat" : { + "format" : "KAFKA", + "properties" : { } + }, + "valueFormat" : { + "format" : "PROTOBUF_NOSR", + "properties" : { } + } + }, + "windowInfo" : null, + "orReplace" : false, + "isSource" : false + }, + "queryPlan" : null + }, { + "@type" : "ksqlPlanV1", + "statementText" : "CREATE STREAM OUTPUT AS SELECT\n TEST.K K,\n URL_EXTRACT_HOST(TEST.URL) HOST\nFROM TEST TEST\nEMIT CHANGES", + "ddlCommand" : { + "@type" : "createStreamV1", + "sourceName" : "OUTPUT", + "schema" : "`K` STRING KEY, `HOST` STRING", + "timestampColumn" : null, + "topicName" : "OUTPUT", + "formats" : { + "keyFormat" : { + "format" : "KAFKA", + "properties" : { } + }, + "valueFormat" : { + "format" : "PROTOBUF_NOSR", + "properties" : { } + } + }, + "windowInfo" : null, + "orReplace" : false, + "isSource" : false + }, + "queryPlan" : { + "sources" : [ "TEST" ], + "sink" : "OUTPUT", + "physicalPlan" : { + "@type" : "streamSinkV1", + "properties" : { + "queryContext" : "OUTPUT" + }, + "source" : { + "@type" : "streamSelectV1", + "properties" : { + "queryContext" : "Project" + }, + "source" : { + "@type" : "streamSourceV1", + "properties" : { + "queryContext" : "KsqlTopic/Source" + }, + "topicName" : "test_topic", + "formats" : { + "keyFormat" : { + "format" : "KAFKA", + "properties" : { } + }, + "valueFormat" : { + "format" : "PROTOBUF_NOSR", + "properties" : { } + } + }, + "timestampColumn" : null, + "sourceSchema" : "`K` STRING KEY, `URL` STRING", + "pseudoColumnVersion" : 1 + }, + "keyColumnNames" : [ "K" ], + "selectedKeys" : null, + "selectExpressions" : [ "URL_EXTRACT_HOST(URL) AS HOST" ] + }, + "formats" : { + "keyFormat" : { + "format" : "KAFKA", + "properties" : { } + }, + "valueFormat" : { + "format" : "PROTOBUF_NOSR", + "properties" : { } + } + }, + "topicName" : "OUTPUT", + "timestampColumn" : null + }, + "queryId" : "CSAS_OUTPUT_0", + "runtimeId" : null + } + } ], + "configs" : { + "ksql.extension.dir" : "ext", + "ksql.streams.cache.max.bytes.buffering" : "0", + "metric.reporters" : "", + "ksql.query.status.running.threshold.seconds" : "300", + "ksql.connect.basic.auth.credentials.reload" : "false", + "ksql.output.topic.name.prefix" : "", + "ksql.query.pull.stream.enabled" : "true", + "ksql.query.push.v2.interpreter.enabled" : "true", + "ksql.queryanonymizer.logs_enabled" : "true", + "ksql.variable.substitution.enable" : "true", + "ksql.streams.shutdown.timeout.ms" : "300000", + "ksql.query.pull.max.allowed.offset.lag" : "9223372036854775807", + "ksql.query.pull.max.qps" : "2147483647", + "ksql.access.validator.enable" : "auto", + "ksql.streams.bootstrap.servers" : "localhost:0", + "ksql.query.pull.metrics.enabled" : "true", + "ksql.metrics.extension" : null, + "ksql.query.push.v2.alos.enabled" : "true", + "ksql.query.push.v2.max.hourly.bandwidth.megabytes" : "2147483647", + "ksql.query.pull.range.scan.enabled" : "true", + "ksql.transient.query.cleanup.service.initial.delay.seconds" : "600", + "ksql.hidden.topics" : "_confluent.*,__confluent.*,_schemas,__consumer_offsets,__transaction_state,connect-configs,connect-offsets,connect-status,connect-statuses", + "ksql.lambdas.enabled" : "true", + "ksql.source.table.materialization.enabled" : "true", + "ksql.query.pull.max.hourly.bandwidth.megabytes" : "2147483647", + "ksql.sink.window.change.log.additional.retention" : "1000000", + "ksql.query.persistent.active.limit" : "2147483647", + "ksql.persistence.wrap.single.values" : null, + "ksql.query.transient.max.bytes.buffering.total" : "-1", + "ksql.connect.basic.auth.credentials.source" : "NONE", + "ksql.schema.registry.url" : "schema_registry.url:0", + "ksql.properties.overrides.denylist" : "", + "ksql.service.id" : "some.ksql.service.id", + "ksql.query.push.v2.max.catchup.consumers" : "5", + "ksql.query.push.v2.enabled" : "false", + "ksql.transient.query.cleanup.service.enable" : "true", + "ksql.query.push.v2.metrics.enabled" : "true", + "ksql.rowpartition.rowoffset.enabled" : "true", + "ksql.streams.commit.interval.ms" : "2000", + "ksql.query.pull.table.scan.enabled" : "true", + "ksql.streams.auto.commit.interval.ms" : "0", + "ksql.streams.topology.optimization" : "all", + "ksql.endpoint.migrate.query" : "true", + "ksql.query.push.v2.registry.installed" : "false", + "ksql.streams.num.stream.threads" : "4", + "ksql.metrics.tags.custom" : "", + "ksql.query.push.v2.catchup.consumer.msg.window" : "50", + "ksql.runtime.feature.shared.enabled" : "false", + "ksql.udf.collect.metrics" : "false", + "ksql.new.query.planner.enabled" : "false", + "ksql.connect.request.headers.plugin" : null, + "ksql.security.extension.class" : null, + "ksql.transient.prefix" : "transient_", + "ksql.headers.columns.enabled" : "true", + "ksql.streams.default.deserialization.exception.handler" : "io.confluent.ksql.errors.LogMetricAndContinueExceptionHandler", + "ksql.connect.request.timeout.ms" : "5000", + "ksql.query.pull.enable.standby.reads" : "false", + "ksql.persistence.default.format.key" : "KAFKA", + "ksql.query.persistent.max.bytes.buffering.total" : "-1", + "ksql.query.error.max.queue.size" : "10", + "ksql.query.cleanup.shutdown.timeout.ms" : "30000", + "ksql.internal.topic.min.insync.replicas" : "1", + "ksql.internal.topic.replicas" : "1", + "ksql.insert.into.values.enabled" : "true", + "ksql.queryanonymizer.cluster_namespace" : null, + "ksql.create.or.replace.enabled" : "true", + "ksql.shared.runtimes.count" : "2", + "ksql.cast.strings.preserve.nulls" : "true", + "ksql.authorization.cache.max.entries" : "10000", + "ksql.pull.queries.enable" : "true", + "ksql.transient.query.cleanup.service.period.seconds" : "600", + "ksql.suppress.enabled" : "false", + "ksql.readonly.topics" : "_confluent.*,__confluent.*,_schemas,__consumer_offsets,__transaction_state,connect-configs,connect-offsets,connect-status,connect-statuses", + "ksql.connect.basic.auth.credentials.file" : "", + "ksql.authorization.cache.expiry.time.secs" : "30", + "ksql.query.retry.backoff.initial.ms" : "15000", + "ksql.query.pull.max.concurrent.requests" : "2147483647", + "ksql.streams.auto.offset.reset" : "earliest", + "ksql.connect.url" : "http://localhost:8083", + "ksql.query.push.v2.new.latest.delay.ms" : "5000", + "ksql.query.push.v2.latest.reset.age.ms" : "30000", + "ksql.streams.default.production.exception.handler" : "io.confluent.ksql.errors.ProductionExceptionHandlerUtil$LogAndFailProductionExceptionHandler", + "ksql.query.pull.interpreter.enabled" : "true", + "ksql.query.pull.limit.clause.enabled" : "true", + "ksql.query.pull.router.thread.pool.size" : "50", + "ksql.query.push.v2.continuation.tokens.enabled" : "false", + "ksql.query.retry.backoff.max.ms" : "900000", + "ksql.timestamp.throw.on.invalid" : "false", + "ksql.persistence.default.format.value" : null, + "ksql.udfs.enabled" : "true", + "ksql.udf.enable.security.manager" : "true", + "ksql.connect.worker.config" : "", + "ksql.nested.error.set.null" : "true", + "ksql.query.pull.thread.pool.size" : "50", + "ksql.persistent.prefix" : "query_", + "ksql.metastore.backup.location" : "", + "ksql.error.classifier.regex" : "", + "ksql.suppress.buffer.size.bytes" : "-1" + } +} \ No newline at end of file diff --git a/ksqldb-functional-tests/src/test/resources/historical_plans/url_-_extract_a_host_from_a_URL_using_URL_EXTRACT_HOST_-_PROTOBUF_NOSR/7.3.0_1651070263890/spec.json b/ksqldb-functional-tests/src/test/resources/historical_plans/url_-_extract_a_host_from_a_URL_using_URL_EXTRACT_HOST_-_PROTOBUF_NOSR/7.3.0_1651070263890/spec.json new file mode 100644 index 000000000000..73219b0a3d7c --- /dev/null +++ b/ksqldb-functional-tests/src/test/resources/historical_plans/url_-_extract_a_host_from_a_URL_using_URL_EXTRACT_HOST_-_PROTOBUF_NOSR/7.3.0_1651070263890/spec.json @@ -0,0 +1,114 @@ +{ + "version" : "7.3.0", + "timestamp" : 1651070263890, + "path" : "query-validation-tests/url.json", + "schemas" : { + "CSAS_OUTPUT_0.KsqlTopic.Source" : { + "schema" : "`K` STRING KEY, `URL` STRING", + "keyFormat" : { + "format" : "KAFKA" + }, + "valueFormat" : { + "format" : "PROTOBUF_NOSR" + } + }, + "CSAS_OUTPUT_0.OUTPUT" : { + "schema" : "`K` STRING KEY, `HOST` STRING", + "keyFormat" : { + "format" : "KAFKA" + }, + "valueFormat" : { + "format" : "PROTOBUF_NOSR" + } + } + }, + "testCase" : { + "name" : "extract a host from a URL using URL_EXTRACT_HOST - PROTOBUF_NOSR", + "inputs" : [ { + "topic" : "test_topic", + "key" : "1", + "value" : { + "url" : "http://www.test.com/?query#fragment" + }, + "timestamp" : 0 + }, { + "topic" : "test_topic", + "key" : "2", + "value" : { + "url" : "http://test@confluent.io:8080" + }, + "timestamp" : 1 + } ], + "outputs" : [ { + "topic" : "OUTPUT", + "key" : "1", + "value" : { + "HOST" : "www.test.com" + }, + "timestamp" : 0 + }, { + "topic" : "OUTPUT", + "key" : "2", + "value" : { + "HOST" : "confluent.io" + }, + "timestamp" : 1 + } ], + "topics" : [ { + "name" : "OUTPUT", + "replicas" : 1, + "numPartitions" : 4 + }, { + "name" : "test_topic", + "replicas" : 1, + "numPartitions" : 4 + } ], + "statements" : [ "CREATE STREAM TEST (K STRING KEY, url VARCHAR) WITH (kafka_topic='test_topic', value_format='PROTOBUF_NOSR');", "CREATE STREAM OUTPUT AS SELECT K, URL_EXTRACT_HOST(url) as HOST FROM TEST;" ], + "post" : { + "sources" : [ { + "name" : "OUTPUT", + "type" : "STREAM", + "schema" : "`K` STRING KEY, `HOST` STRING", + "keyFormat" : { + "format" : "KAFKA" + }, + "valueFormat" : "PROTOBUF_NOSR", + "keyFeatures" : [ ], + "valueFeatures" : [ ], + "isSource" : false + }, { + "name" : "TEST", + "type" : "STREAM", + "schema" : "`K` STRING KEY, `URL` STRING", + "keyFormat" : { + "format" : "KAFKA" + }, + "valueFormat" : "PROTOBUF_NOSR", + "keyFeatures" : [ ], + "valueFeatures" : [ ], + "isSource" : false + } ], + "topics" : { + "topics" : [ { + "name" : "test_topic", + "keyFormat" : { + "format" : "KAFKA" + }, + "valueFormat" : { + "format" : "PROTOBUF_NOSR" + }, + "partitions" : 4 + }, { + "name" : "OUTPUT", + "keyFormat" : { + "format" : "KAFKA" + }, + "valueFormat" : { + "format" : "PROTOBUF_NOSR" + }, + "partitions" : 4 + } ] + } + } + } +} \ No newline at end of file diff --git a/ksqldb-functional-tests/src/test/resources/historical_plans/url_-_extract_a_host_from_a_URL_using_URL_EXTRACT_HOST_-_PROTOBUF_NOSR/7.3.0_1651070263890/topology b/ksqldb-functional-tests/src/test/resources/historical_plans/url_-_extract_a_host_from_a_URL_using_URL_EXTRACT_HOST_-_PROTOBUF_NOSR/7.3.0_1651070263890/topology new file mode 100644 index 000000000000..441a8f282644 --- /dev/null +++ b/ksqldb-functional-tests/src/test/resources/historical_plans/url_-_extract_a_host_from_a_URL_using_URL_EXTRACT_HOST_-_PROTOBUF_NOSR/7.3.0_1651070263890/topology @@ -0,0 +1,13 @@ +Topologies: + Sub-topology: 0 + Source: KSTREAM-SOURCE-0000000000 (topics: [test_topic]) + --> KSTREAM-TRANSFORMVALUES-0000000001 + Processor: KSTREAM-TRANSFORMVALUES-0000000001 (stores: []) + --> Project + <-- KSTREAM-SOURCE-0000000000 + Processor: Project (stores: []) + --> KSTREAM-SINK-0000000003 + <-- KSTREAM-TRANSFORMVALUES-0000000001 + Sink: KSTREAM-SINK-0000000003 (topic: OUTPUT) + <-- Project + diff --git a/ksqldb-functional-tests/src/test/resources/historical_plans/url_-_extract_a_parameter_from_a_URL_using_URL_EXTRACT_PARAMETER_-_PROTOBUF/7.3.0_1651070264511/plan.json b/ksqldb-functional-tests/src/test/resources/historical_plans/url_-_extract_a_parameter_from_a_URL_using_URL_EXTRACT_PARAMETER_-_PROTOBUF/7.3.0_1651070264511/plan.json new file mode 100644 index 000000000000..bd6928503b3b --- /dev/null +++ b/ksqldb-functional-tests/src/test/resources/historical_plans/url_-_extract_a_parameter_from_a_URL_using_URL_EXTRACT_PARAMETER_-_PROTOBUF/7.3.0_1651070264511/plan.json @@ -0,0 +1,211 @@ +{ + "plan" : [ { + "@type" : "ksqlPlanV1", + "statementText" : "CREATE STREAM TEST (K STRING KEY, URL STRING) WITH (KAFKA_TOPIC='test_topic', KEY_FORMAT='KAFKA', VALUE_FORMAT='PROTOBUF');", + "ddlCommand" : { + "@type" : "createStreamV1", + "sourceName" : "TEST", + "schema" : "`K` STRING KEY, `URL` STRING", + "timestampColumn" : null, + "topicName" : "test_topic", + "formats" : { + "keyFormat" : { + "format" : "KAFKA", + "properties" : { } + }, + "valueFormat" : { + "format" : "PROTOBUF", + "properties" : { + "unwrapPrimitives" : "true" + } + } + }, + "windowInfo" : null, + "orReplace" : false, + "isSource" : false + }, + "queryPlan" : null + }, { + "@type" : "ksqlPlanV1", + "statementText" : "CREATE STREAM OUTPUT AS SELECT\n TEST.K K,\n URL_EXTRACT_PARAMETER(TEST.URL, 'one') PARAM_A,\n URL_EXTRACT_PARAMETER(TEST.URL, 'two') PARAM_B\nFROM TEST TEST\nEMIT CHANGES", + "ddlCommand" : { + "@type" : "createStreamV1", + "sourceName" : "OUTPUT", + "schema" : "`K` STRING KEY, `PARAM_A` STRING, `PARAM_B` STRING", + "timestampColumn" : null, + "topicName" : "OUTPUT", + "formats" : { + "keyFormat" : { + "format" : "KAFKA", + "properties" : { } + }, + "valueFormat" : { + "format" : "PROTOBUF", + "properties" : { + "unwrapPrimitives" : "true" + } + } + }, + "windowInfo" : null, + "orReplace" : false, + "isSource" : false + }, + "queryPlan" : { + "sources" : [ "TEST" ], + "sink" : "OUTPUT", + "physicalPlan" : { + "@type" : "streamSinkV1", + "properties" : { + "queryContext" : "OUTPUT" + }, + "source" : { + "@type" : "streamSelectV1", + "properties" : { + "queryContext" : "Project" + }, + "source" : { + "@type" : "streamSourceV1", + "properties" : { + "queryContext" : "KsqlTopic/Source" + }, + "topicName" : "test_topic", + "formats" : { + "keyFormat" : { + "format" : "KAFKA", + "properties" : { } + }, + "valueFormat" : { + "format" : "PROTOBUF", + "properties" : { + "unwrapPrimitives" : "true" + } + } + }, + "timestampColumn" : null, + "sourceSchema" : "`K` STRING KEY, `URL` STRING", + "pseudoColumnVersion" : 1 + }, + "keyColumnNames" : [ "K" ], + "selectedKeys" : null, + "selectExpressions" : [ "URL_EXTRACT_PARAMETER(URL, 'one') AS PARAM_A", "URL_EXTRACT_PARAMETER(URL, 'two') AS PARAM_B" ] + }, + "formats" : { + "keyFormat" : { + "format" : "KAFKA", + "properties" : { } + }, + "valueFormat" : { + "format" : "PROTOBUF", + "properties" : { + "unwrapPrimitives" : "true" + } + } + }, + "topicName" : "OUTPUT", + "timestampColumn" : null + }, + "queryId" : "CSAS_OUTPUT_0", + "runtimeId" : null + } + } ], + "configs" : { + "ksql.extension.dir" : "ext", + "ksql.streams.cache.max.bytes.buffering" : "0", + "metric.reporters" : "", + "ksql.query.status.running.threshold.seconds" : "300", + "ksql.connect.basic.auth.credentials.reload" : "false", + "ksql.output.topic.name.prefix" : "", + "ksql.query.pull.stream.enabled" : "true", + "ksql.query.push.v2.interpreter.enabled" : "true", + "ksql.queryanonymizer.logs_enabled" : "true", + "ksql.variable.substitution.enable" : "true", + "ksql.streams.shutdown.timeout.ms" : "300000", + "ksql.query.pull.max.allowed.offset.lag" : "9223372036854775807", + "ksql.query.pull.max.qps" : "2147483647", + "ksql.access.validator.enable" : "auto", + "ksql.streams.bootstrap.servers" : "localhost:0", + "ksql.query.pull.metrics.enabled" : "true", + "ksql.metrics.extension" : null, + "ksql.query.push.v2.alos.enabled" : "true", + "ksql.query.push.v2.max.hourly.bandwidth.megabytes" : "2147483647", + "ksql.query.pull.range.scan.enabled" : "true", + "ksql.transient.query.cleanup.service.initial.delay.seconds" : "600", + "ksql.hidden.topics" : "_confluent.*,__confluent.*,_schemas,__consumer_offsets,__transaction_state,connect-configs,connect-offsets,connect-status,connect-statuses", + "ksql.lambdas.enabled" : "true", + "ksql.source.table.materialization.enabled" : "true", + "ksql.query.pull.max.hourly.bandwidth.megabytes" : "2147483647", + "ksql.sink.window.change.log.additional.retention" : "1000000", + "ksql.query.persistent.active.limit" : "2147483647", + "ksql.persistence.wrap.single.values" : null, + "ksql.query.transient.max.bytes.buffering.total" : "-1", + "ksql.connect.basic.auth.credentials.source" : "NONE", + "ksql.schema.registry.url" : "schema_registry.url:0", + "ksql.properties.overrides.denylist" : "", + "ksql.service.id" : "some.ksql.service.id", + "ksql.query.push.v2.max.catchup.consumers" : "5", + "ksql.query.push.v2.enabled" : "false", + "ksql.transient.query.cleanup.service.enable" : "true", + "ksql.query.push.v2.metrics.enabled" : "true", + "ksql.rowpartition.rowoffset.enabled" : "true", + "ksql.streams.commit.interval.ms" : "2000", + "ksql.query.pull.table.scan.enabled" : "true", + "ksql.streams.auto.commit.interval.ms" : "0", + "ksql.streams.topology.optimization" : "all", + "ksql.endpoint.migrate.query" : "true", + "ksql.query.push.v2.registry.installed" : "false", + "ksql.streams.num.stream.threads" : "4", + "ksql.metrics.tags.custom" : "", + "ksql.query.push.v2.catchup.consumer.msg.window" : "50", + "ksql.runtime.feature.shared.enabled" : "false", + "ksql.udf.collect.metrics" : "false", + "ksql.new.query.planner.enabled" : "false", + "ksql.connect.request.headers.plugin" : null, + "ksql.security.extension.class" : null, + "ksql.transient.prefix" : "transient_", + "ksql.headers.columns.enabled" : "true", + "ksql.streams.default.deserialization.exception.handler" : "io.confluent.ksql.errors.LogMetricAndContinueExceptionHandler", + "ksql.connect.request.timeout.ms" : "5000", + "ksql.query.pull.enable.standby.reads" : "false", + "ksql.persistence.default.format.key" : "KAFKA", + "ksql.query.persistent.max.bytes.buffering.total" : "-1", + "ksql.query.error.max.queue.size" : "10", + "ksql.query.cleanup.shutdown.timeout.ms" : "30000", + "ksql.internal.topic.min.insync.replicas" : "1", + "ksql.internal.topic.replicas" : "1", + "ksql.insert.into.values.enabled" : "true", + "ksql.queryanonymizer.cluster_namespace" : null, + "ksql.create.or.replace.enabled" : "true", + "ksql.shared.runtimes.count" : "2", + "ksql.cast.strings.preserve.nulls" : "true", + "ksql.authorization.cache.max.entries" : "10000", + "ksql.pull.queries.enable" : "true", + "ksql.transient.query.cleanup.service.period.seconds" : "600", + "ksql.suppress.enabled" : "false", + "ksql.readonly.topics" : "_confluent.*,__confluent.*,_schemas,__consumer_offsets,__transaction_state,connect-configs,connect-offsets,connect-status,connect-statuses", + "ksql.connect.basic.auth.credentials.file" : "", + "ksql.authorization.cache.expiry.time.secs" : "30", + "ksql.query.retry.backoff.initial.ms" : "15000", + "ksql.query.pull.max.concurrent.requests" : "2147483647", + "ksql.streams.auto.offset.reset" : "earliest", + "ksql.connect.url" : "http://localhost:8083", + "ksql.query.push.v2.new.latest.delay.ms" : "5000", + "ksql.query.push.v2.latest.reset.age.ms" : "30000", + "ksql.streams.default.production.exception.handler" : "io.confluent.ksql.errors.ProductionExceptionHandlerUtil$LogAndFailProductionExceptionHandler", + "ksql.query.pull.interpreter.enabled" : "true", + "ksql.query.pull.limit.clause.enabled" : "true", + "ksql.query.pull.router.thread.pool.size" : "50", + "ksql.query.push.v2.continuation.tokens.enabled" : "false", + "ksql.query.retry.backoff.max.ms" : "900000", + "ksql.timestamp.throw.on.invalid" : "false", + "ksql.persistence.default.format.value" : null, + "ksql.udfs.enabled" : "true", + "ksql.udf.enable.security.manager" : "true", + "ksql.connect.worker.config" : "", + "ksql.nested.error.set.null" : "true", + "ksql.query.pull.thread.pool.size" : "50", + "ksql.persistent.prefix" : "query_", + "ksql.metastore.backup.location" : "", + "ksql.error.classifier.regex" : "", + "ksql.suppress.buffer.size.bytes" : "-1" + } +} \ No newline at end of file diff --git a/ksqldb-functional-tests/src/test/resources/historical_plans/url_-_extract_a_parameter_from_a_URL_using_URL_EXTRACT_PARAMETER_-_PROTOBUF/7.3.0_1651070264511/spec.json b/ksqldb-functional-tests/src/test/resources/historical_plans/url_-_extract_a_parameter_from_a_URL_using_URL_EXTRACT_PARAMETER_-_PROTOBUF/7.3.0_1651070264511/spec.json new file mode 100644 index 000000000000..52f5d39fcc72 --- /dev/null +++ b/ksqldb-functional-tests/src/test/resources/historical_plans/url_-_extract_a_parameter_from_a_URL_using_URL_EXTRACT_PARAMETER_-_PROTOBUF/7.3.0_1651070264511/spec.json @@ -0,0 +1,117 @@ +{ + "version" : "7.3.0", + "timestamp" : 1651070264511, + "path" : "query-validation-tests/url.json", + "schemas" : { + "CSAS_OUTPUT_0.KsqlTopic.Source" : { + "schema" : "`K` STRING KEY, `URL` STRING", + "keyFormat" : { + "format" : "KAFKA" + }, + "valueFormat" : { + "format" : "PROTOBUF", + "properties" : { + "unwrapPrimitives" : "true" + } + } + }, + "CSAS_OUTPUT_0.OUTPUT" : { + "schema" : "`K` STRING KEY, `PARAM_A` STRING, `PARAM_B` STRING", + "keyFormat" : { + "format" : "KAFKA" + }, + "valueFormat" : { + "format" : "PROTOBUF", + "properties" : { + "unwrapPrimitives" : "true" + } + } + } + }, + "testCase" : { + "name" : "extract a parameter from a URL using URL_EXTRACT_PARAMETER - PROTOBUF", + "inputs" : [ { + "topic" : "test_topic", + "key" : "1", + "value" : { + "url" : "http://www.test.com/?one=a&two=b&three" + }, + "timestamp" : 0 + } ], + "outputs" : [ { + "topic" : "OUTPUT", + "key" : "1", + "value" : { + "PARAM_A" : "a", + "PARAM_B" : "b" + }, + "timestamp" : 0 + } ], + "topics" : [ { + "name" : "OUTPUT", + "replicas" : 1, + "numPartitions" : 4 + }, { + "name" : "test_topic", + "valueSchema" : "syntax = \"proto3\";\n\nmessage ConnectDefault1 {\n string URL = 1;\n}\n", + "valueFormat" : "PROTOBUF", + "replicas" : 1, + "numPartitions" : 4 + } ], + "statements" : [ "CREATE STREAM TEST (K STRING KEY, url VARCHAR) WITH (kafka_topic='test_topic', value_format='PROTOBUF');", "CREATE STREAM OUTPUT AS SELECT K, URL_EXTRACT_PARAMETER(url,'one') as PARAM_A, URL_EXTRACT_PARAMETER(url,'two') as PARAM_B FROM TEST;" ], + "post" : { + "sources" : [ { + "name" : "OUTPUT", + "type" : "STREAM", + "schema" : "`K` STRING KEY, `PARAM_A` STRING, `PARAM_B` STRING", + "keyFormat" : { + "format" : "KAFKA" + }, + "valueFormat" : "PROTOBUF", + "keyFeatures" : [ ], + "valueFeatures" : [ ], + "isSource" : false + }, { + "name" : "TEST", + "type" : "STREAM", + "schema" : "`K` STRING KEY, `URL` STRING", + "keyFormat" : { + "format" : "KAFKA" + }, + "valueFormat" : "PROTOBUF", + "keyFeatures" : [ ], + "valueFeatures" : [ ], + "isSource" : false + } ], + "topics" : { + "topics" : [ { + "name" : "test_topic", + "keyFormat" : { + "format" : "KAFKA" + }, + "valueFormat" : { + "format" : "PROTOBUF", + "properties" : { + "unwrapPrimitives" : "true" + } + }, + "partitions" : 4, + "valueSchema" : "syntax = \"proto3\";\n\nmessage ConnectDefault1 {\n string URL = 1;\n}\n" + }, { + "name" : "OUTPUT", + "keyFormat" : { + "format" : "KAFKA" + }, + "valueFormat" : { + "format" : "PROTOBUF", + "properties" : { + "unwrapPrimitives" : "true" + } + }, + "partitions" : 4, + "valueSchema" : "syntax = \"proto3\";\n\nmessage ConnectDefault1 {\n string PARAM_A = 1;\n string PARAM_B = 2;\n}\n" + } ] + } + } + } +} \ No newline at end of file diff --git a/ksqldb-functional-tests/src/test/resources/historical_plans/url_-_extract_a_parameter_from_a_URL_using_URL_EXTRACT_PARAMETER_-_PROTOBUF/7.3.0_1651070264511/topology b/ksqldb-functional-tests/src/test/resources/historical_plans/url_-_extract_a_parameter_from_a_URL_using_URL_EXTRACT_PARAMETER_-_PROTOBUF/7.3.0_1651070264511/topology new file mode 100644 index 000000000000..441a8f282644 --- /dev/null +++ b/ksqldb-functional-tests/src/test/resources/historical_plans/url_-_extract_a_parameter_from_a_URL_using_URL_EXTRACT_PARAMETER_-_PROTOBUF/7.3.0_1651070264511/topology @@ -0,0 +1,13 @@ +Topologies: + Sub-topology: 0 + Source: KSTREAM-SOURCE-0000000000 (topics: [test_topic]) + --> KSTREAM-TRANSFORMVALUES-0000000001 + Processor: KSTREAM-TRANSFORMVALUES-0000000001 (stores: []) + --> Project + <-- KSTREAM-SOURCE-0000000000 + Processor: Project (stores: []) + --> KSTREAM-SINK-0000000003 + <-- KSTREAM-TRANSFORMVALUES-0000000001 + Sink: KSTREAM-SINK-0000000003 (topic: OUTPUT) + <-- Project + diff --git a/ksqldb-functional-tests/src/test/resources/historical_plans/url_-_extract_a_parameter_from_a_URL_using_URL_EXTRACT_PARAMETER_-_PROTOBUF_NOSR/7.3.0_1651070264537/plan.json b/ksqldb-functional-tests/src/test/resources/historical_plans/url_-_extract_a_parameter_from_a_URL_using_URL_EXTRACT_PARAMETER_-_PROTOBUF_NOSR/7.3.0_1651070264537/plan.json new file mode 100644 index 000000000000..ade04e24d7dc --- /dev/null +++ b/ksqldb-functional-tests/src/test/resources/historical_plans/url_-_extract_a_parameter_from_a_URL_using_URL_EXTRACT_PARAMETER_-_PROTOBUF_NOSR/7.3.0_1651070264537/plan.json @@ -0,0 +1,203 @@ +{ + "plan" : [ { + "@type" : "ksqlPlanV1", + "statementText" : "CREATE STREAM TEST (K STRING KEY, URL STRING) WITH (KAFKA_TOPIC='test_topic', KEY_FORMAT='KAFKA', VALUE_FORMAT='PROTOBUF_NOSR');", + "ddlCommand" : { + "@type" : "createStreamV1", + "sourceName" : "TEST", + "schema" : "`K` STRING KEY, `URL` STRING", + "timestampColumn" : null, + "topicName" : "test_topic", + "formats" : { + "keyFormat" : { + "format" : "KAFKA", + "properties" : { } + }, + "valueFormat" : { + "format" : "PROTOBUF_NOSR", + "properties" : { } + } + }, + "windowInfo" : null, + "orReplace" : false, + "isSource" : false + }, + "queryPlan" : null + }, { + "@type" : "ksqlPlanV1", + "statementText" : "CREATE STREAM OUTPUT AS SELECT\n TEST.K K,\n URL_EXTRACT_PARAMETER(TEST.URL, 'one') PARAM_A,\n URL_EXTRACT_PARAMETER(TEST.URL, 'two') PARAM_B\nFROM TEST TEST\nEMIT CHANGES", + "ddlCommand" : { + "@type" : "createStreamV1", + "sourceName" : "OUTPUT", + "schema" : "`K` STRING KEY, `PARAM_A` STRING, `PARAM_B` STRING", + "timestampColumn" : null, + "topicName" : "OUTPUT", + "formats" : { + "keyFormat" : { + "format" : "KAFKA", + "properties" : { } + }, + "valueFormat" : { + "format" : "PROTOBUF_NOSR", + "properties" : { } + } + }, + "windowInfo" : null, + "orReplace" : false, + "isSource" : false + }, + "queryPlan" : { + "sources" : [ "TEST" ], + "sink" : "OUTPUT", + "physicalPlan" : { + "@type" : "streamSinkV1", + "properties" : { + "queryContext" : "OUTPUT" + }, + "source" : { + "@type" : "streamSelectV1", + "properties" : { + "queryContext" : "Project" + }, + "source" : { + "@type" : "streamSourceV1", + "properties" : { + "queryContext" : "KsqlTopic/Source" + }, + "topicName" : "test_topic", + "formats" : { + "keyFormat" : { + "format" : "KAFKA", + "properties" : { } + }, + "valueFormat" : { + "format" : "PROTOBUF_NOSR", + "properties" : { } + } + }, + "timestampColumn" : null, + "sourceSchema" : "`K` STRING KEY, `URL` STRING", + "pseudoColumnVersion" : 1 + }, + "keyColumnNames" : [ "K" ], + "selectedKeys" : null, + "selectExpressions" : [ "URL_EXTRACT_PARAMETER(URL, 'one') AS PARAM_A", "URL_EXTRACT_PARAMETER(URL, 'two') AS PARAM_B" ] + }, + "formats" : { + "keyFormat" : { + "format" : "KAFKA", + "properties" : { } + }, + "valueFormat" : { + "format" : "PROTOBUF_NOSR", + "properties" : { } + } + }, + "topicName" : "OUTPUT", + "timestampColumn" : null + }, + "queryId" : "CSAS_OUTPUT_0", + "runtimeId" : null + } + } ], + "configs" : { + "ksql.extension.dir" : "ext", + "ksql.streams.cache.max.bytes.buffering" : "0", + "metric.reporters" : "", + "ksql.query.status.running.threshold.seconds" : "300", + "ksql.connect.basic.auth.credentials.reload" : "false", + "ksql.output.topic.name.prefix" : "", + "ksql.query.pull.stream.enabled" : "true", + "ksql.query.push.v2.interpreter.enabled" : "true", + "ksql.queryanonymizer.logs_enabled" : "true", + "ksql.variable.substitution.enable" : "true", + "ksql.streams.shutdown.timeout.ms" : "300000", + "ksql.query.pull.max.allowed.offset.lag" : "9223372036854775807", + "ksql.query.pull.max.qps" : "2147483647", + "ksql.access.validator.enable" : "auto", + "ksql.streams.bootstrap.servers" : "localhost:0", + "ksql.query.pull.metrics.enabled" : "true", + "ksql.metrics.extension" : null, + "ksql.query.push.v2.alos.enabled" : "true", + "ksql.query.push.v2.max.hourly.bandwidth.megabytes" : "2147483647", + "ksql.query.pull.range.scan.enabled" : "true", + "ksql.transient.query.cleanup.service.initial.delay.seconds" : "600", + "ksql.hidden.topics" : "_confluent.*,__confluent.*,_schemas,__consumer_offsets,__transaction_state,connect-configs,connect-offsets,connect-status,connect-statuses", + "ksql.lambdas.enabled" : "true", + "ksql.source.table.materialization.enabled" : "true", + "ksql.query.pull.max.hourly.bandwidth.megabytes" : "2147483647", + "ksql.sink.window.change.log.additional.retention" : "1000000", + "ksql.query.persistent.active.limit" : "2147483647", + "ksql.persistence.wrap.single.values" : null, + "ksql.query.transient.max.bytes.buffering.total" : "-1", + "ksql.connect.basic.auth.credentials.source" : "NONE", + "ksql.schema.registry.url" : "schema_registry.url:0", + "ksql.properties.overrides.denylist" : "", + "ksql.service.id" : "some.ksql.service.id", + "ksql.query.push.v2.max.catchup.consumers" : "5", + "ksql.query.push.v2.enabled" : "false", + "ksql.transient.query.cleanup.service.enable" : "true", + "ksql.query.push.v2.metrics.enabled" : "true", + "ksql.rowpartition.rowoffset.enabled" : "true", + "ksql.streams.commit.interval.ms" : "2000", + "ksql.query.pull.table.scan.enabled" : "true", + "ksql.streams.auto.commit.interval.ms" : "0", + "ksql.streams.topology.optimization" : "all", + "ksql.endpoint.migrate.query" : "true", + "ksql.query.push.v2.registry.installed" : "false", + "ksql.streams.num.stream.threads" : "4", + "ksql.metrics.tags.custom" : "", + "ksql.query.push.v2.catchup.consumer.msg.window" : "50", + "ksql.runtime.feature.shared.enabled" : "false", + "ksql.udf.collect.metrics" : "false", + "ksql.new.query.planner.enabled" : "false", + "ksql.connect.request.headers.plugin" : null, + "ksql.security.extension.class" : null, + "ksql.transient.prefix" : "transient_", + "ksql.headers.columns.enabled" : "true", + "ksql.streams.default.deserialization.exception.handler" : "io.confluent.ksql.errors.LogMetricAndContinueExceptionHandler", + "ksql.connect.request.timeout.ms" : "5000", + "ksql.query.pull.enable.standby.reads" : "false", + "ksql.persistence.default.format.key" : "KAFKA", + "ksql.query.persistent.max.bytes.buffering.total" : "-1", + "ksql.query.error.max.queue.size" : "10", + "ksql.query.cleanup.shutdown.timeout.ms" : "30000", + "ksql.internal.topic.min.insync.replicas" : "1", + "ksql.internal.topic.replicas" : "1", + "ksql.insert.into.values.enabled" : "true", + "ksql.queryanonymizer.cluster_namespace" : null, + "ksql.create.or.replace.enabled" : "true", + "ksql.shared.runtimes.count" : "2", + "ksql.cast.strings.preserve.nulls" : "true", + "ksql.authorization.cache.max.entries" : "10000", + "ksql.pull.queries.enable" : "true", + "ksql.transient.query.cleanup.service.period.seconds" : "600", + "ksql.suppress.enabled" : "false", + "ksql.readonly.topics" : "_confluent.*,__confluent.*,_schemas,__consumer_offsets,__transaction_state,connect-configs,connect-offsets,connect-status,connect-statuses", + "ksql.connect.basic.auth.credentials.file" : "", + "ksql.authorization.cache.expiry.time.secs" : "30", + "ksql.query.retry.backoff.initial.ms" : "15000", + "ksql.query.pull.max.concurrent.requests" : "2147483647", + "ksql.streams.auto.offset.reset" : "earliest", + "ksql.connect.url" : "http://localhost:8083", + "ksql.query.push.v2.new.latest.delay.ms" : "5000", + "ksql.query.push.v2.latest.reset.age.ms" : "30000", + "ksql.streams.default.production.exception.handler" : "io.confluent.ksql.errors.ProductionExceptionHandlerUtil$LogAndFailProductionExceptionHandler", + "ksql.query.pull.interpreter.enabled" : "true", + "ksql.query.pull.limit.clause.enabled" : "true", + "ksql.query.pull.router.thread.pool.size" : "50", + "ksql.query.push.v2.continuation.tokens.enabled" : "false", + "ksql.query.retry.backoff.max.ms" : "900000", + "ksql.timestamp.throw.on.invalid" : "false", + "ksql.persistence.default.format.value" : null, + "ksql.udfs.enabled" : "true", + "ksql.udf.enable.security.manager" : "true", + "ksql.connect.worker.config" : "", + "ksql.nested.error.set.null" : "true", + "ksql.query.pull.thread.pool.size" : "50", + "ksql.persistent.prefix" : "query_", + "ksql.metastore.backup.location" : "", + "ksql.error.classifier.regex" : "", + "ksql.suppress.buffer.size.bytes" : "-1" + } +} \ No newline at end of file diff --git a/ksqldb-functional-tests/src/test/resources/historical_plans/url_-_extract_a_parameter_from_a_URL_using_URL_EXTRACT_PARAMETER_-_PROTOBUF_NOSR/7.3.0_1651070264537/spec.json b/ksqldb-functional-tests/src/test/resources/historical_plans/url_-_extract_a_parameter_from_a_URL_using_URL_EXTRACT_PARAMETER_-_PROTOBUF_NOSR/7.3.0_1651070264537/spec.json new file mode 100644 index 000000000000..755ce3f56549 --- /dev/null +++ b/ksqldb-functional-tests/src/test/resources/historical_plans/url_-_extract_a_parameter_from_a_URL_using_URL_EXTRACT_PARAMETER_-_PROTOBUF_NOSR/7.3.0_1651070264537/spec.json @@ -0,0 +1,101 @@ +{ + "version" : "7.3.0", + "timestamp" : 1651070264537, + "path" : "query-validation-tests/url.json", + "schemas" : { + "CSAS_OUTPUT_0.KsqlTopic.Source" : { + "schema" : "`K` STRING KEY, `URL` STRING", + "keyFormat" : { + "format" : "KAFKA" + }, + "valueFormat" : { + "format" : "PROTOBUF_NOSR" + } + }, + "CSAS_OUTPUT_0.OUTPUT" : { + "schema" : "`K` STRING KEY, `PARAM_A` STRING, `PARAM_B` STRING", + "keyFormat" : { + "format" : "KAFKA" + }, + "valueFormat" : { + "format" : "PROTOBUF_NOSR" + } + } + }, + "testCase" : { + "name" : "extract a parameter from a URL using URL_EXTRACT_PARAMETER - PROTOBUF_NOSR", + "inputs" : [ { + "topic" : "test_topic", + "key" : "1", + "value" : { + "url" : "http://www.test.com/?one=a&two=b&three" + }, + "timestamp" : 0 + } ], + "outputs" : [ { + "topic" : "OUTPUT", + "key" : "1", + "value" : { + "PARAM_A" : "a", + "PARAM_B" : "b" + }, + "timestamp" : 0 + } ], + "topics" : [ { + "name" : "OUTPUT", + "replicas" : 1, + "numPartitions" : 4 + }, { + "name" : "test_topic", + "replicas" : 1, + "numPartitions" : 4 + } ], + "statements" : [ "CREATE STREAM TEST (K STRING KEY, url VARCHAR) WITH (kafka_topic='test_topic', value_format='PROTOBUF_NOSR');", "CREATE STREAM OUTPUT AS SELECT K, URL_EXTRACT_PARAMETER(url,'one') as PARAM_A, URL_EXTRACT_PARAMETER(url,'two') as PARAM_B FROM TEST;" ], + "post" : { + "sources" : [ { + "name" : "OUTPUT", + "type" : "STREAM", + "schema" : "`K` STRING KEY, `PARAM_A` STRING, `PARAM_B` STRING", + "keyFormat" : { + "format" : "KAFKA" + }, + "valueFormat" : "PROTOBUF_NOSR", + "keyFeatures" : [ ], + "valueFeatures" : [ ], + "isSource" : false + }, { + "name" : "TEST", + "type" : "STREAM", + "schema" : "`K` STRING KEY, `URL` STRING", + "keyFormat" : { + "format" : "KAFKA" + }, + "valueFormat" : "PROTOBUF_NOSR", + "keyFeatures" : [ ], + "valueFeatures" : [ ], + "isSource" : false + } ], + "topics" : { + "topics" : [ { + "name" : "test_topic", + "keyFormat" : { + "format" : "KAFKA" + }, + "valueFormat" : { + "format" : "PROTOBUF_NOSR" + }, + "partitions" : 4 + }, { + "name" : "OUTPUT", + "keyFormat" : { + "format" : "KAFKA" + }, + "valueFormat" : { + "format" : "PROTOBUF_NOSR" + }, + "partitions" : 4 + } ] + } + } + } +} \ No newline at end of file diff --git a/ksqldb-functional-tests/src/test/resources/historical_plans/url_-_extract_a_parameter_from_a_URL_using_URL_EXTRACT_PARAMETER_-_PROTOBUF_NOSR/7.3.0_1651070264537/topology b/ksqldb-functional-tests/src/test/resources/historical_plans/url_-_extract_a_parameter_from_a_URL_using_URL_EXTRACT_PARAMETER_-_PROTOBUF_NOSR/7.3.0_1651070264537/topology new file mode 100644 index 000000000000..441a8f282644 --- /dev/null +++ b/ksqldb-functional-tests/src/test/resources/historical_plans/url_-_extract_a_parameter_from_a_URL_using_URL_EXTRACT_PARAMETER_-_PROTOBUF_NOSR/7.3.0_1651070264537/topology @@ -0,0 +1,13 @@ +Topologies: + Sub-topology: 0 + Source: KSTREAM-SOURCE-0000000000 (topics: [test_topic]) + --> KSTREAM-TRANSFORMVALUES-0000000001 + Processor: KSTREAM-TRANSFORMVALUES-0000000001 (stores: []) + --> Project + <-- KSTREAM-SOURCE-0000000000 + Processor: Project (stores: []) + --> KSTREAM-SINK-0000000003 + <-- KSTREAM-TRANSFORMVALUES-0000000001 + Sink: KSTREAM-SINK-0000000003 (topic: OUTPUT) + <-- Project + diff --git a/ksqldb-functional-tests/src/test/resources/historical_plans/url_-_extract_a_path_from_a_URL_using_URL_EXTRACT_PATH_-_PROTOBUF/7.3.0_1651070264696/plan.json b/ksqldb-functional-tests/src/test/resources/historical_plans/url_-_extract_a_path_from_a_URL_using_URL_EXTRACT_PATH_-_PROTOBUF/7.3.0_1651070264696/plan.json new file mode 100644 index 000000000000..eb5f3e788784 --- /dev/null +++ b/ksqldb-functional-tests/src/test/resources/historical_plans/url_-_extract_a_path_from_a_URL_using_URL_EXTRACT_PATH_-_PROTOBUF/7.3.0_1651070264696/plan.json @@ -0,0 +1,211 @@ +{ + "plan" : [ { + "@type" : "ksqlPlanV1", + "statementText" : "CREATE STREAM TEST (K STRING KEY, URL STRING) WITH (KAFKA_TOPIC='test_topic', KEY_FORMAT='KAFKA', VALUE_FORMAT='PROTOBUF');", + "ddlCommand" : { + "@type" : "createStreamV1", + "sourceName" : "TEST", + "schema" : "`K` STRING KEY, `URL` STRING", + "timestampColumn" : null, + "topicName" : "test_topic", + "formats" : { + "keyFormat" : { + "format" : "KAFKA", + "properties" : { } + }, + "valueFormat" : { + "format" : "PROTOBUF", + "properties" : { + "unwrapPrimitives" : "true" + } + } + }, + "windowInfo" : null, + "orReplace" : false, + "isSource" : false + }, + "queryPlan" : null + }, { + "@type" : "ksqlPlanV1", + "statementText" : "CREATE STREAM OUTPUT AS SELECT\n TEST.K K,\n URL_EXTRACT_PATH(TEST.URL) PATH\nFROM TEST TEST\nEMIT CHANGES", + "ddlCommand" : { + "@type" : "createStreamV1", + "sourceName" : "OUTPUT", + "schema" : "`K` STRING KEY, `PATH` STRING", + "timestampColumn" : null, + "topicName" : "OUTPUT", + "formats" : { + "keyFormat" : { + "format" : "KAFKA", + "properties" : { } + }, + "valueFormat" : { + "format" : "PROTOBUF", + "properties" : { + "unwrapPrimitives" : "true" + } + } + }, + "windowInfo" : null, + "orReplace" : false, + "isSource" : false + }, + "queryPlan" : { + "sources" : [ "TEST" ], + "sink" : "OUTPUT", + "physicalPlan" : { + "@type" : "streamSinkV1", + "properties" : { + "queryContext" : "OUTPUT" + }, + "source" : { + "@type" : "streamSelectV1", + "properties" : { + "queryContext" : "Project" + }, + "source" : { + "@type" : "streamSourceV1", + "properties" : { + "queryContext" : "KsqlTopic/Source" + }, + "topicName" : "test_topic", + "formats" : { + "keyFormat" : { + "format" : "KAFKA", + "properties" : { } + }, + "valueFormat" : { + "format" : "PROTOBUF", + "properties" : { + "unwrapPrimitives" : "true" + } + } + }, + "timestampColumn" : null, + "sourceSchema" : "`K` STRING KEY, `URL` STRING", + "pseudoColumnVersion" : 1 + }, + "keyColumnNames" : [ "K" ], + "selectedKeys" : null, + "selectExpressions" : [ "URL_EXTRACT_PATH(URL) AS PATH" ] + }, + "formats" : { + "keyFormat" : { + "format" : "KAFKA", + "properties" : { } + }, + "valueFormat" : { + "format" : "PROTOBUF", + "properties" : { + "unwrapPrimitives" : "true" + } + } + }, + "topicName" : "OUTPUT", + "timestampColumn" : null + }, + "queryId" : "CSAS_OUTPUT_0", + "runtimeId" : null + } + } ], + "configs" : { + "ksql.extension.dir" : "ext", + "ksql.streams.cache.max.bytes.buffering" : "0", + "metric.reporters" : "", + "ksql.query.status.running.threshold.seconds" : "300", + "ksql.connect.basic.auth.credentials.reload" : "false", + "ksql.output.topic.name.prefix" : "", + "ksql.query.pull.stream.enabled" : "true", + "ksql.query.push.v2.interpreter.enabled" : "true", + "ksql.queryanonymizer.logs_enabled" : "true", + "ksql.variable.substitution.enable" : "true", + "ksql.streams.shutdown.timeout.ms" : "300000", + "ksql.query.pull.max.allowed.offset.lag" : "9223372036854775807", + "ksql.query.pull.max.qps" : "2147483647", + "ksql.access.validator.enable" : "auto", + "ksql.streams.bootstrap.servers" : "localhost:0", + "ksql.query.pull.metrics.enabled" : "true", + "ksql.metrics.extension" : null, + "ksql.query.push.v2.alos.enabled" : "true", + "ksql.query.push.v2.max.hourly.bandwidth.megabytes" : "2147483647", + "ksql.query.pull.range.scan.enabled" : "true", + "ksql.transient.query.cleanup.service.initial.delay.seconds" : "600", + "ksql.hidden.topics" : "_confluent.*,__confluent.*,_schemas,__consumer_offsets,__transaction_state,connect-configs,connect-offsets,connect-status,connect-statuses", + "ksql.lambdas.enabled" : "true", + "ksql.source.table.materialization.enabled" : "true", + "ksql.query.pull.max.hourly.bandwidth.megabytes" : "2147483647", + "ksql.sink.window.change.log.additional.retention" : "1000000", + "ksql.query.persistent.active.limit" : "2147483647", + "ksql.persistence.wrap.single.values" : null, + "ksql.query.transient.max.bytes.buffering.total" : "-1", + "ksql.connect.basic.auth.credentials.source" : "NONE", + "ksql.schema.registry.url" : "schema_registry.url:0", + "ksql.properties.overrides.denylist" : "", + "ksql.service.id" : "some.ksql.service.id", + "ksql.query.push.v2.max.catchup.consumers" : "5", + "ksql.query.push.v2.enabled" : "false", + "ksql.transient.query.cleanup.service.enable" : "true", + "ksql.query.push.v2.metrics.enabled" : "true", + "ksql.rowpartition.rowoffset.enabled" : "true", + "ksql.streams.commit.interval.ms" : "2000", + "ksql.query.pull.table.scan.enabled" : "true", + "ksql.streams.auto.commit.interval.ms" : "0", + "ksql.streams.topology.optimization" : "all", + "ksql.endpoint.migrate.query" : "true", + "ksql.query.push.v2.registry.installed" : "false", + "ksql.streams.num.stream.threads" : "4", + "ksql.metrics.tags.custom" : "", + "ksql.query.push.v2.catchup.consumer.msg.window" : "50", + "ksql.runtime.feature.shared.enabled" : "false", + "ksql.udf.collect.metrics" : "false", + "ksql.new.query.planner.enabled" : "false", + "ksql.connect.request.headers.plugin" : null, + "ksql.security.extension.class" : null, + "ksql.transient.prefix" : "transient_", + "ksql.headers.columns.enabled" : "true", + "ksql.streams.default.deserialization.exception.handler" : "io.confluent.ksql.errors.LogMetricAndContinueExceptionHandler", + "ksql.connect.request.timeout.ms" : "5000", + "ksql.query.pull.enable.standby.reads" : "false", + "ksql.persistence.default.format.key" : "KAFKA", + "ksql.query.persistent.max.bytes.buffering.total" : "-1", + "ksql.query.error.max.queue.size" : "10", + "ksql.query.cleanup.shutdown.timeout.ms" : "30000", + "ksql.internal.topic.min.insync.replicas" : "1", + "ksql.internal.topic.replicas" : "1", + "ksql.insert.into.values.enabled" : "true", + "ksql.queryanonymizer.cluster_namespace" : null, + "ksql.create.or.replace.enabled" : "true", + "ksql.shared.runtimes.count" : "2", + "ksql.cast.strings.preserve.nulls" : "true", + "ksql.authorization.cache.max.entries" : "10000", + "ksql.pull.queries.enable" : "true", + "ksql.transient.query.cleanup.service.period.seconds" : "600", + "ksql.suppress.enabled" : "false", + "ksql.readonly.topics" : "_confluent.*,__confluent.*,_schemas,__consumer_offsets,__transaction_state,connect-configs,connect-offsets,connect-status,connect-statuses", + "ksql.connect.basic.auth.credentials.file" : "", + "ksql.authorization.cache.expiry.time.secs" : "30", + "ksql.query.retry.backoff.initial.ms" : "15000", + "ksql.query.pull.max.concurrent.requests" : "2147483647", + "ksql.streams.auto.offset.reset" : "earliest", + "ksql.connect.url" : "http://localhost:8083", + "ksql.query.push.v2.new.latest.delay.ms" : "5000", + "ksql.query.push.v2.latest.reset.age.ms" : "30000", + "ksql.streams.default.production.exception.handler" : "io.confluent.ksql.errors.ProductionExceptionHandlerUtil$LogAndFailProductionExceptionHandler", + "ksql.query.pull.interpreter.enabled" : "true", + "ksql.query.pull.limit.clause.enabled" : "true", + "ksql.query.pull.router.thread.pool.size" : "50", + "ksql.query.push.v2.continuation.tokens.enabled" : "false", + "ksql.query.retry.backoff.max.ms" : "900000", + "ksql.timestamp.throw.on.invalid" : "false", + "ksql.persistence.default.format.value" : null, + "ksql.udfs.enabled" : "true", + "ksql.udf.enable.security.manager" : "true", + "ksql.connect.worker.config" : "", + "ksql.nested.error.set.null" : "true", + "ksql.query.pull.thread.pool.size" : "50", + "ksql.persistent.prefix" : "query_", + "ksql.metastore.backup.location" : "", + "ksql.error.classifier.regex" : "", + "ksql.suppress.buffer.size.bytes" : "-1" + } +} \ No newline at end of file diff --git a/ksqldb-functional-tests/src/test/resources/historical_plans/url_-_extract_a_path_from_a_URL_using_URL_EXTRACT_PATH_-_PROTOBUF/7.3.0_1651070264696/spec.json b/ksqldb-functional-tests/src/test/resources/historical_plans/url_-_extract_a_path_from_a_URL_using_URL_EXTRACT_PATH_-_PROTOBUF/7.3.0_1651070264696/spec.json new file mode 100644 index 000000000000..b465a008c1d9 --- /dev/null +++ b/ksqldb-functional-tests/src/test/resources/historical_plans/url_-_extract_a_path_from_a_URL_using_URL_EXTRACT_PATH_-_PROTOBUF/7.3.0_1651070264696/spec.json @@ -0,0 +1,144 @@ +{ + "version" : "7.3.0", + "timestamp" : 1651070264696, + "path" : "query-validation-tests/url.json", + "schemas" : { + "CSAS_OUTPUT_0.KsqlTopic.Source" : { + "schema" : "`K` STRING KEY, `URL` STRING", + "keyFormat" : { + "format" : "KAFKA" + }, + "valueFormat" : { + "format" : "PROTOBUF", + "properties" : { + "unwrapPrimitives" : "true" + } + } + }, + "CSAS_OUTPUT_0.OUTPUT" : { + "schema" : "`K` STRING KEY, `PATH` STRING", + "keyFormat" : { + "format" : "KAFKA" + }, + "valueFormat" : { + "format" : "PROTOBUF", + "properties" : { + "unwrapPrimitives" : "true" + } + } + } + }, + "testCase" : { + "name" : "extract a path from a URL using URL_EXTRACT_PATH - PROTOBUF", + "inputs" : [ { + "topic" : "test_topic", + "key" : "1", + "value" : { + "url" : "http://www.test.com?query#fragment" + }, + "timestamp" : 0 + }, { + "topic" : "test_topic", + "key" : "2", + "value" : { + "url" : "http://www.test.com/path?query" + }, + "timestamp" : 1 + }, { + "topic" : "test_topic", + "key" : "3", + "value" : { + "url" : "http://test@confluent.io:8080/nested/path?query&jobs" + }, + "timestamp" : 2 + } ], + "outputs" : [ { + "topic" : "OUTPUT", + "key" : "1", + "value" : { + "PATH" : "" + }, + "timestamp" : 0 + }, { + "topic" : "OUTPUT", + "key" : "2", + "value" : { + "PATH" : "/path" + }, + "timestamp" : 1 + }, { + "topic" : "OUTPUT", + "key" : "3", + "value" : { + "PATH" : "/nested/path" + }, + "timestamp" : 2 + } ], + "topics" : [ { + "name" : "OUTPUT", + "replicas" : 1, + "numPartitions" : 4 + }, { + "name" : "test_topic", + "valueSchema" : "syntax = \"proto3\";\n\nmessage ConnectDefault1 {\n string URL = 1;\n}\n", + "valueFormat" : "PROTOBUF", + "replicas" : 1, + "numPartitions" : 4 + } ], + "statements" : [ "CREATE STREAM TEST (K STRING KEY, url VARCHAR) WITH (kafka_topic='test_topic', value_format='PROTOBUF');", "CREATE STREAM OUTPUT AS SELECT K, URL_EXTRACT_PATH(url) as PATH FROM TEST;" ], + "post" : { + "sources" : [ { + "name" : "OUTPUT", + "type" : "STREAM", + "schema" : "`K` STRING KEY, `PATH` STRING", + "keyFormat" : { + "format" : "KAFKA" + }, + "valueFormat" : "PROTOBUF", + "keyFeatures" : [ ], + "valueFeatures" : [ ], + "isSource" : false + }, { + "name" : "TEST", + "type" : "STREAM", + "schema" : "`K` STRING KEY, `URL` STRING", + "keyFormat" : { + "format" : "KAFKA" + }, + "valueFormat" : "PROTOBUF", + "keyFeatures" : [ ], + "valueFeatures" : [ ], + "isSource" : false + } ], + "topics" : { + "topics" : [ { + "name" : "test_topic", + "keyFormat" : { + "format" : "KAFKA" + }, + "valueFormat" : { + "format" : "PROTOBUF", + "properties" : { + "unwrapPrimitives" : "true" + } + }, + "partitions" : 4, + "valueSchema" : "syntax = \"proto3\";\n\nmessage ConnectDefault1 {\n string URL = 1;\n}\n" + }, { + "name" : "OUTPUT", + "keyFormat" : { + "format" : "KAFKA" + }, + "valueFormat" : { + "format" : "PROTOBUF", + "properties" : { + "unwrapPrimitives" : "true" + } + }, + "partitions" : 4, + "valueSchema" : "syntax = \"proto3\";\n\nmessage ConnectDefault1 {\n string PATH = 1;\n}\n" + } ] + } + } + } +} \ No newline at end of file diff --git a/ksqldb-functional-tests/src/test/resources/historical_plans/url_-_extract_a_path_from_a_URL_using_URL_EXTRACT_PATH_-_PROTOBUF/7.3.0_1651070264696/topology b/ksqldb-functional-tests/src/test/resources/historical_plans/url_-_extract_a_path_from_a_URL_using_URL_EXTRACT_PATH_-_PROTOBUF/7.3.0_1651070264696/topology new file mode 100644 index 000000000000..441a8f282644 --- /dev/null +++ b/ksqldb-functional-tests/src/test/resources/historical_plans/url_-_extract_a_path_from_a_URL_using_URL_EXTRACT_PATH_-_PROTOBUF/7.3.0_1651070264696/topology @@ -0,0 +1,13 @@ +Topologies: + Sub-topology: 0 + Source: KSTREAM-SOURCE-0000000000 (topics: [test_topic]) + --> KSTREAM-TRANSFORMVALUES-0000000001 + Processor: KSTREAM-TRANSFORMVALUES-0000000001 (stores: []) + --> Project + <-- KSTREAM-SOURCE-0000000000 + Processor: Project (stores: []) + --> KSTREAM-SINK-0000000003 + <-- KSTREAM-TRANSFORMVALUES-0000000001 + Sink: KSTREAM-SINK-0000000003 (topic: OUTPUT) + <-- Project + diff --git a/ksqldb-functional-tests/src/test/resources/historical_plans/url_-_extract_a_path_from_a_URL_using_URL_EXTRACT_PATH_-_PROTOBUF_NOSR/7.3.0_1651070264721/plan.json b/ksqldb-functional-tests/src/test/resources/historical_plans/url_-_extract_a_path_from_a_URL_using_URL_EXTRACT_PATH_-_PROTOBUF_NOSR/7.3.0_1651070264721/plan.json new file mode 100644 index 000000000000..4d88399bbf58 --- /dev/null +++ b/ksqldb-functional-tests/src/test/resources/historical_plans/url_-_extract_a_path_from_a_URL_using_URL_EXTRACT_PATH_-_PROTOBUF_NOSR/7.3.0_1651070264721/plan.json @@ -0,0 +1,203 @@ +{ + "plan" : [ { + "@type" : "ksqlPlanV1", + "statementText" : "CREATE STREAM TEST (K STRING KEY, URL STRING) WITH (KAFKA_TOPIC='test_topic', KEY_FORMAT='KAFKA', VALUE_FORMAT='PROTOBUF_NOSR');", + "ddlCommand" : { + "@type" : "createStreamV1", + "sourceName" : "TEST", + "schema" : "`K` STRING KEY, `URL` STRING", + "timestampColumn" : null, + "topicName" : "test_topic", + "formats" : { + "keyFormat" : { + "format" : "KAFKA", + "properties" : { } + }, + "valueFormat" : { + "format" : "PROTOBUF_NOSR", + "properties" : { } + } + }, + "windowInfo" : null, + "orReplace" : false, + "isSource" : false + }, + "queryPlan" : null + }, { + "@type" : "ksqlPlanV1", + "statementText" : "CREATE STREAM OUTPUT AS SELECT\n TEST.K K,\n URL_EXTRACT_PATH(TEST.URL) PATH\nFROM TEST TEST\nEMIT CHANGES", + "ddlCommand" : { + "@type" : "createStreamV1", + "sourceName" : "OUTPUT", + "schema" : "`K` STRING KEY, `PATH` STRING", + "timestampColumn" : null, + "topicName" : "OUTPUT", + "formats" : { + "keyFormat" : { + "format" : "KAFKA", + "properties" : { } + }, + "valueFormat" : { + "format" : "PROTOBUF_NOSR", + "properties" : { } + } + }, + "windowInfo" : null, + "orReplace" : false, + "isSource" : false + }, + "queryPlan" : { + "sources" : [ "TEST" ], + "sink" : "OUTPUT", + "physicalPlan" : { + "@type" : "streamSinkV1", + "properties" : { + "queryContext" : "OUTPUT" + }, + "source" : { + "@type" : "streamSelectV1", + "properties" : { + "queryContext" : "Project" + }, + "source" : { + "@type" : "streamSourceV1", + "properties" : { + "queryContext" : "KsqlTopic/Source" + }, + "topicName" : "test_topic", + "formats" : { + "keyFormat" : { + "format" : "KAFKA", + "properties" : { } + }, + "valueFormat" : { + "format" : "PROTOBUF_NOSR", + "properties" : { } + } + }, + "timestampColumn" : null, + "sourceSchema" : "`K` STRING KEY, `URL` STRING", + "pseudoColumnVersion" : 1 + }, + "keyColumnNames" : [ "K" ], + "selectedKeys" : null, + "selectExpressions" : [ "URL_EXTRACT_PATH(URL) AS PATH" ] + }, + "formats" : { + "keyFormat" : { + "format" : "KAFKA", + "properties" : { } + }, + "valueFormat" : { + "format" : "PROTOBUF_NOSR", + "properties" : { } + } + }, + "topicName" : "OUTPUT", + "timestampColumn" : null + }, + "queryId" : "CSAS_OUTPUT_0", + "runtimeId" : null + } + } ], + "configs" : { + "ksql.extension.dir" : "ext", + "ksql.streams.cache.max.bytes.buffering" : "0", + "metric.reporters" : "", + "ksql.query.status.running.threshold.seconds" : "300", + "ksql.connect.basic.auth.credentials.reload" : "false", + "ksql.output.topic.name.prefix" : "", + "ksql.query.pull.stream.enabled" : "true", + "ksql.query.push.v2.interpreter.enabled" : "true", + "ksql.queryanonymizer.logs_enabled" : "true", + "ksql.variable.substitution.enable" : "true", + "ksql.streams.shutdown.timeout.ms" : "300000", + "ksql.query.pull.max.allowed.offset.lag" : "9223372036854775807", + "ksql.query.pull.max.qps" : "2147483647", + "ksql.access.validator.enable" : "auto", + "ksql.streams.bootstrap.servers" : "localhost:0", + "ksql.query.pull.metrics.enabled" : "true", + "ksql.metrics.extension" : null, + "ksql.query.push.v2.alos.enabled" : "true", + "ksql.query.push.v2.max.hourly.bandwidth.megabytes" : "2147483647", + "ksql.query.pull.range.scan.enabled" : "true", + "ksql.transient.query.cleanup.service.initial.delay.seconds" : "600", + "ksql.hidden.topics" : "_confluent.*,__confluent.*,_schemas,__consumer_offsets,__transaction_state,connect-configs,connect-offsets,connect-status,connect-statuses", + "ksql.lambdas.enabled" : "true", + "ksql.source.table.materialization.enabled" : "true", + "ksql.query.pull.max.hourly.bandwidth.megabytes" : "2147483647", + "ksql.sink.window.change.log.additional.retention" : "1000000", + "ksql.query.persistent.active.limit" : "2147483647", + "ksql.persistence.wrap.single.values" : null, + "ksql.query.transient.max.bytes.buffering.total" : "-1", + "ksql.connect.basic.auth.credentials.source" : "NONE", + "ksql.schema.registry.url" : "schema_registry.url:0", + "ksql.properties.overrides.denylist" : "", + "ksql.service.id" : "some.ksql.service.id", + "ksql.query.push.v2.max.catchup.consumers" : "5", + "ksql.query.push.v2.enabled" : "false", + "ksql.transient.query.cleanup.service.enable" : "true", + "ksql.query.push.v2.metrics.enabled" : "true", + "ksql.rowpartition.rowoffset.enabled" : "true", + "ksql.streams.commit.interval.ms" : "2000", + "ksql.query.pull.table.scan.enabled" : "true", + "ksql.streams.auto.commit.interval.ms" : "0", + "ksql.streams.topology.optimization" : "all", + "ksql.endpoint.migrate.query" : "true", + "ksql.query.push.v2.registry.installed" : "false", + "ksql.streams.num.stream.threads" : "4", + "ksql.metrics.tags.custom" : "", + "ksql.query.push.v2.catchup.consumer.msg.window" : "50", + "ksql.runtime.feature.shared.enabled" : "false", + "ksql.udf.collect.metrics" : "false", + "ksql.new.query.planner.enabled" : "false", + "ksql.connect.request.headers.plugin" : null, + "ksql.security.extension.class" : null, + "ksql.transient.prefix" : "transient_", + "ksql.headers.columns.enabled" : "true", + "ksql.streams.default.deserialization.exception.handler" : "io.confluent.ksql.errors.LogMetricAndContinueExceptionHandler", + "ksql.connect.request.timeout.ms" : "5000", + "ksql.query.pull.enable.standby.reads" : "false", + "ksql.persistence.default.format.key" : "KAFKA", + "ksql.query.persistent.max.bytes.buffering.total" : "-1", + "ksql.query.error.max.queue.size" : "10", + "ksql.query.cleanup.shutdown.timeout.ms" : "30000", + "ksql.internal.topic.min.insync.replicas" : "1", + "ksql.internal.topic.replicas" : "1", + "ksql.insert.into.values.enabled" : "true", + "ksql.queryanonymizer.cluster_namespace" : null, + "ksql.create.or.replace.enabled" : "true", + "ksql.shared.runtimes.count" : "2", + "ksql.cast.strings.preserve.nulls" : "true", + "ksql.authorization.cache.max.entries" : "10000", + "ksql.pull.queries.enable" : "true", + "ksql.transient.query.cleanup.service.period.seconds" : "600", + "ksql.suppress.enabled" : "false", + "ksql.readonly.topics" : "_confluent.*,__confluent.*,_schemas,__consumer_offsets,__transaction_state,connect-configs,connect-offsets,connect-status,connect-statuses", + "ksql.connect.basic.auth.credentials.file" : "", + "ksql.authorization.cache.expiry.time.secs" : "30", + "ksql.query.retry.backoff.initial.ms" : "15000", + "ksql.query.pull.max.concurrent.requests" : "2147483647", + "ksql.streams.auto.offset.reset" : "earliest", + "ksql.connect.url" : "http://localhost:8083", + "ksql.query.push.v2.new.latest.delay.ms" : "5000", + "ksql.query.push.v2.latest.reset.age.ms" : "30000", + "ksql.streams.default.production.exception.handler" : "io.confluent.ksql.errors.ProductionExceptionHandlerUtil$LogAndFailProductionExceptionHandler", + "ksql.query.pull.interpreter.enabled" : "true", + "ksql.query.pull.limit.clause.enabled" : "true", + "ksql.query.pull.router.thread.pool.size" : "50", + "ksql.query.push.v2.continuation.tokens.enabled" : "false", + "ksql.query.retry.backoff.max.ms" : "900000", + "ksql.timestamp.throw.on.invalid" : "false", + "ksql.persistence.default.format.value" : null, + "ksql.udfs.enabled" : "true", + "ksql.udf.enable.security.manager" : "true", + "ksql.connect.worker.config" : "", + "ksql.nested.error.set.null" : "true", + "ksql.query.pull.thread.pool.size" : "50", + "ksql.persistent.prefix" : "query_", + "ksql.metastore.backup.location" : "", + "ksql.error.classifier.regex" : "", + "ksql.suppress.buffer.size.bytes" : "-1" + } +} \ No newline at end of file diff --git a/ksqldb-functional-tests/src/test/resources/historical_plans/url_-_extract_a_path_from_a_URL_using_URL_EXTRACT_PATH_-_PROTOBUF_NOSR/7.3.0_1651070264721/spec.json b/ksqldb-functional-tests/src/test/resources/historical_plans/url_-_extract_a_path_from_a_URL_using_URL_EXTRACT_PATH_-_PROTOBUF_NOSR/7.3.0_1651070264721/spec.json new file mode 100644 index 000000000000..bc1fbf0e7731 --- /dev/null +++ b/ksqldb-functional-tests/src/test/resources/historical_plans/url_-_extract_a_path_from_a_URL_using_URL_EXTRACT_PATH_-_PROTOBUF_NOSR/7.3.0_1651070264721/spec.json @@ -0,0 +1,128 @@ +{ + "version" : "7.3.0", + "timestamp" : 1651070264721, + "path" : "query-validation-tests/url.json", + "schemas" : { + "CSAS_OUTPUT_0.KsqlTopic.Source" : { + "schema" : "`K` STRING KEY, `URL` STRING", + "keyFormat" : { + "format" : "KAFKA" + }, + "valueFormat" : { + "format" : "PROTOBUF_NOSR" + } + }, + "CSAS_OUTPUT_0.OUTPUT" : { + "schema" : "`K` STRING KEY, `PATH` STRING", + "keyFormat" : { + "format" : "KAFKA" + }, + "valueFormat" : { + "format" : "PROTOBUF_NOSR" + } + } + }, + "testCase" : { + "name" : "extract a path from a URL using URL_EXTRACT_PATH - PROTOBUF_NOSR", + "inputs" : [ { + "topic" : "test_topic", + "key" : "1", + "value" : { + "url" : "http://www.test.com?query#fragment" + }, + "timestamp" : 0 + }, { + "topic" : "test_topic", + "key" : "2", + "value" : { + "url" : "http://www.test.com/path?query" + }, + "timestamp" : 1 + }, { + "topic" : "test_topic", + "key" : "3", + "value" : { + "url" : "http://test@confluent.io:8080/nested/path?query&jobs" + }, + "timestamp" : 2 + } ], + "outputs" : [ { + "topic" : "OUTPUT", + "key" : "1", + "value" : { + "PATH" : "" + }, + "timestamp" : 0 + }, { + "topic" : "OUTPUT", + "key" : "2", + "value" : { + "PATH" : "/path" + }, + "timestamp" : 1 + }, { + "topic" : "OUTPUT", + "key" : "3", + "value" : { + "PATH" : "/nested/path" + }, + "timestamp" : 2 + } ], + "topics" : [ { + "name" : "OUTPUT", + "replicas" : 1, + "numPartitions" : 4 + }, { + "name" : "test_topic", + "replicas" : 1, + "numPartitions" : 4 + } ], + "statements" : [ "CREATE STREAM TEST (K STRING KEY, url VARCHAR) WITH (kafka_topic='test_topic', value_format='PROTOBUF_NOSR');", "CREATE STREAM OUTPUT AS SELECT K, URL_EXTRACT_PATH(url) as PATH FROM TEST;" ], + "post" : { + "sources" : [ { + "name" : "OUTPUT", + "type" : "STREAM", + "schema" : "`K` STRING KEY, `PATH` STRING", + "keyFormat" : { + "format" : "KAFKA" + }, + "valueFormat" : "PROTOBUF_NOSR", + "keyFeatures" : [ ], + "valueFeatures" : [ ], + "isSource" : false + }, { + "name" : "TEST", + "type" : "STREAM", + "schema" : "`K` STRING KEY, `URL` STRING", + "keyFormat" : { + "format" : "KAFKA" + }, + "valueFormat" : "PROTOBUF_NOSR", + "keyFeatures" : [ ], + "valueFeatures" : [ ], + "isSource" : false + } ], + "topics" : { + "topics" : [ { + "name" : "test_topic", + "keyFormat" : { + "format" : "KAFKA" + }, + "valueFormat" : { + "format" : "PROTOBUF_NOSR" + }, + "partitions" : 4 + }, { + "name" : "OUTPUT", + "keyFormat" : { + "format" : "KAFKA" + }, + "valueFormat" : { + "format" : "PROTOBUF_NOSR" + }, + "partitions" : 4 + } ] + } + } + } +} \ No newline at end of file diff --git a/ksqldb-functional-tests/src/test/resources/historical_plans/url_-_extract_a_path_from_a_URL_using_URL_EXTRACT_PATH_-_PROTOBUF_NOSR/7.3.0_1651070264721/topology b/ksqldb-functional-tests/src/test/resources/historical_plans/url_-_extract_a_path_from_a_URL_using_URL_EXTRACT_PATH_-_PROTOBUF_NOSR/7.3.0_1651070264721/topology new file mode 100644 index 000000000000..441a8f282644 --- /dev/null +++ b/ksqldb-functional-tests/src/test/resources/historical_plans/url_-_extract_a_path_from_a_URL_using_URL_EXTRACT_PATH_-_PROTOBUF_NOSR/7.3.0_1651070264721/topology @@ -0,0 +1,13 @@ +Topologies: + Sub-topology: 0 + Source: KSTREAM-SOURCE-0000000000 (topics: [test_topic]) + --> KSTREAM-TRANSFORMVALUES-0000000001 + Processor: KSTREAM-TRANSFORMVALUES-0000000001 (stores: []) + --> Project + <-- KSTREAM-SOURCE-0000000000 + Processor: Project (stores: []) + --> KSTREAM-SINK-0000000003 + <-- KSTREAM-TRANSFORMVALUES-0000000001 + Sink: KSTREAM-SINK-0000000003 (topic: OUTPUT) + <-- Project + diff --git a/ksqldb-functional-tests/src/test/resources/historical_plans/url_-_extract_a_port_from_a_URL_using_URL_EXTRACT_PORT_-_AVRO/7.3.0_1651070264771/plan.json b/ksqldb-functional-tests/src/test/resources/historical_plans/url_-_extract_a_port_from_a_URL_using_URL_EXTRACT_PORT_-_AVRO/7.3.0_1651070264771/plan.json new file mode 100644 index 000000000000..460b82e909d2 --- /dev/null +++ b/ksqldb-functional-tests/src/test/resources/historical_plans/url_-_extract_a_port_from_a_URL_using_URL_EXTRACT_PORT_-_AVRO/7.3.0_1651070264771/plan.json @@ -0,0 +1,203 @@ +{ + "plan" : [ { + "@type" : "ksqlPlanV1", + "statementText" : "CREATE STREAM TEST (K STRING KEY, URL STRING) WITH (KAFKA_TOPIC='test_topic', KEY_FORMAT='KAFKA', VALUE_FORMAT='AVRO');", + "ddlCommand" : { + "@type" : "createStreamV1", + "sourceName" : "TEST", + "schema" : "`K` STRING KEY, `URL` STRING", + "timestampColumn" : null, + "topicName" : "test_topic", + "formats" : { + "keyFormat" : { + "format" : "KAFKA", + "properties" : { } + }, + "valueFormat" : { + "format" : "AVRO", + "properties" : { } + } + }, + "windowInfo" : null, + "orReplace" : false, + "isSource" : false + }, + "queryPlan" : null + }, { + "@type" : "ksqlPlanV1", + "statementText" : "CREATE STREAM OUTPUT AS SELECT\n TEST.K K,\n URL_EXTRACT_PORT(TEST.URL) PORT\nFROM TEST TEST\nEMIT CHANGES", + "ddlCommand" : { + "@type" : "createStreamV1", + "sourceName" : "OUTPUT", + "schema" : "`K` STRING KEY, `PORT` INTEGER", + "timestampColumn" : null, + "topicName" : "OUTPUT", + "formats" : { + "keyFormat" : { + "format" : "KAFKA", + "properties" : { } + }, + "valueFormat" : { + "format" : "AVRO", + "properties" : { } + } + }, + "windowInfo" : null, + "orReplace" : false, + "isSource" : false + }, + "queryPlan" : { + "sources" : [ "TEST" ], + "sink" : "OUTPUT", + "physicalPlan" : { + "@type" : "streamSinkV1", + "properties" : { + "queryContext" : "OUTPUT" + }, + "source" : { + "@type" : "streamSelectV1", + "properties" : { + "queryContext" : "Project" + }, + "source" : { + "@type" : "streamSourceV1", + "properties" : { + "queryContext" : "KsqlTopic/Source" + }, + "topicName" : "test_topic", + "formats" : { + "keyFormat" : { + "format" : "KAFKA", + "properties" : { } + }, + "valueFormat" : { + "format" : "AVRO", + "properties" : { } + } + }, + "timestampColumn" : null, + "sourceSchema" : "`K` STRING KEY, `URL` STRING", + "pseudoColumnVersion" : 1 + }, + "keyColumnNames" : [ "K" ], + "selectedKeys" : null, + "selectExpressions" : [ "URL_EXTRACT_PORT(URL) AS PORT" ] + }, + "formats" : { + "keyFormat" : { + "format" : "KAFKA", + "properties" : { } + }, + "valueFormat" : { + "format" : "AVRO", + "properties" : { } + } + }, + "topicName" : "OUTPUT", + "timestampColumn" : null + }, + "queryId" : "CSAS_OUTPUT_0", + "runtimeId" : null + } + } ], + "configs" : { + "ksql.extension.dir" : "ext", + "ksql.streams.cache.max.bytes.buffering" : "0", + "metric.reporters" : "", + "ksql.query.status.running.threshold.seconds" : "300", + "ksql.connect.basic.auth.credentials.reload" : "false", + "ksql.output.topic.name.prefix" : "", + "ksql.query.pull.stream.enabled" : "true", + "ksql.query.push.v2.interpreter.enabled" : "true", + "ksql.queryanonymizer.logs_enabled" : "true", + "ksql.variable.substitution.enable" : "true", + "ksql.streams.shutdown.timeout.ms" : "300000", + "ksql.query.pull.max.allowed.offset.lag" : "9223372036854775807", + "ksql.query.pull.max.qps" : "2147483647", + "ksql.access.validator.enable" : "auto", + "ksql.streams.bootstrap.servers" : "localhost:0", + "ksql.query.pull.metrics.enabled" : "true", + "ksql.metrics.extension" : null, + "ksql.query.push.v2.alos.enabled" : "true", + "ksql.query.push.v2.max.hourly.bandwidth.megabytes" : "2147483647", + "ksql.query.pull.range.scan.enabled" : "true", + "ksql.transient.query.cleanup.service.initial.delay.seconds" : "600", + "ksql.hidden.topics" : "_confluent.*,__confluent.*,_schemas,__consumer_offsets,__transaction_state,connect-configs,connect-offsets,connect-status,connect-statuses", + "ksql.lambdas.enabled" : "true", + "ksql.source.table.materialization.enabled" : "true", + "ksql.query.pull.max.hourly.bandwidth.megabytes" : "2147483647", + "ksql.sink.window.change.log.additional.retention" : "1000000", + "ksql.query.persistent.active.limit" : "2147483647", + "ksql.persistence.wrap.single.values" : null, + "ksql.query.transient.max.bytes.buffering.total" : "-1", + "ksql.connect.basic.auth.credentials.source" : "NONE", + "ksql.schema.registry.url" : "schema_registry.url:0", + "ksql.properties.overrides.denylist" : "", + "ksql.service.id" : "some.ksql.service.id", + "ksql.query.push.v2.max.catchup.consumers" : "5", + "ksql.query.push.v2.enabled" : "false", + "ksql.transient.query.cleanup.service.enable" : "true", + "ksql.query.push.v2.metrics.enabled" : "true", + "ksql.rowpartition.rowoffset.enabled" : "true", + "ksql.streams.commit.interval.ms" : "2000", + "ksql.query.pull.table.scan.enabled" : "true", + "ksql.streams.auto.commit.interval.ms" : "0", + "ksql.streams.topology.optimization" : "all", + "ksql.endpoint.migrate.query" : "true", + "ksql.query.push.v2.registry.installed" : "false", + "ksql.streams.num.stream.threads" : "4", + "ksql.metrics.tags.custom" : "", + "ksql.query.push.v2.catchup.consumer.msg.window" : "50", + "ksql.runtime.feature.shared.enabled" : "false", + "ksql.udf.collect.metrics" : "false", + "ksql.new.query.planner.enabled" : "false", + "ksql.connect.request.headers.plugin" : null, + "ksql.security.extension.class" : null, + "ksql.transient.prefix" : "transient_", + "ksql.headers.columns.enabled" : "true", + "ksql.streams.default.deserialization.exception.handler" : "io.confluent.ksql.errors.LogMetricAndContinueExceptionHandler", + "ksql.connect.request.timeout.ms" : "5000", + "ksql.query.pull.enable.standby.reads" : "false", + "ksql.persistence.default.format.key" : "KAFKA", + "ksql.query.persistent.max.bytes.buffering.total" : "-1", + "ksql.query.error.max.queue.size" : "10", + "ksql.query.cleanup.shutdown.timeout.ms" : "30000", + "ksql.internal.topic.min.insync.replicas" : "1", + "ksql.internal.topic.replicas" : "1", + "ksql.insert.into.values.enabled" : "true", + "ksql.queryanonymizer.cluster_namespace" : null, + "ksql.create.or.replace.enabled" : "true", + "ksql.shared.runtimes.count" : "2", + "ksql.cast.strings.preserve.nulls" : "true", + "ksql.authorization.cache.max.entries" : "10000", + "ksql.pull.queries.enable" : "true", + "ksql.transient.query.cleanup.service.period.seconds" : "600", + "ksql.suppress.enabled" : "false", + "ksql.readonly.topics" : "_confluent.*,__confluent.*,_schemas,__consumer_offsets,__transaction_state,connect-configs,connect-offsets,connect-status,connect-statuses", + "ksql.connect.basic.auth.credentials.file" : "", + "ksql.authorization.cache.expiry.time.secs" : "30", + "ksql.query.retry.backoff.initial.ms" : "15000", + "ksql.query.pull.max.concurrent.requests" : "2147483647", + "ksql.streams.auto.offset.reset" : "earliest", + "ksql.connect.url" : "http://localhost:8083", + "ksql.query.push.v2.new.latest.delay.ms" : "5000", + "ksql.query.push.v2.latest.reset.age.ms" : "30000", + "ksql.streams.default.production.exception.handler" : "io.confluent.ksql.errors.ProductionExceptionHandlerUtil$LogAndFailProductionExceptionHandler", + "ksql.query.pull.interpreter.enabled" : "true", + "ksql.query.pull.limit.clause.enabled" : "true", + "ksql.query.pull.router.thread.pool.size" : "50", + "ksql.query.push.v2.continuation.tokens.enabled" : "false", + "ksql.query.retry.backoff.max.ms" : "900000", + "ksql.timestamp.throw.on.invalid" : "false", + "ksql.persistence.default.format.value" : null, + "ksql.udfs.enabled" : "true", + "ksql.udf.enable.security.manager" : "true", + "ksql.connect.worker.config" : "", + "ksql.nested.error.set.null" : "true", + "ksql.query.pull.thread.pool.size" : "50", + "ksql.persistent.prefix" : "query_", + "ksql.metastore.backup.location" : "", + "ksql.error.classifier.regex" : "", + "ksql.suppress.buffer.size.bytes" : "-1" + } +} \ No newline at end of file diff --git a/ksqldb-functional-tests/src/test/resources/historical_plans/url_-_extract_a_port_from_a_URL_using_URL_EXTRACT_PORT_-_AVRO/7.3.0_1651070264771/spec.json b/ksqldb-functional-tests/src/test/resources/historical_plans/url_-_extract_a_port_from_a_URL_using_URL_EXTRACT_PORT_-_AVRO/7.3.0_1651070264771/spec.json new file mode 100644 index 000000000000..01cf3ea9531a --- /dev/null +++ b/ksqldb-functional-tests/src/test/resources/historical_plans/url_-_extract_a_port_from_a_URL_using_URL_EXTRACT_PORT_-_AVRO/7.3.0_1651070264771/spec.json @@ -0,0 +1,147 @@ +{ + "version" : "7.3.0", + "timestamp" : 1651070264771, + "path" : "query-validation-tests/url.json", + "schemas" : { + "CSAS_OUTPUT_0.KsqlTopic.Source" : { + "schema" : "`K` STRING KEY, `URL` STRING", + "keyFormat" : { + "format" : "KAFKA" + }, + "valueFormat" : { + "format" : "AVRO" + } + }, + "CSAS_OUTPUT_0.OUTPUT" : { + "schema" : "`K` STRING KEY, `PORT` INTEGER", + "keyFormat" : { + "format" : "KAFKA" + }, + "valueFormat" : { + "format" : "AVRO" + } + } + }, + "testCase" : { + "name" : "extract a port from a URL using URL_EXTRACT_PORT - AVRO", + "inputs" : [ { + "topic" : "test_topic", + "key" : "1", + "value" : { + "url" : "http://test@confluent.io" + }, + "timestamp" : 0 + }, { + "topic" : "test_topic", + "key" : "2", + "value" : { + "url" : "http://test@confluent.io:8080/nested/path?query&jobs" + }, + "timestamp" : 1 + } ], + "outputs" : [ { + "topic" : "OUTPUT", + "key" : "1", + "value" : { + "PORT" : null + }, + "timestamp" : 0 + }, { + "topic" : "OUTPUT", + "key" : "2", + "value" : { + "PORT" : 8080 + }, + "timestamp" : 1 + } ], + "topics" : [ { + "name" : "OUTPUT", + "replicas" : 1, + "numPartitions" : 4 + }, { + "name" : "test_topic", + "valueSchema" : { + "type" : "record", + "name" : "KsqlDataSourceSchema", + "namespace" : "io.confluent.ksql.avro_schemas", + "fields" : [ { + "name" : "URL", + "type" : [ "null", "string" ], + "default" : null + } ], + "connect.name" : "io.confluent.ksql.avro_schemas.KsqlDataSourceSchema" + }, + "valueFormat" : "AVRO", + "replicas" : 1, + "numPartitions" : 4 + } ], + "statements" : [ "CREATE STREAM TEST (K STRING KEY, url VARCHAR) WITH (kafka_topic='test_topic', value_format='AVRO');", "CREATE STREAM OUTPUT AS SELECT K, URL_EXTRACT_PORT(url) as PORT FROM TEST;" ], + "post" : { + "sources" : [ { + "name" : "OUTPUT", + "type" : "STREAM", + "schema" : "`K` STRING KEY, `PORT` INTEGER", + "keyFormat" : { + "format" : "KAFKA" + }, + "valueFormat" : "AVRO", + "keyFeatures" : [ ], + "valueFeatures" : [ ], + "isSource" : false + }, { + "name" : "TEST", + "type" : "STREAM", + "schema" : "`K` STRING KEY, `URL` STRING", + "keyFormat" : { + "format" : "KAFKA" + }, + "valueFormat" : "AVRO", + "keyFeatures" : [ ], + "valueFeatures" : [ ], + "isSource" : false + } ], + "topics" : { + "topics" : [ { + "name" : "test_topic", + "keyFormat" : { + "format" : "KAFKA" + }, + "valueFormat" : { + "format" : "AVRO" + }, + "partitions" : 4, + "valueSchema" : { + "type" : "record", + "name" : "KsqlDataSourceSchema", + "namespace" : "io.confluent.ksql.avro_schemas", + "fields" : [ { + "name" : "URL", + "type" : [ "null", "string" ], + "default" : null + } ], + "connect.name" : "io.confluent.ksql.avro_schemas.KsqlDataSourceSchema" + } + }, { + "name" : "OUTPUT", + "keyFormat" : { + "format" : "KAFKA" + }, + "valueFormat" : { + "format" : "AVRO" + }, + "partitions" : 4, + "valueSchema" : { + "type" : "record", + "name" : "KsqlDataSourceSchema", + "namespace" : "io.confluent.ksql.avro_schemas", + "fields" : [ { + "name" : "PORT", + "type" : [ "null", "int" ], + "default" : null + } ] + } + } ] + } + } + } +} \ No newline at end of file diff --git a/ksqldb-functional-tests/src/test/resources/historical_plans/url_-_extract_a_port_from_a_URL_using_URL_EXTRACT_PORT_-_AVRO/7.3.0_1651070264771/topology b/ksqldb-functional-tests/src/test/resources/historical_plans/url_-_extract_a_port_from_a_URL_using_URL_EXTRACT_PORT_-_AVRO/7.3.0_1651070264771/topology new file mode 100644 index 000000000000..441a8f282644 --- /dev/null +++ b/ksqldb-functional-tests/src/test/resources/historical_plans/url_-_extract_a_port_from_a_URL_using_URL_EXTRACT_PORT_-_AVRO/7.3.0_1651070264771/topology @@ -0,0 +1,13 @@ +Topologies: + Sub-topology: 0 + Source: KSTREAM-SOURCE-0000000000 (topics: [test_topic]) + --> KSTREAM-TRANSFORMVALUES-0000000001 + Processor: KSTREAM-TRANSFORMVALUES-0000000001 (stores: []) + --> Project + <-- KSTREAM-SOURCE-0000000000 + Processor: Project (stores: []) + --> KSTREAM-SINK-0000000003 + <-- KSTREAM-TRANSFORMVALUES-0000000001 + Sink: KSTREAM-SINK-0000000003 (topic: OUTPUT) + <-- Project + diff --git a/ksqldb-functional-tests/src/test/resources/historical_plans/url_-_extract_a_port_from_a_URL_using_URL_EXTRACT_PORT_-_PROTOBUFs_-_PROTOBUF/7.3.0_1651071676718/plan.json b/ksqldb-functional-tests/src/test/resources/historical_plans/url_-_extract_a_port_from_a_URL_using_URL_EXTRACT_PORT_-_PROTOBUFs_-_PROTOBUF/7.3.0_1651071676718/plan.json new file mode 100644 index 000000000000..25cf01b4371d --- /dev/null +++ b/ksqldb-functional-tests/src/test/resources/historical_plans/url_-_extract_a_port_from_a_URL_using_URL_EXTRACT_PORT_-_PROTOBUFs_-_PROTOBUF/7.3.0_1651071676718/plan.json @@ -0,0 +1,211 @@ +{ + "plan" : [ { + "@type" : "ksqlPlanV1", + "statementText" : "CREATE STREAM TEST (K STRING KEY, URL STRING) WITH (KAFKA_TOPIC='test_topic', KEY_FORMAT='KAFKA', VALUE_FORMAT='PROTOBUF');", + "ddlCommand" : { + "@type" : "createStreamV1", + "sourceName" : "TEST", + "schema" : "`K` STRING KEY, `URL` STRING", + "timestampColumn" : null, + "topicName" : "test_topic", + "formats" : { + "keyFormat" : { + "format" : "KAFKA", + "properties" : { } + }, + "valueFormat" : { + "format" : "PROTOBUF", + "properties" : { + "unwrapPrimitives" : "true" + } + } + }, + "windowInfo" : null, + "orReplace" : false, + "isSource" : false + }, + "queryPlan" : null + }, { + "@type" : "ksqlPlanV1", + "statementText" : "CREATE STREAM OUTPUT AS SELECT\n TEST.K K,\n URL_EXTRACT_PORT(TEST.URL) PORT\nFROM TEST TEST\nEMIT CHANGES", + "ddlCommand" : { + "@type" : "createStreamV1", + "sourceName" : "OUTPUT", + "schema" : "`K` STRING KEY, `PORT` INTEGER", + "timestampColumn" : null, + "topicName" : "OUTPUT", + "formats" : { + "keyFormat" : { + "format" : "KAFKA", + "properties" : { } + }, + "valueFormat" : { + "format" : "PROTOBUF", + "properties" : { + "unwrapPrimitives" : "true" + } + } + }, + "windowInfo" : null, + "orReplace" : false, + "isSource" : false + }, + "queryPlan" : { + "sources" : [ "TEST" ], + "sink" : "OUTPUT", + "physicalPlan" : { + "@type" : "streamSinkV1", + "properties" : { + "queryContext" : "OUTPUT" + }, + "source" : { + "@type" : "streamSelectV1", + "properties" : { + "queryContext" : "Project" + }, + "source" : { + "@type" : "streamSourceV1", + "properties" : { + "queryContext" : "KsqlTopic/Source" + }, + "topicName" : "test_topic", + "formats" : { + "keyFormat" : { + "format" : "KAFKA", + "properties" : { } + }, + "valueFormat" : { + "format" : "PROTOBUF", + "properties" : { + "unwrapPrimitives" : "true" + } + } + }, + "timestampColumn" : null, + "sourceSchema" : "`K` STRING KEY, `URL` STRING", + "pseudoColumnVersion" : 1 + }, + "keyColumnNames" : [ "K" ], + "selectedKeys" : null, + "selectExpressions" : [ "URL_EXTRACT_PORT(URL) AS PORT" ] + }, + "formats" : { + "keyFormat" : { + "format" : "KAFKA", + "properties" : { } + }, + "valueFormat" : { + "format" : "PROTOBUF", + "properties" : { + "unwrapPrimitives" : "true" + } + } + }, + "topicName" : "OUTPUT", + "timestampColumn" : null + }, + "queryId" : "CSAS_OUTPUT_0", + "runtimeId" : null + } + } ], + "configs" : { + "ksql.extension.dir" : "ext", + "ksql.streams.cache.max.bytes.buffering" : "0", + "metric.reporters" : "", + "ksql.query.status.running.threshold.seconds" : "300", + "ksql.connect.basic.auth.credentials.reload" : "false", + "ksql.output.topic.name.prefix" : "", + "ksql.query.pull.stream.enabled" : "true", + "ksql.query.push.v2.interpreter.enabled" : "true", + "ksql.queryanonymizer.logs_enabled" : "true", + "ksql.variable.substitution.enable" : "true", + "ksql.streams.shutdown.timeout.ms" : "300000", + "ksql.query.pull.max.allowed.offset.lag" : "9223372036854775807", + "ksql.query.pull.max.qps" : "2147483647", + "ksql.access.validator.enable" : "auto", + "ksql.streams.bootstrap.servers" : "localhost:0", + "ksql.query.pull.metrics.enabled" : "true", + "ksql.metrics.extension" : null, + "ksql.query.push.v2.alos.enabled" : "true", + "ksql.query.push.v2.max.hourly.bandwidth.megabytes" : "2147483647", + "ksql.query.pull.range.scan.enabled" : "true", + "ksql.transient.query.cleanup.service.initial.delay.seconds" : "600", + "ksql.hidden.topics" : "_confluent.*,__confluent.*,_schemas,__consumer_offsets,__transaction_state,connect-configs,connect-offsets,connect-status,connect-statuses", + "ksql.lambdas.enabled" : "true", + "ksql.source.table.materialization.enabled" : "true", + "ksql.query.pull.max.hourly.bandwidth.megabytes" : "2147483647", + "ksql.sink.window.change.log.additional.retention" : "1000000", + "ksql.query.persistent.active.limit" : "2147483647", + "ksql.persistence.wrap.single.values" : null, + "ksql.query.transient.max.bytes.buffering.total" : "-1", + "ksql.connect.basic.auth.credentials.source" : "NONE", + "ksql.schema.registry.url" : "schema_registry.url:0", + "ksql.properties.overrides.denylist" : "", + "ksql.service.id" : "some.ksql.service.id", + "ksql.query.push.v2.max.catchup.consumers" : "5", + "ksql.query.push.v2.enabled" : "false", + "ksql.transient.query.cleanup.service.enable" : "true", + "ksql.query.push.v2.metrics.enabled" : "true", + "ksql.rowpartition.rowoffset.enabled" : "true", + "ksql.streams.commit.interval.ms" : "2000", + "ksql.query.pull.table.scan.enabled" : "true", + "ksql.streams.auto.commit.interval.ms" : "0", + "ksql.streams.topology.optimization" : "all", + "ksql.endpoint.migrate.query" : "true", + "ksql.query.push.v2.registry.installed" : "false", + "ksql.streams.num.stream.threads" : "4", + "ksql.metrics.tags.custom" : "", + "ksql.query.push.v2.catchup.consumer.msg.window" : "50", + "ksql.runtime.feature.shared.enabled" : "false", + "ksql.udf.collect.metrics" : "false", + "ksql.new.query.planner.enabled" : "false", + "ksql.connect.request.headers.plugin" : null, + "ksql.security.extension.class" : null, + "ksql.transient.prefix" : "transient_", + "ksql.headers.columns.enabled" : "true", + "ksql.streams.default.deserialization.exception.handler" : "io.confluent.ksql.errors.LogMetricAndContinueExceptionHandler", + "ksql.connect.request.timeout.ms" : "5000", + "ksql.query.pull.enable.standby.reads" : "false", + "ksql.persistence.default.format.key" : "KAFKA", + "ksql.query.persistent.max.bytes.buffering.total" : "-1", + "ksql.query.error.max.queue.size" : "10", + "ksql.query.cleanup.shutdown.timeout.ms" : "30000", + "ksql.internal.topic.min.insync.replicas" : "1", + "ksql.internal.topic.replicas" : "1", + "ksql.insert.into.values.enabled" : "true", + "ksql.queryanonymizer.cluster_namespace" : null, + "ksql.create.or.replace.enabled" : "true", + "ksql.shared.runtimes.count" : "2", + "ksql.cast.strings.preserve.nulls" : "true", + "ksql.authorization.cache.max.entries" : "10000", + "ksql.pull.queries.enable" : "true", + "ksql.transient.query.cleanup.service.period.seconds" : "600", + "ksql.suppress.enabled" : "false", + "ksql.readonly.topics" : "_confluent.*,__confluent.*,_schemas,__consumer_offsets,__transaction_state,connect-configs,connect-offsets,connect-status,connect-statuses", + "ksql.connect.basic.auth.credentials.file" : "", + "ksql.authorization.cache.expiry.time.secs" : "30", + "ksql.query.retry.backoff.initial.ms" : "15000", + "ksql.query.pull.max.concurrent.requests" : "2147483647", + "ksql.streams.auto.offset.reset" : "earliest", + "ksql.connect.url" : "http://localhost:8083", + "ksql.query.push.v2.new.latest.delay.ms" : "5000", + "ksql.query.push.v2.latest.reset.age.ms" : "30000", + "ksql.streams.default.production.exception.handler" : "io.confluent.ksql.errors.ProductionExceptionHandlerUtil$LogAndFailProductionExceptionHandler", + "ksql.query.pull.interpreter.enabled" : "true", + "ksql.query.pull.limit.clause.enabled" : "true", + "ksql.query.pull.router.thread.pool.size" : "50", + "ksql.query.push.v2.continuation.tokens.enabled" : "false", + "ksql.query.retry.backoff.max.ms" : "900000", + "ksql.timestamp.throw.on.invalid" : "false", + "ksql.persistence.default.format.value" : null, + "ksql.udfs.enabled" : "true", + "ksql.udf.enable.security.manager" : "true", + "ksql.connect.worker.config" : "", + "ksql.nested.error.set.null" : "true", + "ksql.query.pull.thread.pool.size" : "50", + "ksql.persistent.prefix" : "query_", + "ksql.metastore.backup.location" : "", + "ksql.error.classifier.regex" : "", + "ksql.suppress.buffer.size.bytes" : "-1" + } +} \ No newline at end of file diff --git a/ksqldb-functional-tests/src/test/resources/historical_plans/url_-_extract_a_port_from_a_URL_using_URL_EXTRACT_PORT_-_PROTOBUFs_-_PROTOBUF/7.3.0_1651071676718/spec.json b/ksqldb-functional-tests/src/test/resources/historical_plans/url_-_extract_a_port_from_a_URL_using_URL_EXTRACT_PORT_-_PROTOBUFs_-_PROTOBUF/7.3.0_1651071676718/spec.json new file mode 100644 index 000000000000..2f4ec88557c5 --- /dev/null +++ b/ksqldb-functional-tests/src/test/resources/historical_plans/url_-_extract_a_port_from_a_URL_using_URL_EXTRACT_PORT_-_PROTOBUFs_-_PROTOBUF/7.3.0_1651071676718/spec.json @@ -0,0 +1,130 @@ +{ + "version" : "7.3.0", + "timestamp" : 1651071676718, + "path" : "query-validation-tests/url.json", + "schemas" : { + "CSAS_OUTPUT_0.KsqlTopic.Source" : { + "schema" : "`K` STRING KEY, `URL` STRING", + "keyFormat" : { + "format" : "KAFKA" + }, + "valueFormat" : { + "format" : "PROTOBUF", + "properties" : { + "unwrapPrimitives" : "true" + } + } + }, + "CSAS_OUTPUT_0.OUTPUT" : { + "schema" : "`K` STRING KEY, `PORT` INTEGER", + "keyFormat" : { + "format" : "KAFKA" + }, + "valueFormat" : { + "format" : "PROTOBUF", + "properties" : { + "unwrapPrimitives" : "true" + } + } + } + }, + "testCase" : { + "name" : "extract a port from a URL using URL_EXTRACT_PORT - PROTOBUFs - PROTOBUF", + "inputs" : [ { + "topic" : "test_topic", + "key" : "1", + "value" : { + "url" : "http://test@confluent.io" + }, + "timestamp" : 0 + }, { + "topic" : "test_topic", + "key" : "2", + "value" : { + "url" : "http://test@confluent.io:8080/nested/path?query&jobs" + }, + "timestamp" : 1 + } ], + "outputs" : [ { + "topic" : "OUTPUT", + "key" : "1", + "value" : { + "PORT" : 0 + }, + "timestamp" : 0 + }, { + "topic" : "OUTPUT", + "key" : "2", + "value" : { + "PORT" : 8080 + }, + "timestamp" : 1 + } ], + "topics" : [ { + "name" : "OUTPUT", + "replicas" : 1, + "numPartitions" : 4 + }, { + "name" : "test_topic", + "valueSchema" : "syntax = \"proto3\";\n\nmessage ConnectDefault1 {\n string URL = 1;\n}\n", + "valueFormat" : "PROTOBUF", + "replicas" : 1, + "numPartitions" : 4 + } ], + "statements" : [ "CREATE STREAM TEST (K STRING KEY, url VARCHAR) WITH (kafka_topic='test_topic', value_format='PROTOBUF');", "CREATE STREAM OUTPUT AS SELECT K, URL_EXTRACT_PORT(url) as PORT FROM TEST;" ], + "post" : { + "sources" : [ { + "name" : "OUTPUT", + "type" : "STREAM", + "schema" : "`K` STRING KEY, `PORT` INTEGER", + "keyFormat" : { + "format" : "KAFKA" + }, + "valueFormat" : "PROTOBUF", + "keyFeatures" : [ ], + "valueFeatures" : [ ], + "isSource" : false + }, { + "name" : "TEST", + "type" : "STREAM", + "schema" : "`K` STRING KEY, `URL` STRING", + "keyFormat" : { + "format" : "KAFKA" + }, + "valueFormat" : "PROTOBUF", + "keyFeatures" : [ ], + "valueFeatures" : [ ], + "isSource" : false + } ], + "topics" : { + "topics" : [ { + "name" : "test_topic", + "keyFormat" : { + "format" : "KAFKA" + }, + "valueFormat" : { + "format" : "PROTOBUF", + "properties" : { + "unwrapPrimitives" : "true" + } + }, + "partitions" : 4, + "valueSchema" : "syntax = \"proto3\";\n\nmessage ConnectDefault1 {\n string URL = 1;\n}\n" + }, { + "name" : "OUTPUT", + "keyFormat" : { + "format" : "KAFKA" + }, + "valueFormat" : { + "format" : "PROTOBUF", + "properties" : { + "unwrapPrimitives" : "true" + } + }, + "partitions" : 4, + "valueSchema" : "syntax = \"proto3\";\n\nmessage ConnectDefault1 {\n int32 PORT = 1;\n}\n" + } ] + } + } + } +} \ No newline at end of file diff --git a/ksqldb-functional-tests/src/test/resources/historical_plans/url_-_extract_a_port_from_a_URL_using_URL_EXTRACT_PORT_-_PROTOBUFs_-_PROTOBUF/7.3.0_1651071676718/topology b/ksqldb-functional-tests/src/test/resources/historical_plans/url_-_extract_a_port_from_a_URL_using_URL_EXTRACT_PORT_-_PROTOBUFs_-_PROTOBUF/7.3.0_1651071676718/topology new file mode 100644 index 000000000000..441a8f282644 --- /dev/null +++ b/ksqldb-functional-tests/src/test/resources/historical_plans/url_-_extract_a_port_from_a_URL_using_URL_EXTRACT_PORT_-_PROTOBUFs_-_PROTOBUF/7.3.0_1651071676718/topology @@ -0,0 +1,13 @@ +Topologies: + Sub-topology: 0 + Source: KSTREAM-SOURCE-0000000000 (topics: [test_topic]) + --> KSTREAM-TRANSFORMVALUES-0000000001 + Processor: KSTREAM-TRANSFORMVALUES-0000000001 (stores: []) + --> Project + <-- KSTREAM-SOURCE-0000000000 + Processor: Project (stores: []) + --> KSTREAM-SINK-0000000003 + <-- KSTREAM-TRANSFORMVALUES-0000000001 + Sink: KSTREAM-SINK-0000000003 (topic: OUTPUT) + <-- Project + diff --git a/ksqldb-functional-tests/src/test/resources/historical_plans/url_-_extract_a_port_from_a_URL_using_URL_EXTRACT_PORT_-_PROTOBUFs_-_PROTOBUF_NOSR/7.3.0_1651071676744/plan.json b/ksqldb-functional-tests/src/test/resources/historical_plans/url_-_extract_a_port_from_a_URL_using_URL_EXTRACT_PORT_-_PROTOBUFs_-_PROTOBUF_NOSR/7.3.0_1651071676744/plan.json new file mode 100644 index 000000000000..7b2a19eea0f0 --- /dev/null +++ b/ksqldb-functional-tests/src/test/resources/historical_plans/url_-_extract_a_port_from_a_URL_using_URL_EXTRACT_PORT_-_PROTOBUFs_-_PROTOBUF_NOSR/7.3.0_1651071676744/plan.json @@ -0,0 +1,203 @@ +{ + "plan" : [ { + "@type" : "ksqlPlanV1", + "statementText" : "CREATE STREAM TEST (K STRING KEY, URL STRING) WITH (KAFKA_TOPIC='test_topic', KEY_FORMAT='KAFKA', VALUE_FORMAT='PROTOBUF_NOSR');", + "ddlCommand" : { + "@type" : "createStreamV1", + "sourceName" : "TEST", + "schema" : "`K` STRING KEY, `URL` STRING", + "timestampColumn" : null, + "topicName" : "test_topic", + "formats" : { + "keyFormat" : { + "format" : "KAFKA", + "properties" : { } + }, + "valueFormat" : { + "format" : "PROTOBUF_NOSR", + "properties" : { } + } + }, + "windowInfo" : null, + "orReplace" : false, + "isSource" : false + }, + "queryPlan" : null + }, { + "@type" : "ksqlPlanV1", + "statementText" : "CREATE STREAM OUTPUT AS SELECT\n TEST.K K,\n URL_EXTRACT_PORT(TEST.URL) PORT\nFROM TEST TEST\nEMIT CHANGES", + "ddlCommand" : { + "@type" : "createStreamV1", + "sourceName" : "OUTPUT", + "schema" : "`K` STRING KEY, `PORT` INTEGER", + "timestampColumn" : null, + "topicName" : "OUTPUT", + "formats" : { + "keyFormat" : { + "format" : "KAFKA", + "properties" : { } + }, + "valueFormat" : { + "format" : "PROTOBUF_NOSR", + "properties" : { } + } + }, + "windowInfo" : null, + "orReplace" : false, + "isSource" : false + }, + "queryPlan" : { + "sources" : [ "TEST" ], + "sink" : "OUTPUT", + "physicalPlan" : { + "@type" : "streamSinkV1", + "properties" : { + "queryContext" : "OUTPUT" + }, + "source" : { + "@type" : "streamSelectV1", + "properties" : { + "queryContext" : "Project" + }, + "source" : { + "@type" : "streamSourceV1", + "properties" : { + "queryContext" : "KsqlTopic/Source" + }, + "topicName" : "test_topic", + "formats" : { + "keyFormat" : { + "format" : "KAFKA", + "properties" : { } + }, + "valueFormat" : { + "format" : "PROTOBUF_NOSR", + "properties" : { } + } + }, + "timestampColumn" : null, + "sourceSchema" : "`K` STRING KEY, `URL` STRING", + "pseudoColumnVersion" : 1 + }, + "keyColumnNames" : [ "K" ], + "selectedKeys" : null, + "selectExpressions" : [ "URL_EXTRACT_PORT(URL) AS PORT" ] + }, + "formats" : { + "keyFormat" : { + "format" : "KAFKA", + "properties" : { } + }, + "valueFormat" : { + "format" : "PROTOBUF_NOSR", + "properties" : { } + } + }, + "topicName" : "OUTPUT", + "timestampColumn" : null + }, + "queryId" : "CSAS_OUTPUT_0", + "runtimeId" : null + } + } ], + "configs" : { + "ksql.extension.dir" : "ext", + "ksql.streams.cache.max.bytes.buffering" : "0", + "metric.reporters" : "", + "ksql.query.status.running.threshold.seconds" : "300", + "ksql.connect.basic.auth.credentials.reload" : "false", + "ksql.output.topic.name.prefix" : "", + "ksql.query.pull.stream.enabled" : "true", + "ksql.query.push.v2.interpreter.enabled" : "true", + "ksql.queryanonymizer.logs_enabled" : "true", + "ksql.variable.substitution.enable" : "true", + "ksql.streams.shutdown.timeout.ms" : "300000", + "ksql.query.pull.max.allowed.offset.lag" : "9223372036854775807", + "ksql.query.pull.max.qps" : "2147483647", + "ksql.access.validator.enable" : "auto", + "ksql.streams.bootstrap.servers" : "localhost:0", + "ksql.query.pull.metrics.enabled" : "true", + "ksql.metrics.extension" : null, + "ksql.query.push.v2.alos.enabled" : "true", + "ksql.query.push.v2.max.hourly.bandwidth.megabytes" : "2147483647", + "ksql.query.pull.range.scan.enabled" : "true", + "ksql.transient.query.cleanup.service.initial.delay.seconds" : "600", + "ksql.hidden.topics" : "_confluent.*,__confluent.*,_schemas,__consumer_offsets,__transaction_state,connect-configs,connect-offsets,connect-status,connect-statuses", + "ksql.lambdas.enabled" : "true", + "ksql.source.table.materialization.enabled" : "true", + "ksql.query.pull.max.hourly.bandwidth.megabytes" : "2147483647", + "ksql.sink.window.change.log.additional.retention" : "1000000", + "ksql.query.persistent.active.limit" : "2147483647", + "ksql.persistence.wrap.single.values" : null, + "ksql.query.transient.max.bytes.buffering.total" : "-1", + "ksql.connect.basic.auth.credentials.source" : "NONE", + "ksql.schema.registry.url" : "schema_registry.url:0", + "ksql.properties.overrides.denylist" : "", + "ksql.service.id" : "some.ksql.service.id", + "ksql.query.push.v2.max.catchup.consumers" : "5", + "ksql.query.push.v2.enabled" : "false", + "ksql.transient.query.cleanup.service.enable" : "true", + "ksql.query.push.v2.metrics.enabled" : "true", + "ksql.rowpartition.rowoffset.enabled" : "true", + "ksql.streams.commit.interval.ms" : "2000", + "ksql.query.pull.table.scan.enabled" : "true", + "ksql.streams.auto.commit.interval.ms" : "0", + "ksql.streams.topology.optimization" : "all", + "ksql.endpoint.migrate.query" : "true", + "ksql.query.push.v2.registry.installed" : "false", + "ksql.streams.num.stream.threads" : "4", + "ksql.metrics.tags.custom" : "", + "ksql.query.push.v2.catchup.consumer.msg.window" : "50", + "ksql.runtime.feature.shared.enabled" : "false", + "ksql.udf.collect.metrics" : "false", + "ksql.new.query.planner.enabled" : "false", + "ksql.connect.request.headers.plugin" : null, + "ksql.security.extension.class" : null, + "ksql.transient.prefix" : "transient_", + "ksql.headers.columns.enabled" : "true", + "ksql.streams.default.deserialization.exception.handler" : "io.confluent.ksql.errors.LogMetricAndContinueExceptionHandler", + "ksql.connect.request.timeout.ms" : "5000", + "ksql.query.pull.enable.standby.reads" : "false", + "ksql.persistence.default.format.key" : "KAFKA", + "ksql.query.persistent.max.bytes.buffering.total" : "-1", + "ksql.query.error.max.queue.size" : "10", + "ksql.query.cleanup.shutdown.timeout.ms" : "30000", + "ksql.internal.topic.min.insync.replicas" : "1", + "ksql.internal.topic.replicas" : "1", + "ksql.insert.into.values.enabled" : "true", + "ksql.queryanonymizer.cluster_namespace" : null, + "ksql.create.or.replace.enabled" : "true", + "ksql.shared.runtimes.count" : "2", + "ksql.cast.strings.preserve.nulls" : "true", + "ksql.authorization.cache.max.entries" : "10000", + "ksql.pull.queries.enable" : "true", + "ksql.transient.query.cleanup.service.period.seconds" : "600", + "ksql.suppress.enabled" : "false", + "ksql.readonly.topics" : "_confluent.*,__confluent.*,_schemas,__consumer_offsets,__transaction_state,connect-configs,connect-offsets,connect-status,connect-statuses", + "ksql.connect.basic.auth.credentials.file" : "", + "ksql.authorization.cache.expiry.time.secs" : "30", + "ksql.query.retry.backoff.initial.ms" : "15000", + "ksql.query.pull.max.concurrent.requests" : "2147483647", + "ksql.streams.auto.offset.reset" : "earliest", + "ksql.connect.url" : "http://localhost:8083", + "ksql.query.push.v2.new.latest.delay.ms" : "5000", + "ksql.query.push.v2.latest.reset.age.ms" : "30000", + "ksql.streams.default.production.exception.handler" : "io.confluent.ksql.errors.ProductionExceptionHandlerUtil$LogAndFailProductionExceptionHandler", + "ksql.query.pull.interpreter.enabled" : "true", + "ksql.query.pull.limit.clause.enabled" : "true", + "ksql.query.pull.router.thread.pool.size" : "50", + "ksql.query.push.v2.continuation.tokens.enabled" : "false", + "ksql.query.retry.backoff.max.ms" : "900000", + "ksql.timestamp.throw.on.invalid" : "false", + "ksql.persistence.default.format.value" : null, + "ksql.udfs.enabled" : "true", + "ksql.udf.enable.security.manager" : "true", + "ksql.connect.worker.config" : "", + "ksql.nested.error.set.null" : "true", + "ksql.query.pull.thread.pool.size" : "50", + "ksql.persistent.prefix" : "query_", + "ksql.metastore.backup.location" : "", + "ksql.error.classifier.regex" : "", + "ksql.suppress.buffer.size.bytes" : "-1" + } +} \ No newline at end of file diff --git a/ksqldb-functional-tests/src/test/resources/historical_plans/url_-_extract_a_port_from_a_URL_using_URL_EXTRACT_PORT_-_PROTOBUFs_-_PROTOBUF_NOSR/7.3.0_1651071676744/spec.json b/ksqldb-functional-tests/src/test/resources/historical_plans/url_-_extract_a_port_from_a_URL_using_URL_EXTRACT_PORT_-_PROTOBUFs_-_PROTOBUF_NOSR/7.3.0_1651071676744/spec.json new file mode 100644 index 000000000000..83cc450c87b7 --- /dev/null +++ b/ksqldb-functional-tests/src/test/resources/historical_plans/url_-_extract_a_port_from_a_URL_using_URL_EXTRACT_PORT_-_PROTOBUFs_-_PROTOBUF_NOSR/7.3.0_1651071676744/spec.json @@ -0,0 +1,114 @@ +{ + "version" : "7.3.0", + "timestamp" : 1651071676744, + "path" : "query-validation-tests/url.json", + "schemas" : { + "CSAS_OUTPUT_0.KsqlTopic.Source" : { + "schema" : "`K` STRING KEY, `URL` STRING", + "keyFormat" : { + "format" : "KAFKA" + }, + "valueFormat" : { + "format" : "PROTOBUF_NOSR" + } + }, + "CSAS_OUTPUT_0.OUTPUT" : { + "schema" : "`K` STRING KEY, `PORT` INTEGER", + "keyFormat" : { + "format" : "KAFKA" + }, + "valueFormat" : { + "format" : "PROTOBUF_NOSR" + } + } + }, + "testCase" : { + "name" : "extract a port from a URL using URL_EXTRACT_PORT - PROTOBUFs - PROTOBUF_NOSR", + "inputs" : [ { + "topic" : "test_topic", + "key" : "1", + "value" : { + "url" : "http://test@confluent.io" + }, + "timestamp" : 0 + }, { + "topic" : "test_topic", + "key" : "2", + "value" : { + "url" : "http://test@confluent.io:8080/nested/path?query&jobs" + }, + "timestamp" : 1 + } ], + "outputs" : [ { + "topic" : "OUTPUT", + "key" : "1", + "value" : { + "PORT" : 0 + }, + "timestamp" : 0 + }, { + "topic" : "OUTPUT", + "key" : "2", + "value" : { + "PORT" : 8080 + }, + "timestamp" : 1 + } ], + "topics" : [ { + "name" : "OUTPUT", + "replicas" : 1, + "numPartitions" : 4 + }, { + "name" : "test_topic", + "replicas" : 1, + "numPartitions" : 4 + } ], + "statements" : [ "CREATE STREAM TEST (K STRING KEY, url VARCHAR) WITH (kafka_topic='test_topic', value_format='PROTOBUF_NOSR');", "CREATE STREAM OUTPUT AS SELECT K, URL_EXTRACT_PORT(url) as PORT FROM TEST;" ], + "post" : { + "sources" : [ { + "name" : "OUTPUT", + "type" : "STREAM", + "schema" : "`K` STRING KEY, `PORT` INTEGER", + "keyFormat" : { + "format" : "KAFKA" + }, + "valueFormat" : "PROTOBUF_NOSR", + "keyFeatures" : [ ], + "valueFeatures" : [ ], + "isSource" : false + }, { + "name" : "TEST", + "type" : "STREAM", + "schema" : "`K` STRING KEY, `URL` STRING", + "keyFormat" : { + "format" : "KAFKA" + }, + "valueFormat" : "PROTOBUF_NOSR", + "keyFeatures" : [ ], + "valueFeatures" : [ ], + "isSource" : false + } ], + "topics" : { + "topics" : [ { + "name" : "test_topic", + "keyFormat" : { + "format" : "KAFKA" + }, + "valueFormat" : { + "format" : "PROTOBUF_NOSR" + }, + "partitions" : 4 + }, { + "name" : "OUTPUT", + "keyFormat" : { + "format" : "KAFKA" + }, + "valueFormat" : { + "format" : "PROTOBUF_NOSR" + }, + "partitions" : 4 + } ] + } + } + } +} \ No newline at end of file diff --git a/ksqldb-functional-tests/src/test/resources/historical_plans/url_-_extract_a_port_from_a_URL_using_URL_EXTRACT_PORT_-_PROTOBUFs_-_PROTOBUF_NOSR/7.3.0_1651071676744/topology b/ksqldb-functional-tests/src/test/resources/historical_plans/url_-_extract_a_port_from_a_URL_using_URL_EXTRACT_PORT_-_PROTOBUFs_-_PROTOBUF_NOSR/7.3.0_1651071676744/topology new file mode 100644 index 000000000000..441a8f282644 --- /dev/null +++ b/ksqldb-functional-tests/src/test/resources/historical_plans/url_-_extract_a_port_from_a_URL_using_URL_EXTRACT_PORT_-_PROTOBUFs_-_PROTOBUF_NOSR/7.3.0_1651071676744/topology @@ -0,0 +1,13 @@ +Topologies: + Sub-topology: 0 + Source: KSTREAM-SOURCE-0000000000 (topics: [test_topic]) + --> KSTREAM-TRANSFORMVALUES-0000000001 + Processor: KSTREAM-TRANSFORMVALUES-0000000001 (stores: []) + --> Project + <-- KSTREAM-SOURCE-0000000000 + Processor: Project (stores: []) + --> KSTREAM-SINK-0000000003 + <-- KSTREAM-TRANSFORMVALUES-0000000001 + Sink: KSTREAM-SINK-0000000003 (topic: OUTPUT) + <-- Project + diff --git a/ksqldb-functional-tests/src/test/resources/historical_plans/url_-_extract_a_protocol_from_a_URL_using_URL_EXTRACT_PROTOCOL_-_AVRO/7.3.0_1651071676796/plan.json b/ksqldb-functional-tests/src/test/resources/historical_plans/url_-_extract_a_protocol_from_a_URL_using_URL_EXTRACT_PROTOCOL_-_AVRO/7.3.0_1651071676796/plan.json new file mode 100644 index 000000000000..ef2070b8ade1 --- /dev/null +++ b/ksqldb-functional-tests/src/test/resources/historical_plans/url_-_extract_a_protocol_from_a_URL_using_URL_EXTRACT_PROTOCOL_-_AVRO/7.3.0_1651071676796/plan.json @@ -0,0 +1,203 @@ +{ + "plan" : [ { + "@type" : "ksqlPlanV1", + "statementText" : "CREATE STREAM TEST (K STRING KEY, URL STRING) WITH (KAFKA_TOPIC='test_topic', KEY_FORMAT='KAFKA', VALUE_FORMAT='AVRO');", + "ddlCommand" : { + "@type" : "createStreamV1", + "sourceName" : "TEST", + "schema" : "`K` STRING KEY, `URL` STRING", + "timestampColumn" : null, + "topicName" : "test_topic", + "formats" : { + "keyFormat" : { + "format" : "KAFKA", + "properties" : { } + }, + "valueFormat" : { + "format" : "AVRO", + "properties" : { } + } + }, + "windowInfo" : null, + "orReplace" : false, + "isSource" : false + }, + "queryPlan" : null + }, { + "@type" : "ksqlPlanV1", + "statementText" : "CREATE STREAM OUTPUT AS SELECT\n TEST.K K,\n URL_EXTRACT_PROTOCOL(TEST.URL) PROTOCOL\nFROM TEST TEST\nEMIT CHANGES", + "ddlCommand" : { + "@type" : "createStreamV1", + "sourceName" : "OUTPUT", + "schema" : "`K` STRING KEY, `PROTOCOL` STRING", + "timestampColumn" : null, + "topicName" : "OUTPUT", + "formats" : { + "keyFormat" : { + "format" : "KAFKA", + "properties" : { } + }, + "valueFormat" : { + "format" : "AVRO", + "properties" : { } + } + }, + "windowInfo" : null, + "orReplace" : false, + "isSource" : false + }, + "queryPlan" : { + "sources" : [ "TEST" ], + "sink" : "OUTPUT", + "physicalPlan" : { + "@type" : "streamSinkV1", + "properties" : { + "queryContext" : "OUTPUT" + }, + "source" : { + "@type" : "streamSelectV1", + "properties" : { + "queryContext" : "Project" + }, + "source" : { + "@type" : "streamSourceV1", + "properties" : { + "queryContext" : "KsqlTopic/Source" + }, + "topicName" : "test_topic", + "formats" : { + "keyFormat" : { + "format" : "KAFKA", + "properties" : { } + }, + "valueFormat" : { + "format" : "AVRO", + "properties" : { } + } + }, + "timestampColumn" : null, + "sourceSchema" : "`K` STRING KEY, `URL` STRING", + "pseudoColumnVersion" : 1 + }, + "keyColumnNames" : [ "K" ], + "selectedKeys" : null, + "selectExpressions" : [ "URL_EXTRACT_PROTOCOL(URL) AS PROTOCOL" ] + }, + "formats" : { + "keyFormat" : { + "format" : "KAFKA", + "properties" : { } + }, + "valueFormat" : { + "format" : "AVRO", + "properties" : { } + } + }, + "topicName" : "OUTPUT", + "timestampColumn" : null + }, + "queryId" : "CSAS_OUTPUT_0", + "runtimeId" : null + } + } ], + "configs" : { + "ksql.extension.dir" : "ext", + "ksql.streams.cache.max.bytes.buffering" : "0", + "metric.reporters" : "", + "ksql.query.status.running.threshold.seconds" : "300", + "ksql.connect.basic.auth.credentials.reload" : "false", + "ksql.output.topic.name.prefix" : "", + "ksql.query.pull.stream.enabled" : "true", + "ksql.query.push.v2.interpreter.enabled" : "true", + "ksql.queryanonymizer.logs_enabled" : "true", + "ksql.variable.substitution.enable" : "true", + "ksql.streams.shutdown.timeout.ms" : "300000", + "ksql.query.pull.max.allowed.offset.lag" : "9223372036854775807", + "ksql.query.pull.max.qps" : "2147483647", + "ksql.access.validator.enable" : "auto", + "ksql.streams.bootstrap.servers" : "localhost:0", + "ksql.query.pull.metrics.enabled" : "true", + "ksql.metrics.extension" : null, + "ksql.query.push.v2.alos.enabled" : "true", + "ksql.query.push.v2.max.hourly.bandwidth.megabytes" : "2147483647", + "ksql.query.pull.range.scan.enabled" : "true", + "ksql.transient.query.cleanup.service.initial.delay.seconds" : "600", + "ksql.hidden.topics" : "_confluent.*,__confluent.*,_schemas,__consumer_offsets,__transaction_state,connect-configs,connect-offsets,connect-status,connect-statuses", + "ksql.lambdas.enabled" : "true", + "ksql.source.table.materialization.enabled" : "true", + "ksql.query.pull.max.hourly.bandwidth.megabytes" : "2147483647", + "ksql.sink.window.change.log.additional.retention" : "1000000", + "ksql.query.persistent.active.limit" : "2147483647", + "ksql.persistence.wrap.single.values" : null, + "ksql.query.transient.max.bytes.buffering.total" : "-1", + "ksql.connect.basic.auth.credentials.source" : "NONE", + "ksql.schema.registry.url" : "schema_registry.url:0", + "ksql.properties.overrides.denylist" : "", + "ksql.service.id" : "some.ksql.service.id", + "ksql.query.push.v2.max.catchup.consumers" : "5", + "ksql.query.push.v2.enabled" : "false", + "ksql.transient.query.cleanup.service.enable" : "true", + "ksql.query.push.v2.metrics.enabled" : "true", + "ksql.rowpartition.rowoffset.enabled" : "true", + "ksql.streams.commit.interval.ms" : "2000", + "ksql.query.pull.table.scan.enabled" : "true", + "ksql.streams.auto.commit.interval.ms" : "0", + "ksql.streams.topology.optimization" : "all", + "ksql.endpoint.migrate.query" : "true", + "ksql.query.push.v2.registry.installed" : "false", + "ksql.streams.num.stream.threads" : "4", + "ksql.metrics.tags.custom" : "", + "ksql.query.push.v2.catchup.consumer.msg.window" : "50", + "ksql.runtime.feature.shared.enabled" : "false", + "ksql.udf.collect.metrics" : "false", + "ksql.new.query.planner.enabled" : "false", + "ksql.connect.request.headers.plugin" : null, + "ksql.security.extension.class" : null, + "ksql.transient.prefix" : "transient_", + "ksql.headers.columns.enabled" : "true", + "ksql.streams.default.deserialization.exception.handler" : "io.confluent.ksql.errors.LogMetricAndContinueExceptionHandler", + "ksql.connect.request.timeout.ms" : "5000", + "ksql.query.pull.enable.standby.reads" : "false", + "ksql.persistence.default.format.key" : "KAFKA", + "ksql.query.persistent.max.bytes.buffering.total" : "-1", + "ksql.query.error.max.queue.size" : "10", + "ksql.query.cleanup.shutdown.timeout.ms" : "30000", + "ksql.internal.topic.min.insync.replicas" : "1", + "ksql.internal.topic.replicas" : "1", + "ksql.insert.into.values.enabled" : "true", + "ksql.queryanonymizer.cluster_namespace" : null, + "ksql.create.or.replace.enabled" : "true", + "ksql.shared.runtimes.count" : "2", + "ksql.cast.strings.preserve.nulls" : "true", + "ksql.authorization.cache.max.entries" : "10000", + "ksql.pull.queries.enable" : "true", + "ksql.transient.query.cleanup.service.period.seconds" : "600", + "ksql.suppress.enabled" : "false", + "ksql.readonly.topics" : "_confluent.*,__confluent.*,_schemas,__consumer_offsets,__transaction_state,connect-configs,connect-offsets,connect-status,connect-statuses", + "ksql.connect.basic.auth.credentials.file" : "", + "ksql.authorization.cache.expiry.time.secs" : "30", + "ksql.query.retry.backoff.initial.ms" : "15000", + "ksql.query.pull.max.concurrent.requests" : "2147483647", + "ksql.streams.auto.offset.reset" : "earliest", + "ksql.connect.url" : "http://localhost:8083", + "ksql.query.push.v2.new.latest.delay.ms" : "5000", + "ksql.query.push.v2.latest.reset.age.ms" : "30000", + "ksql.streams.default.production.exception.handler" : "io.confluent.ksql.errors.ProductionExceptionHandlerUtil$LogAndFailProductionExceptionHandler", + "ksql.query.pull.interpreter.enabled" : "true", + "ksql.query.pull.limit.clause.enabled" : "true", + "ksql.query.pull.router.thread.pool.size" : "50", + "ksql.query.push.v2.continuation.tokens.enabled" : "false", + "ksql.query.retry.backoff.max.ms" : "900000", + "ksql.timestamp.throw.on.invalid" : "false", + "ksql.persistence.default.format.value" : null, + "ksql.udfs.enabled" : "true", + "ksql.udf.enable.security.manager" : "true", + "ksql.connect.worker.config" : "", + "ksql.nested.error.set.null" : "true", + "ksql.query.pull.thread.pool.size" : "50", + "ksql.persistent.prefix" : "query_", + "ksql.metastore.backup.location" : "", + "ksql.error.classifier.regex" : "", + "ksql.suppress.buffer.size.bytes" : "-1" + } +} \ No newline at end of file diff --git a/ksqldb-functional-tests/src/test/resources/historical_plans/url_-_extract_a_protocol_from_a_URL_using_URL_EXTRACT_PROTOCOL_-_AVRO/7.3.0_1651071676796/spec.json b/ksqldb-functional-tests/src/test/resources/historical_plans/url_-_extract_a_protocol_from_a_URL_using_URL_EXTRACT_PROTOCOL_-_AVRO/7.3.0_1651071676796/spec.json new file mode 100644 index 000000000000..9035c81c74e8 --- /dev/null +++ b/ksqldb-functional-tests/src/test/resources/historical_plans/url_-_extract_a_protocol_from_a_URL_using_URL_EXTRACT_PROTOCOL_-_AVRO/7.3.0_1651071676796/spec.json @@ -0,0 +1,161 @@ +{ + "version" : "7.3.0", + "timestamp" : 1651071676796, + "path" : "query-validation-tests/url.json", + "schemas" : { + "CSAS_OUTPUT_0.KsqlTopic.Source" : { + "schema" : "`K` STRING KEY, `URL` STRING", + "keyFormat" : { + "format" : "KAFKA" + }, + "valueFormat" : { + "format" : "AVRO" + } + }, + "CSAS_OUTPUT_0.OUTPUT" : { + "schema" : "`K` STRING KEY, `PROTOCOL` STRING", + "keyFormat" : { + "format" : "KAFKA" + }, + "valueFormat" : { + "format" : "AVRO" + } + } + }, + "testCase" : { + "name" : "extract a protocol from a URL using URL_EXTRACT_PROTOCOL - AVRO", + "inputs" : [ { + "topic" : "test_topic", + "key" : "1", + "value" : { + "url" : "http://test@confluent.io" + }, + "timestamp" : 0 + }, { + "topic" : "test_topic", + "key" : "2", + "value" : { + "url" : "https://test@confluent.io:8080/nested/path?query&jobs" + }, + "timestamp" : 1 + }, { + "topic" : "test_topic", + "key" : "3", + "value" : { + "url" : "www.confluent.io" + }, + "timestamp" : 2 + } ], + "outputs" : [ { + "topic" : "OUTPUT", + "key" : "1", + "value" : { + "PROTOCOL" : "http" + }, + "timestamp" : 0 + }, { + "topic" : "OUTPUT", + "key" : "2", + "value" : { + "PROTOCOL" : "https" + }, + "timestamp" : 1 + }, { + "topic" : "OUTPUT", + "key" : "3", + "value" : { + "PROTOCOL" : null + }, + "timestamp" : 2 + } ], + "topics" : [ { + "name" : "OUTPUT", + "replicas" : 1, + "numPartitions" : 4 + }, { + "name" : "test_topic", + "valueSchema" : { + "type" : "record", + "name" : "KsqlDataSourceSchema", + "namespace" : "io.confluent.ksql.avro_schemas", + "fields" : [ { + "name" : "URL", + "type" : [ "null", "string" ], + "default" : null + } ], + "connect.name" : "io.confluent.ksql.avro_schemas.KsqlDataSourceSchema" + }, + "valueFormat" : "AVRO", + "replicas" : 1, + "numPartitions" : 4 + } ], + "statements" : [ "CREATE STREAM TEST (K STRING KEY, url VARCHAR) WITH (kafka_topic='test_topic', value_format='AVRO');", "CREATE STREAM OUTPUT AS SELECT K, URL_EXTRACT_PROTOCOL(url) as PROTOCOL FROM TEST;" ], + "post" : { + "sources" : [ { + "name" : "OUTPUT", + "type" : "STREAM", + "schema" : "`K` STRING KEY, `PROTOCOL` STRING", + "keyFormat" : { + "format" : "KAFKA" + }, + "valueFormat" : "AVRO", + "keyFeatures" : [ ], + "valueFeatures" : [ ], + "isSource" : false + }, { + "name" : "TEST", + "type" : "STREAM", + "schema" : "`K` STRING KEY, `URL` STRING", + "keyFormat" : { + "format" : "KAFKA" + }, + "valueFormat" : "AVRO", + "keyFeatures" : [ ], + "valueFeatures" : [ ], + "isSource" : false + } ], + "topics" : { + "topics" : [ { + "name" : "test_topic", + "keyFormat" : { + "format" : "KAFKA" + }, + "valueFormat" : { + "format" : "AVRO" + }, + "partitions" : 4, + "valueSchema" : { + "type" : "record", + "name" : "KsqlDataSourceSchema", + "namespace" : "io.confluent.ksql.avro_schemas", + "fields" : [ { + "name" : "URL", + "type" : [ "null", "string" ], + "default" : null + } ], + "connect.name" : "io.confluent.ksql.avro_schemas.KsqlDataSourceSchema" + } + }, { + "name" : "OUTPUT", + "keyFormat" : { + "format" : "KAFKA" + }, + "valueFormat" : { + "format" : "AVRO" + }, + "partitions" : 4, + "valueSchema" : { + "type" : "record", + "name" : "KsqlDataSourceSchema", + "namespace" : "io.confluent.ksql.avro_schemas", + "fields" : [ { + "name" : "PROTOCOL", + "type" : [ "null", "string" ], + "default" : null + } ] + } + } ] + } + } + } +} \ No newline at end of file diff --git a/ksqldb-functional-tests/src/test/resources/historical_plans/url_-_extract_a_protocol_from_a_URL_using_URL_EXTRACT_PROTOCOL_-_AVRO/7.3.0_1651071676796/topology b/ksqldb-functional-tests/src/test/resources/historical_plans/url_-_extract_a_protocol_from_a_URL_using_URL_EXTRACT_PROTOCOL_-_AVRO/7.3.0_1651071676796/topology new file mode 100644 index 000000000000..441a8f282644 --- /dev/null +++ b/ksqldb-functional-tests/src/test/resources/historical_plans/url_-_extract_a_protocol_from_a_URL_using_URL_EXTRACT_PROTOCOL_-_AVRO/7.3.0_1651071676796/topology @@ -0,0 +1,13 @@ +Topologies: + Sub-topology: 0 + Source: KSTREAM-SOURCE-0000000000 (topics: [test_topic]) + --> KSTREAM-TRANSFORMVALUES-0000000001 + Processor: KSTREAM-TRANSFORMVALUES-0000000001 (stores: []) + --> Project + <-- KSTREAM-SOURCE-0000000000 + Processor: Project (stores: []) + --> KSTREAM-SINK-0000000003 + <-- KSTREAM-TRANSFORMVALUES-0000000001 + Sink: KSTREAM-SINK-0000000003 (topic: OUTPUT) + <-- Project + diff --git a/ksqldb-functional-tests/src/test/resources/historical_plans/url_-_extract_a_protocol_from_a_URL_using_URL_EXTRACT_PROTOCOL_-_PROTOBUFs_-_PROTOBUF/7.3.0_1651071676835/plan.json b/ksqldb-functional-tests/src/test/resources/historical_plans/url_-_extract_a_protocol_from_a_URL_using_URL_EXTRACT_PROTOCOL_-_PROTOBUFs_-_PROTOBUF/7.3.0_1651071676835/plan.json new file mode 100644 index 000000000000..29a85028472c --- /dev/null +++ b/ksqldb-functional-tests/src/test/resources/historical_plans/url_-_extract_a_protocol_from_a_URL_using_URL_EXTRACT_PROTOCOL_-_PROTOBUFs_-_PROTOBUF/7.3.0_1651071676835/plan.json @@ -0,0 +1,211 @@ +{ + "plan" : [ { + "@type" : "ksqlPlanV1", + "statementText" : "CREATE STREAM TEST (K STRING KEY, URL STRING) WITH (KAFKA_TOPIC='test_topic', KEY_FORMAT='KAFKA', VALUE_FORMAT='PROTOBUF');", + "ddlCommand" : { + "@type" : "createStreamV1", + "sourceName" : "TEST", + "schema" : "`K` STRING KEY, `URL` STRING", + "timestampColumn" : null, + "topicName" : "test_topic", + "formats" : { + "keyFormat" : { + "format" : "KAFKA", + "properties" : { } + }, + "valueFormat" : { + "format" : "PROTOBUF", + "properties" : { + "unwrapPrimitives" : "true" + } + } + }, + "windowInfo" : null, + "orReplace" : false, + "isSource" : false + }, + "queryPlan" : null + }, { + "@type" : "ksqlPlanV1", + "statementText" : "CREATE STREAM OUTPUT AS SELECT\n TEST.K K,\n URL_EXTRACT_PROTOCOL(TEST.URL) PROTOCOL\nFROM TEST TEST\nEMIT CHANGES", + "ddlCommand" : { + "@type" : "createStreamV1", + "sourceName" : "OUTPUT", + "schema" : "`K` STRING KEY, `PROTOCOL` STRING", + "timestampColumn" : null, + "topicName" : "OUTPUT", + "formats" : { + "keyFormat" : { + "format" : "KAFKA", + "properties" : { } + }, + "valueFormat" : { + "format" : "PROTOBUF", + "properties" : { + "unwrapPrimitives" : "true" + } + } + }, + "windowInfo" : null, + "orReplace" : false, + "isSource" : false + }, + "queryPlan" : { + "sources" : [ "TEST" ], + "sink" : "OUTPUT", + "physicalPlan" : { + "@type" : "streamSinkV1", + "properties" : { + "queryContext" : "OUTPUT" + }, + "source" : { + "@type" : "streamSelectV1", + "properties" : { + "queryContext" : "Project" + }, + "source" : { + "@type" : "streamSourceV1", + "properties" : { + "queryContext" : "KsqlTopic/Source" + }, + "topicName" : "test_topic", + "formats" : { + "keyFormat" : { + "format" : "KAFKA", + "properties" : { } + }, + "valueFormat" : { + "format" : "PROTOBUF", + "properties" : { + "unwrapPrimitives" : "true" + } + } + }, + "timestampColumn" : null, + "sourceSchema" : "`K` STRING KEY, `URL` STRING", + "pseudoColumnVersion" : 1 + }, + "keyColumnNames" : [ "K" ], + "selectedKeys" : null, + "selectExpressions" : [ "URL_EXTRACT_PROTOCOL(URL) AS PROTOCOL" ] + }, + "formats" : { + "keyFormat" : { + "format" : "KAFKA", + "properties" : { } + }, + "valueFormat" : { + "format" : "PROTOBUF", + "properties" : { + "unwrapPrimitives" : "true" + } + } + }, + "topicName" : "OUTPUT", + "timestampColumn" : null + }, + "queryId" : "CSAS_OUTPUT_0", + "runtimeId" : null + } + } ], + "configs" : { + "ksql.extension.dir" : "ext", + "ksql.streams.cache.max.bytes.buffering" : "0", + "metric.reporters" : "", + "ksql.query.status.running.threshold.seconds" : "300", + "ksql.connect.basic.auth.credentials.reload" : "false", + "ksql.output.topic.name.prefix" : "", + "ksql.query.pull.stream.enabled" : "true", + "ksql.query.push.v2.interpreter.enabled" : "true", + "ksql.queryanonymizer.logs_enabled" : "true", + "ksql.variable.substitution.enable" : "true", + "ksql.streams.shutdown.timeout.ms" : "300000", + "ksql.query.pull.max.allowed.offset.lag" : "9223372036854775807", + "ksql.query.pull.max.qps" : "2147483647", + "ksql.access.validator.enable" : "auto", + "ksql.streams.bootstrap.servers" : "localhost:0", + "ksql.query.pull.metrics.enabled" : "true", + "ksql.metrics.extension" : null, + "ksql.query.push.v2.alos.enabled" : "true", + "ksql.query.push.v2.max.hourly.bandwidth.megabytes" : "2147483647", + "ksql.query.pull.range.scan.enabled" : "true", + "ksql.transient.query.cleanup.service.initial.delay.seconds" : "600", + "ksql.hidden.topics" : "_confluent.*,__confluent.*,_schemas,__consumer_offsets,__transaction_state,connect-configs,connect-offsets,connect-status,connect-statuses", + "ksql.lambdas.enabled" : "true", + "ksql.source.table.materialization.enabled" : "true", + "ksql.query.pull.max.hourly.bandwidth.megabytes" : "2147483647", + "ksql.sink.window.change.log.additional.retention" : "1000000", + "ksql.query.persistent.active.limit" : "2147483647", + "ksql.persistence.wrap.single.values" : null, + "ksql.query.transient.max.bytes.buffering.total" : "-1", + "ksql.connect.basic.auth.credentials.source" : "NONE", + "ksql.schema.registry.url" : "schema_registry.url:0", + "ksql.properties.overrides.denylist" : "", + "ksql.service.id" : "some.ksql.service.id", + "ksql.query.push.v2.max.catchup.consumers" : "5", + "ksql.query.push.v2.enabled" : "false", + "ksql.transient.query.cleanup.service.enable" : "true", + "ksql.query.push.v2.metrics.enabled" : "true", + "ksql.rowpartition.rowoffset.enabled" : "true", + "ksql.streams.commit.interval.ms" : "2000", + "ksql.query.pull.table.scan.enabled" : "true", + "ksql.streams.auto.commit.interval.ms" : "0", + "ksql.streams.topology.optimization" : "all", + "ksql.endpoint.migrate.query" : "true", + "ksql.query.push.v2.registry.installed" : "false", + "ksql.streams.num.stream.threads" : "4", + "ksql.metrics.tags.custom" : "", + "ksql.query.push.v2.catchup.consumer.msg.window" : "50", + "ksql.runtime.feature.shared.enabled" : "false", + "ksql.udf.collect.metrics" : "false", + "ksql.new.query.planner.enabled" : "false", + "ksql.connect.request.headers.plugin" : null, + "ksql.security.extension.class" : null, + "ksql.transient.prefix" : "transient_", + "ksql.headers.columns.enabled" : "true", + "ksql.streams.default.deserialization.exception.handler" : "io.confluent.ksql.errors.LogMetricAndContinueExceptionHandler", + "ksql.connect.request.timeout.ms" : "5000", + "ksql.query.pull.enable.standby.reads" : "false", + "ksql.persistence.default.format.key" : "KAFKA", + "ksql.query.persistent.max.bytes.buffering.total" : "-1", + "ksql.query.error.max.queue.size" : "10", + "ksql.query.cleanup.shutdown.timeout.ms" : "30000", + "ksql.internal.topic.min.insync.replicas" : "1", + "ksql.internal.topic.replicas" : "1", + "ksql.insert.into.values.enabled" : "true", + "ksql.queryanonymizer.cluster_namespace" : null, + "ksql.create.or.replace.enabled" : "true", + "ksql.shared.runtimes.count" : "2", + "ksql.cast.strings.preserve.nulls" : "true", + "ksql.authorization.cache.max.entries" : "10000", + "ksql.pull.queries.enable" : "true", + "ksql.transient.query.cleanup.service.period.seconds" : "600", + "ksql.suppress.enabled" : "false", + "ksql.readonly.topics" : "_confluent.*,__confluent.*,_schemas,__consumer_offsets,__transaction_state,connect-configs,connect-offsets,connect-status,connect-statuses", + "ksql.connect.basic.auth.credentials.file" : "", + "ksql.authorization.cache.expiry.time.secs" : "30", + "ksql.query.retry.backoff.initial.ms" : "15000", + "ksql.query.pull.max.concurrent.requests" : "2147483647", + "ksql.streams.auto.offset.reset" : "earliest", + "ksql.connect.url" : "http://localhost:8083", + "ksql.query.push.v2.new.latest.delay.ms" : "5000", + "ksql.query.push.v2.latest.reset.age.ms" : "30000", + "ksql.streams.default.production.exception.handler" : "io.confluent.ksql.errors.ProductionExceptionHandlerUtil$LogAndFailProductionExceptionHandler", + "ksql.query.pull.interpreter.enabled" : "true", + "ksql.query.pull.limit.clause.enabled" : "true", + "ksql.query.pull.router.thread.pool.size" : "50", + "ksql.query.push.v2.continuation.tokens.enabled" : "false", + "ksql.query.retry.backoff.max.ms" : "900000", + "ksql.timestamp.throw.on.invalid" : "false", + "ksql.persistence.default.format.value" : null, + "ksql.udfs.enabled" : "true", + "ksql.udf.enable.security.manager" : "true", + "ksql.connect.worker.config" : "", + "ksql.nested.error.set.null" : "true", + "ksql.query.pull.thread.pool.size" : "50", + "ksql.persistent.prefix" : "query_", + "ksql.metastore.backup.location" : "", + "ksql.error.classifier.regex" : "", + "ksql.suppress.buffer.size.bytes" : "-1" + } +} \ No newline at end of file diff --git a/ksqldb-functional-tests/src/test/resources/historical_plans/url_-_extract_a_protocol_from_a_URL_using_URL_EXTRACT_PROTOCOL_-_PROTOBUFs_-_PROTOBUF/7.3.0_1651071676835/spec.json b/ksqldb-functional-tests/src/test/resources/historical_plans/url_-_extract_a_protocol_from_a_URL_using_URL_EXTRACT_PROTOCOL_-_PROTOBUFs_-_PROTOBUF/7.3.0_1651071676835/spec.json new file mode 100644 index 000000000000..559740c9b683 --- /dev/null +++ b/ksqldb-functional-tests/src/test/resources/historical_plans/url_-_extract_a_protocol_from_a_URL_using_URL_EXTRACT_PROTOCOL_-_PROTOBUFs_-_PROTOBUF/7.3.0_1651071676835/spec.json @@ -0,0 +1,144 @@ +{ + "version" : "7.3.0", + "timestamp" : 1651071676835, + "path" : "query-validation-tests/url.json", + "schemas" : { + "CSAS_OUTPUT_0.KsqlTopic.Source" : { + "schema" : "`K` STRING KEY, `URL` STRING", + "keyFormat" : { + "format" : "KAFKA" + }, + "valueFormat" : { + "format" : "PROTOBUF", + "properties" : { + "unwrapPrimitives" : "true" + } + } + }, + "CSAS_OUTPUT_0.OUTPUT" : { + "schema" : "`K` STRING KEY, `PROTOCOL` STRING", + "keyFormat" : { + "format" : "KAFKA" + }, + "valueFormat" : { + "format" : "PROTOBUF", + "properties" : { + "unwrapPrimitives" : "true" + } + } + } + }, + "testCase" : { + "name" : "extract a protocol from a URL using URL_EXTRACT_PROTOCOL - PROTOBUFs - PROTOBUF", + "inputs" : [ { + "topic" : "test_topic", + "key" : "1", + "value" : { + "url" : "http://test@confluent.io" + }, + "timestamp" : 0 + }, { + "topic" : "test_topic", + "key" : "2", + "value" : { + "url" : "https://test@confluent.io:8080/nested/path?query&jobs" + }, + "timestamp" : 1 + }, { + "topic" : "test_topic", + "key" : "3", + "value" : { + "url" : "www.confluent.io" + }, + "timestamp" : 2 + } ], + "outputs" : [ { + "topic" : "OUTPUT", + "key" : "1", + "value" : { + "PROTOCOL" : "http" + }, + "timestamp" : 0 + }, { + "topic" : "OUTPUT", + "key" : "2", + "value" : { + "PROTOCOL" : "https" + }, + "timestamp" : 1 + }, { + "topic" : "OUTPUT", + "key" : "3", + "value" : { + "PROTOCOL" : "" + }, + "timestamp" : 2 + } ], + "topics" : [ { + "name" : "OUTPUT", + "replicas" : 1, + "numPartitions" : 4 + }, { + "name" : "test_topic", + "valueSchema" : "syntax = \"proto3\";\n\nmessage ConnectDefault1 {\n string URL = 1;\n}\n", + "valueFormat" : "PROTOBUF", + "replicas" : 1, + "numPartitions" : 4 + } ], + "statements" : [ "CREATE STREAM TEST (K STRING KEY, url VARCHAR) WITH (kafka_topic='test_topic', value_format='PROTOBUF');", "CREATE STREAM OUTPUT AS SELECT K, URL_EXTRACT_PROTOCOL(url) as PROTOCOL FROM TEST;" ], + "post" : { + "sources" : [ { + "name" : "OUTPUT", + "type" : "STREAM", + "schema" : "`K` STRING KEY, `PROTOCOL` STRING", + "keyFormat" : { + "format" : "KAFKA" + }, + "valueFormat" : "PROTOBUF", + "keyFeatures" : [ ], + "valueFeatures" : [ ], + "isSource" : false + }, { + "name" : "TEST", + "type" : "STREAM", + "schema" : "`K` STRING KEY, `URL` STRING", + "keyFormat" : { + "format" : "KAFKA" + }, + "valueFormat" : "PROTOBUF", + "keyFeatures" : [ ], + "valueFeatures" : [ ], + "isSource" : false + } ], + "topics" : { + "topics" : [ { + "name" : "test_topic", + "keyFormat" : { + "format" : "KAFKA" + }, + "valueFormat" : { + "format" : "PROTOBUF", + "properties" : { + "unwrapPrimitives" : "true" + } + }, + "partitions" : 4, + "valueSchema" : "syntax = \"proto3\";\n\nmessage ConnectDefault1 {\n string URL = 1;\n}\n" + }, { + "name" : "OUTPUT", + "keyFormat" : { + "format" : "KAFKA" + }, + "valueFormat" : { + "format" : "PROTOBUF", + "properties" : { + "unwrapPrimitives" : "true" + } + }, + "partitions" : 4, + "valueSchema" : "syntax = \"proto3\";\n\nmessage ConnectDefault1 {\n string PROTOCOL = 1;\n}\n" + } ] + } + } + } +} \ No newline at end of file diff --git a/ksqldb-functional-tests/src/test/resources/historical_plans/url_-_extract_a_protocol_from_a_URL_using_URL_EXTRACT_PROTOCOL_-_PROTOBUFs_-_PROTOBUF/7.3.0_1651071676835/topology b/ksqldb-functional-tests/src/test/resources/historical_plans/url_-_extract_a_protocol_from_a_URL_using_URL_EXTRACT_PROTOCOL_-_PROTOBUFs_-_PROTOBUF/7.3.0_1651071676835/topology new file mode 100644 index 000000000000..441a8f282644 --- /dev/null +++ b/ksqldb-functional-tests/src/test/resources/historical_plans/url_-_extract_a_protocol_from_a_URL_using_URL_EXTRACT_PROTOCOL_-_PROTOBUFs_-_PROTOBUF/7.3.0_1651071676835/topology @@ -0,0 +1,13 @@ +Topologies: + Sub-topology: 0 + Source: KSTREAM-SOURCE-0000000000 (topics: [test_topic]) + --> KSTREAM-TRANSFORMVALUES-0000000001 + Processor: KSTREAM-TRANSFORMVALUES-0000000001 (stores: []) + --> Project + <-- KSTREAM-SOURCE-0000000000 + Processor: Project (stores: []) + --> KSTREAM-SINK-0000000003 + <-- KSTREAM-TRANSFORMVALUES-0000000001 + Sink: KSTREAM-SINK-0000000003 (topic: OUTPUT) + <-- Project + diff --git a/ksqldb-functional-tests/src/test/resources/historical_plans/url_-_extract_a_protocol_from_a_URL_using_URL_EXTRACT_PROTOCOL_-_PROTOBUFs_-_PROTOBUF_NOSR/7.3.0_1651071676861/plan.json b/ksqldb-functional-tests/src/test/resources/historical_plans/url_-_extract_a_protocol_from_a_URL_using_URL_EXTRACT_PROTOCOL_-_PROTOBUFs_-_PROTOBUF_NOSR/7.3.0_1651071676861/plan.json new file mode 100644 index 000000000000..97c4ddf92d96 --- /dev/null +++ b/ksqldb-functional-tests/src/test/resources/historical_plans/url_-_extract_a_protocol_from_a_URL_using_URL_EXTRACT_PROTOCOL_-_PROTOBUFs_-_PROTOBUF_NOSR/7.3.0_1651071676861/plan.json @@ -0,0 +1,203 @@ +{ + "plan" : [ { + "@type" : "ksqlPlanV1", + "statementText" : "CREATE STREAM TEST (K STRING KEY, URL STRING) WITH (KAFKA_TOPIC='test_topic', KEY_FORMAT='KAFKA', VALUE_FORMAT='PROTOBUF_NOSR');", + "ddlCommand" : { + "@type" : "createStreamV1", + "sourceName" : "TEST", + "schema" : "`K` STRING KEY, `URL` STRING", + "timestampColumn" : null, + "topicName" : "test_topic", + "formats" : { + "keyFormat" : { + "format" : "KAFKA", + "properties" : { } + }, + "valueFormat" : { + "format" : "PROTOBUF_NOSR", + "properties" : { } + } + }, + "windowInfo" : null, + "orReplace" : false, + "isSource" : false + }, + "queryPlan" : null + }, { + "@type" : "ksqlPlanV1", + "statementText" : "CREATE STREAM OUTPUT AS SELECT\n TEST.K K,\n URL_EXTRACT_PROTOCOL(TEST.URL) PROTOCOL\nFROM TEST TEST\nEMIT CHANGES", + "ddlCommand" : { + "@type" : "createStreamV1", + "sourceName" : "OUTPUT", + "schema" : "`K` STRING KEY, `PROTOCOL` STRING", + "timestampColumn" : null, + "topicName" : "OUTPUT", + "formats" : { + "keyFormat" : { + "format" : "KAFKA", + "properties" : { } + }, + "valueFormat" : { + "format" : "PROTOBUF_NOSR", + "properties" : { } + } + }, + "windowInfo" : null, + "orReplace" : false, + "isSource" : false + }, + "queryPlan" : { + "sources" : [ "TEST" ], + "sink" : "OUTPUT", + "physicalPlan" : { + "@type" : "streamSinkV1", + "properties" : { + "queryContext" : "OUTPUT" + }, + "source" : { + "@type" : "streamSelectV1", + "properties" : { + "queryContext" : "Project" + }, + "source" : { + "@type" : "streamSourceV1", + "properties" : { + "queryContext" : "KsqlTopic/Source" + }, + "topicName" : "test_topic", + "formats" : { + "keyFormat" : { + "format" : "KAFKA", + "properties" : { } + }, + "valueFormat" : { + "format" : "PROTOBUF_NOSR", + "properties" : { } + } + }, + "timestampColumn" : null, + "sourceSchema" : "`K` STRING KEY, `URL` STRING", + "pseudoColumnVersion" : 1 + }, + "keyColumnNames" : [ "K" ], + "selectedKeys" : null, + "selectExpressions" : [ "URL_EXTRACT_PROTOCOL(URL) AS PROTOCOL" ] + }, + "formats" : { + "keyFormat" : { + "format" : "KAFKA", + "properties" : { } + }, + "valueFormat" : { + "format" : "PROTOBUF_NOSR", + "properties" : { } + } + }, + "topicName" : "OUTPUT", + "timestampColumn" : null + }, + "queryId" : "CSAS_OUTPUT_0", + "runtimeId" : null + } + } ], + "configs" : { + "ksql.extension.dir" : "ext", + "ksql.streams.cache.max.bytes.buffering" : "0", + "metric.reporters" : "", + "ksql.query.status.running.threshold.seconds" : "300", + "ksql.connect.basic.auth.credentials.reload" : "false", + "ksql.output.topic.name.prefix" : "", + "ksql.query.pull.stream.enabled" : "true", + "ksql.query.push.v2.interpreter.enabled" : "true", + "ksql.queryanonymizer.logs_enabled" : "true", + "ksql.variable.substitution.enable" : "true", + "ksql.streams.shutdown.timeout.ms" : "300000", + "ksql.query.pull.max.allowed.offset.lag" : "9223372036854775807", + "ksql.query.pull.max.qps" : "2147483647", + "ksql.access.validator.enable" : "auto", + "ksql.streams.bootstrap.servers" : "localhost:0", + "ksql.query.pull.metrics.enabled" : "true", + "ksql.metrics.extension" : null, + "ksql.query.push.v2.alos.enabled" : "true", + "ksql.query.push.v2.max.hourly.bandwidth.megabytes" : "2147483647", + "ksql.query.pull.range.scan.enabled" : "true", + "ksql.transient.query.cleanup.service.initial.delay.seconds" : "600", + "ksql.hidden.topics" : "_confluent.*,__confluent.*,_schemas,__consumer_offsets,__transaction_state,connect-configs,connect-offsets,connect-status,connect-statuses", + "ksql.lambdas.enabled" : "true", + "ksql.source.table.materialization.enabled" : "true", + "ksql.query.pull.max.hourly.bandwidth.megabytes" : "2147483647", + "ksql.sink.window.change.log.additional.retention" : "1000000", + "ksql.query.persistent.active.limit" : "2147483647", + "ksql.persistence.wrap.single.values" : null, + "ksql.query.transient.max.bytes.buffering.total" : "-1", + "ksql.connect.basic.auth.credentials.source" : "NONE", + "ksql.schema.registry.url" : "schema_registry.url:0", + "ksql.properties.overrides.denylist" : "", + "ksql.service.id" : "some.ksql.service.id", + "ksql.query.push.v2.max.catchup.consumers" : "5", + "ksql.query.push.v2.enabled" : "false", + "ksql.transient.query.cleanup.service.enable" : "true", + "ksql.query.push.v2.metrics.enabled" : "true", + "ksql.rowpartition.rowoffset.enabled" : "true", + "ksql.streams.commit.interval.ms" : "2000", + "ksql.query.pull.table.scan.enabled" : "true", + "ksql.streams.auto.commit.interval.ms" : "0", + "ksql.streams.topology.optimization" : "all", + "ksql.endpoint.migrate.query" : "true", + "ksql.query.push.v2.registry.installed" : "false", + "ksql.streams.num.stream.threads" : "4", + "ksql.metrics.tags.custom" : "", + "ksql.query.push.v2.catchup.consumer.msg.window" : "50", + "ksql.runtime.feature.shared.enabled" : "false", + "ksql.udf.collect.metrics" : "false", + "ksql.new.query.planner.enabled" : "false", + "ksql.connect.request.headers.plugin" : null, + "ksql.security.extension.class" : null, + "ksql.transient.prefix" : "transient_", + "ksql.headers.columns.enabled" : "true", + "ksql.streams.default.deserialization.exception.handler" : "io.confluent.ksql.errors.LogMetricAndContinueExceptionHandler", + "ksql.connect.request.timeout.ms" : "5000", + "ksql.query.pull.enable.standby.reads" : "false", + "ksql.persistence.default.format.key" : "KAFKA", + "ksql.query.persistent.max.bytes.buffering.total" : "-1", + "ksql.query.error.max.queue.size" : "10", + "ksql.query.cleanup.shutdown.timeout.ms" : "30000", + "ksql.internal.topic.min.insync.replicas" : "1", + "ksql.internal.topic.replicas" : "1", + "ksql.insert.into.values.enabled" : "true", + "ksql.queryanonymizer.cluster_namespace" : null, + "ksql.create.or.replace.enabled" : "true", + "ksql.shared.runtimes.count" : "2", + "ksql.cast.strings.preserve.nulls" : "true", + "ksql.authorization.cache.max.entries" : "10000", + "ksql.pull.queries.enable" : "true", + "ksql.transient.query.cleanup.service.period.seconds" : "600", + "ksql.suppress.enabled" : "false", + "ksql.readonly.topics" : "_confluent.*,__confluent.*,_schemas,__consumer_offsets,__transaction_state,connect-configs,connect-offsets,connect-status,connect-statuses", + "ksql.connect.basic.auth.credentials.file" : "", + "ksql.authorization.cache.expiry.time.secs" : "30", + "ksql.query.retry.backoff.initial.ms" : "15000", + "ksql.query.pull.max.concurrent.requests" : "2147483647", + "ksql.streams.auto.offset.reset" : "earliest", + "ksql.connect.url" : "http://localhost:8083", + "ksql.query.push.v2.new.latest.delay.ms" : "5000", + "ksql.query.push.v2.latest.reset.age.ms" : "30000", + "ksql.streams.default.production.exception.handler" : "io.confluent.ksql.errors.ProductionExceptionHandlerUtil$LogAndFailProductionExceptionHandler", + "ksql.query.pull.interpreter.enabled" : "true", + "ksql.query.pull.limit.clause.enabled" : "true", + "ksql.query.pull.router.thread.pool.size" : "50", + "ksql.query.push.v2.continuation.tokens.enabled" : "false", + "ksql.query.retry.backoff.max.ms" : "900000", + "ksql.timestamp.throw.on.invalid" : "false", + "ksql.persistence.default.format.value" : null, + "ksql.udfs.enabled" : "true", + "ksql.udf.enable.security.manager" : "true", + "ksql.connect.worker.config" : "", + "ksql.nested.error.set.null" : "true", + "ksql.query.pull.thread.pool.size" : "50", + "ksql.persistent.prefix" : "query_", + "ksql.metastore.backup.location" : "", + "ksql.error.classifier.regex" : "", + "ksql.suppress.buffer.size.bytes" : "-1" + } +} \ No newline at end of file diff --git a/ksqldb-functional-tests/src/test/resources/historical_plans/url_-_extract_a_protocol_from_a_URL_using_URL_EXTRACT_PROTOCOL_-_PROTOBUFs_-_PROTOBUF_NOSR/7.3.0_1651071676861/spec.json b/ksqldb-functional-tests/src/test/resources/historical_plans/url_-_extract_a_protocol_from_a_URL_using_URL_EXTRACT_PROTOCOL_-_PROTOBUFs_-_PROTOBUF_NOSR/7.3.0_1651071676861/spec.json new file mode 100644 index 000000000000..b8d1a91a2195 --- /dev/null +++ b/ksqldb-functional-tests/src/test/resources/historical_plans/url_-_extract_a_protocol_from_a_URL_using_URL_EXTRACT_PROTOCOL_-_PROTOBUFs_-_PROTOBUF_NOSR/7.3.0_1651071676861/spec.json @@ -0,0 +1,128 @@ +{ + "version" : "7.3.0", + "timestamp" : 1651071676861, + "path" : "query-validation-tests/url.json", + "schemas" : { + "CSAS_OUTPUT_0.KsqlTopic.Source" : { + "schema" : "`K` STRING KEY, `URL` STRING", + "keyFormat" : { + "format" : "KAFKA" + }, + "valueFormat" : { + "format" : "PROTOBUF_NOSR" + } + }, + "CSAS_OUTPUT_0.OUTPUT" : { + "schema" : "`K` STRING KEY, `PROTOCOL` STRING", + "keyFormat" : { + "format" : "KAFKA" + }, + "valueFormat" : { + "format" : "PROTOBUF_NOSR" + } + } + }, + "testCase" : { + "name" : "extract a protocol from a URL using URL_EXTRACT_PROTOCOL - PROTOBUFs - PROTOBUF_NOSR", + "inputs" : [ { + "topic" : "test_topic", + "key" : "1", + "value" : { + "url" : "http://test@confluent.io" + }, + "timestamp" : 0 + }, { + "topic" : "test_topic", + "key" : "2", + "value" : { + "url" : "https://test@confluent.io:8080/nested/path?query&jobs" + }, + "timestamp" : 1 + }, { + "topic" : "test_topic", + "key" : "3", + "value" : { + "url" : "www.confluent.io" + }, + "timestamp" : 2 + } ], + "outputs" : [ { + "topic" : "OUTPUT", + "key" : "1", + "value" : { + "PROTOCOL" : "http" + }, + "timestamp" : 0 + }, { + "topic" : "OUTPUT", + "key" : "2", + "value" : { + "PROTOCOL" : "https" + }, + "timestamp" : 1 + }, { + "topic" : "OUTPUT", + "key" : "3", + "value" : { + "PROTOCOL" : "" + }, + "timestamp" : 2 + } ], + "topics" : [ { + "name" : "OUTPUT", + "replicas" : 1, + "numPartitions" : 4 + }, { + "name" : "test_topic", + "replicas" : 1, + "numPartitions" : 4 + } ], + "statements" : [ "CREATE STREAM TEST (K STRING KEY, url VARCHAR) WITH (kafka_topic='test_topic', value_format='PROTOBUF_NOSR');", "CREATE STREAM OUTPUT AS SELECT K, URL_EXTRACT_PROTOCOL(url) as PROTOCOL FROM TEST;" ], + "post" : { + "sources" : [ { + "name" : "OUTPUT", + "type" : "STREAM", + "schema" : "`K` STRING KEY, `PROTOCOL` STRING", + "keyFormat" : { + "format" : "KAFKA" + }, + "valueFormat" : "PROTOBUF_NOSR", + "keyFeatures" : [ ], + "valueFeatures" : [ ], + "isSource" : false + }, { + "name" : "TEST", + "type" : "STREAM", + "schema" : "`K` STRING KEY, `URL` STRING", + "keyFormat" : { + "format" : "KAFKA" + }, + "valueFormat" : "PROTOBUF_NOSR", + "keyFeatures" : [ ], + "valueFeatures" : [ ], + "isSource" : false + } ], + "topics" : { + "topics" : [ { + "name" : "test_topic", + "keyFormat" : { + "format" : "KAFKA" + }, + "valueFormat" : { + "format" : "PROTOBUF_NOSR" + }, + "partitions" : 4 + }, { + "name" : "OUTPUT", + "keyFormat" : { + "format" : "KAFKA" + }, + "valueFormat" : { + "format" : "PROTOBUF_NOSR" + }, + "partitions" : 4 + } ] + } + } + } +} \ No newline at end of file diff --git a/ksqldb-functional-tests/src/test/resources/historical_plans/url_-_extract_a_protocol_from_a_URL_using_URL_EXTRACT_PROTOCOL_-_PROTOBUFs_-_PROTOBUF_NOSR/7.3.0_1651071676861/topology b/ksqldb-functional-tests/src/test/resources/historical_plans/url_-_extract_a_protocol_from_a_URL_using_URL_EXTRACT_PROTOCOL_-_PROTOBUFs_-_PROTOBUF_NOSR/7.3.0_1651071676861/topology new file mode 100644 index 000000000000..441a8f282644 --- /dev/null +++ b/ksqldb-functional-tests/src/test/resources/historical_plans/url_-_extract_a_protocol_from_a_URL_using_URL_EXTRACT_PROTOCOL_-_PROTOBUFs_-_PROTOBUF_NOSR/7.3.0_1651071676861/topology @@ -0,0 +1,13 @@ +Topologies: + Sub-topology: 0 + Source: KSTREAM-SOURCE-0000000000 (topics: [test_topic]) + --> KSTREAM-TRANSFORMVALUES-0000000001 + Processor: KSTREAM-TRANSFORMVALUES-0000000001 (stores: []) + --> Project + <-- KSTREAM-SOURCE-0000000000 + Processor: Project (stores: []) + --> KSTREAM-SINK-0000000003 + <-- KSTREAM-TRANSFORMVALUES-0000000001 + Sink: KSTREAM-SINK-0000000003 (topic: OUTPUT) + <-- Project + diff --git a/ksqldb-functional-tests/src/test/resources/historical_plans/url_-_extract_a_query_from_a_URL_using_URL_EXTRACT_QUERY_-_AVRO/7.3.0_1651071676915/plan.json b/ksqldb-functional-tests/src/test/resources/historical_plans/url_-_extract_a_query_from_a_URL_using_URL_EXTRACT_QUERY_-_AVRO/7.3.0_1651071676915/plan.json new file mode 100644 index 000000000000..a552f81572cd --- /dev/null +++ b/ksqldb-functional-tests/src/test/resources/historical_plans/url_-_extract_a_query_from_a_URL_using_URL_EXTRACT_QUERY_-_AVRO/7.3.0_1651071676915/plan.json @@ -0,0 +1,203 @@ +{ + "plan" : [ { + "@type" : "ksqlPlanV1", + "statementText" : "CREATE STREAM TEST (K STRING KEY, URL STRING) WITH (KAFKA_TOPIC='test_topic', KEY_FORMAT='KAFKA', VALUE_FORMAT='AVRO');", + "ddlCommand" : { + "@type" : "createStreamV1", + "sourceName" : "TEST", + "schema" : "`K` STRING KEY, `URL` STRING", + "timestampColumn" : null, + "topicName" : "test_topic", + "formats" : { + "keyFormat" : { + "format" : "KAFKA", + "properties" : { } + }, + "valueFormat" : { + "format" : "AVRO", + "properties" : { } + } + }, + "windowInfo" : null, + "orReplace" : false, + "isSource" : false + }, + "queryPlan" : null + }, { + "@type" : "ksqlPlanV1", + "statementText" : "CREATE STREAM OUTPUT AS SELECT\n TEST.K K,\n URL_EXTRACT_QUERY(TEST.URL) Q\nFROM TEST TEST\nEMIT CHANGES", + "ddlCommand" : { + "@type" : "createStreamV1", + "sourceName" : "OUTPUT", + "schema" : "`K` STRING KEY, `Q` STRING", + "timestampColumn" : null, + "topicName" : "OUTPUT", + "formats" : { + "keyFormat" : { + "format" : "KAFKA", + "properties" : { } + }, + "valueFormat" : { + "format" : "AVRO", + "properties" : { } + } + }, + "windowInfo" : null, + "orReplace" : false, + "isSource" : false + }, + "queryPlan" : { + "sources" : [ "TEST" ], + "sink" : "OUTPUT", + "physicalPlan" : { + "@type" : "streamSinkV1", + "properties" : { + "queryContext" : "OUTPUT" + }, + "source" : { + "@type" : "streamSelectV1", + "properties" : { + "queryContext" : "Project" + }, + "source" : { + "@type" : "streamSourceV1", + "properties" : { + "queryContext" : "KsqlTopic/Source" + }, + "topicName" : "test_topic", + "formats" : { + "keyFormat" : { + "format" : "KAFKA", + "properties" : { } + }, + "valueFormat" : { + "format" : "AVRO", + "properties" : { } + } + }, + "timestampColumn" : null, + "sourceSchema" : "`K` STRING KEY, `URL` STRING", + "pseudoColumnVersion" : 1 + }, + "keyColumnNames" : [ "K" ], + "selectedKeys" : null, + "selectExpressions" : [ "URL_EXTRACT_QUERY(URL) AS Q" ] + }, + "formats" : { + "keyFormat" : { + "format" : "KAFKA", + "properties" : { } + }, + "valueFormat" : { + "format" : "AVRO", + "properties" : { } + } + }, + "topicName" : "OUTPUT", + "timestampColumn" : null + }, + "queryId" : "CSAS_OUTPUT_0", + "runtimeId" : null + } + } ], + "configs" : { + "ksql.extension.dir" : "ext", + "ksql.streams.cache.max.bytes.buffering" : "0", + "metric.reporters" : "", + "ksql.query.status.running.threshold.seconds" : "300", + "ksql.connect.basic.auth.credentials.reload" : "false", + "ksql.output.topic.name.prefix" : "", + "ksql.query.pull.stream.enabled" : "true", + "ksql.query.push.v2.interpreter.enabled" : "true", + "ksql.queryanonymizer.logs_enabled" : "true", + "ksql.variable.substitution.enable" : "true", + "ksql.streams.shutdown.timeout.ms" : "300000", + "ksql.query.pull.max.allowed.offset.lag" : "9223372036854775807", + "ksql.query.pull.max.qps" : "2147483647", + "ksql.access.validator.enable" : "auto", + "ksql.streams.bootstrap.servers" : "localhost:0", + "ksql.query.pull.metrics.enabled" : "true", + "ksql.metrics.extension" : null, + "ksql.query.push.v2.alos.enabled" : "true", + "ksql.query.push.v2.max.hourly.bandwidth.megabytes" : "2147483647", + "ksql.query.pull.range.scan.enabled" : "true", + "ksql.transient.query.cleanup.service.initial.delay.seconds" : "600", + "ksql.hidden.topics" : "_confluent.*,__confluent.*,_schemas,__consumer_offsets,__transaction_state,connect-configs,connect-offsets,connect-status,connect-statuses", + "ksql.lambdas.enabled" : "true", + "ksql.source.table.materialization.enabled" : "true", + "ksql.query.pull.max.hourly.bandwidth.megabytes" : "2147483647", + "ksql.sink.window.change.log.additional.retention" : "1000000", + "ksql.query.persistent.active.limit" : "2147483647", + "ksql.persistence.wrap.single.values" : null, + "ksql.query.transient.max.bytes.buffering.total" : "-1", + "ksql.connect.basic.auth.credentials.source" : "NONE", + "ksql.schema.registry.url" : "schema_registry.url:0", + "ksql.properties.overrides.denylist" : "", + "ksql.service.id" : "some.ksql.service.id", + "ksql.query.push.v2.max.catchup.consumers" : "5", + "ksql.query.push.v2.enabled" : "false", + "ksql.transient.query.cleanup.service.enable" : "true", + "ksql.query.push.v2.metrics.enabled" : "true", + "ksql.rowpartition.rowoffset.enabled" : "true", + "ksql.streams.commit.interval.ms" : "2000", + "ksql.query.pull.table.scan.enabled" : "true", + "ksql.streams.auto.commit.interval.ms" : "0", + "ksql.streams.topology.optimization" : "all", + "ksql.endpoint.migrate.query" : "true", + "ksql.query.push.v2.registry.installed" : "false", + "ksql.streams.num.stream.threads" : "4", + "ksql.metrics.tags.custom" : "", + "ksql.query.push.v2.catchup.consumer.msg.window" : "50", + "ksql.runtime.feature.shared.enabled" : "false", + "ksql.udf.collect.metrics" : "false", + "ksql.new.query.planner.enabled" : "false", + "ksql.connect.request.headers.plugin" : null, + "ksql.security.extension.class" : null, + "ksql.transient.prefix" : "transient_", + "ksql.headers.columns.enabled" : "true", + "ksql.streams.default.deserialization.exception.handler" : "io.confluent.ksql.errors.LogMetricAndContinueExceptionHandler", + "ksql.connect.request.timeout.ms" : "5000", + "ksql.query.pull.enable.standby.reads" : "false", + "ksql.persistence.default.format.key" : "KAFKA", + "ksql.query.persistent.max.bytes.buffering.total" : "-1", + "ksql.query.error.max.queue.size" : "10", + "ksql.query.cleanup.shutdown.timeout.ms" : "30000", + "ksql.internal.topic.min.insync.replicas" : "1", + "ksql.internal.topic.replicas" : "1", + "ksql.insert.into.values.enabled" : "true", + "ksql.queryanonymizer.cluster_namespace" : null, + "ksql.create.or.replace.enabled" : "true", + "ksql.shared.runtimes.count" : "2", + "ksql.cast.strings.preserve.nulls" : "true", + "ksql.authorization.cache.max.entries" : "10000", + "ksql.pull.queries.enable" : "true", + "ksql.transient.query.cleanup.service.period.seconds" : "600", + "ksql.suppress.enabled" : "false", + "ksql.readonly.topics" : "_confluent.*,__confluent.*,_schemas,__consumer_offsets,__transaction_state,connect-configs,connect-offsets,connect-status,connect-statuses", + "ksql.connect.basic.auth.credentials.file" : "", + "ksql.authorization.cache.expiry.time.secs" : "30", + "ksql.query.retry.backoff.initial.ms" : "15000", + "ksql.query.pull.max.concurrent.requests" : "2147483647", + "ksql.streams.auto.offset.reset" : "earliest", + "ksql.connect.url" : "http://localhost:8083", + "ksql.query.push.v2.new.latest.delay.ms" : "5000", + "ksql.query.push.v2.latest.reset.age.ms" : "30000", + "ksql.streams.default.production.exception.handler" : "io.confluent.ksql.errors.ProductionExceptionHandlerUtil$LogAndFailProductionExceptionHandler", + "ksql.query.pull.interpreter.enabled" : "true", + "ksql.query.pull.limit.clause.enabled" : "true", + "ksql.query.pull.router.thread.pool.size" : "50", + "ksql.query.push.v2.continuation.tokens.enabled" : "false", + "ksql.query.retry.backoff.max.ms" : "900000", + "ksql.timestamp.throw.on.invalid" : "false", + "ksql.persistence.default.format.value" : null, + "ksql.udfs.enabled" : "true", + "ksql.udf.enable.security.manager" : "true", + "ksql.connect.worker.config" : "", + "ksql.nested.error.set.null" : "true", + "ksql.query.pull.thread.pool.size" : "50", + "ksql.persistent.prefix" : "query_", + "ksql.metastore.backup.location" : "", + "ksql.error.classifier.regex" : "", + "ksql.suppress.buffer.size.bytes" : "-1" + } +} \ No newline at end of file diff --git a/ksqldb-functional-tests/src/test/resources/historical_plans/url_-_extract_a_query_from_a_URL_using_URL_EXTRACT_QUERY_-_AVRO/7.3.0_1651071676915/spec.json b/ksqldb-functional-tests/src/test/resources/historical_plans/url_-_extract_a_query_from_a_URL_using_URL_EXTRACT_QUERY_-_AVRO/7.3.0_1651071676915/spec.json new file mode 100644 index 000000000000..c7e6b352508a --- /dev/null +++ b/ksqldb-functional-tests/src/test/resources/historical_plans/url_-_extract_a_query_from_a_URL_using_URL_EXTRACT_QUERY_-_AVRO/7.3.0_1651071676915/spec.json @@ -0,0 +1,175 @@ +{ + "version" : "7.3.0", + "timestamp" : 1651071676915, + "path" : "query-validation-tests/url.json", + "schemas" : { + "CSAS_OUTPUT_0.KsqlTopic.Source" : { + "schema" : "`K` STRING KEY, `URL` STRING", + "keyFormat" : { + "format" : "KAFKA" + }, + "valueFormat" : { + "format" : "AVRO" + } + }, + "CSAS_OUTPUT_0.OUTPUT" : { + "schema" : "`K` STRING KEY, `Q` STRING", + "keyFormat" : { + "format" : "KAFKA" + }, + "valueFormat" : { + "format" : "AVRO" + } + } + }, + "testCase" : { + "name" : "extract a query from a URL using URL_EXTRACT_QUERY - AVRO", + "inputs" : [ { + "topic" : "test_topic", + "key" : "1", + "value" : { + "url" : "http://www.test.com?query#fragment" + }, + "timestamp" : 0 + }, { + "topic" : "test_topic", + "key" : "2", + "value" : { + "url" : "http://www.test.com/path?q1&q2" + }, + "timestamp" : 1 + }, { + "topic" : "test_topic", + "key" : "3", + "value" : { + "url" : "http://test@confluent.io:8080/nested/path?q=2" + }, + "timestamp" : 2 + }, { + "topic" : "test_topic", + "key" : "4", + "value" : { + "url" : "http://test@confluent.io:8080/path" + }, + "timestamp" : 3 + } ], + "outputs" : [ { + "topic" : "OUTPUT", + "key" : "1", + "value" : { + "Q" : "query" + }, + "timestamp" : 0 + }, { + "topic" : "OUTPUT", + "key" : "2", + "value" : { + "Q" : "q1&q2" + }, + "timestamp" : 1 + }, { + "topic" : "OUTPUT", + "key" : "3", + "value" : { + "Q" : "q=2" + }, + "timestamp" : 2 + }, { + "topic" : "OUTPUT", + "key" : "4", + "value" : { + "Q" : null + }, + "timestamp" : 3 + } ], + "topics" : [ { + "name" : "OUTPUT", + "replicas" : 1, + "numPartitions" : 4 + }, { + "name" : "test_topic", + "valueSchema" : { + "type" : "record", + "name" : "KsqlDataSourceSchema", + "namespace" : "io.confluent.ksql.avro_schemas", + "fields" : [ { + "name" : "URL", + "type" : [ "null", "string" ], + "default" : null + } ], + "connect.name" : "io.confluent.ksql.avro_schemas.KsqlDataSourceSchema" + }, + "valueFormat" : "AVRO", + "replicas" : 1, + "numPartitions" : 4 + } ], + "statements" : [ "CREATE STREAM TEST (K STRING KEY, url VARCHAR) WITH (kafka_topic='test_topic', value_format='AVRO');", "CREATE STREAM OUTPUT AS SELECT K, URL_EXTRACT_QUERY(url) as Q FROM TEST;" ], + "post" : { + "sources" : [ { + "name" : "OUTPUT", + "type" : "STREAM", + "schema" : "`K` STRING KEY, `Q` STRING", + "keyFormat" : { + "format" : "KAFKA" + }, + "valueFormat" : "AVRO", + "keyFeatures" : [ ], + "valueFeatures" : [ ], + "isSource" : false + }, { + "name" : "TEST", + "type" : "STREAM", + "schema" : "`K` STRING KEY, `URL` STRING", + "keyFormat" : { + "format" : "KAFKA" + }, + "valueFormat" : "AVRO", + "keyFeatures" : [ ], + "valueFeatures" : [ ], + "isSource" : false + } ], + "topics" : { + "topics" : [ { + "name" : "test_topic", + "keyFormat" : { + "format" : "KAFKA" + }, + "valueFormat" : { + "format" : "AVRO" + }, + "partitions" : 4, + "valueSchema" : { + "type" : "record", + "name" : "KsqlDataSourceSchema", + "namespace" : "io.confluent.ksql.avro_schemas", + "fields" : [ { + "name" : "URL", + "type" : [ "null", "string" ], + "default" : null + } ], + "connect.name" : "io.confluent.ksql.avro_schemas.KsqlDataSourceSchema" + } + }, { + "name" : "OUTPUT", + "keyFormat" : { + "format" : "KAFKA" + }, + "valueFormat" : { + "format" : "AVRO" + }, + "partitions" : 4, + "valueSchema" : { + "type" : "record", + "name" : "KsqlDataSourceSchema", + "namespace" : "io.confluent.ksql.avro_schemas", + "fields" : [ { + "name" : "Q", + "type" : [ "null", "string" ], + "default" : null + } ] + } + } ] + } + } + } +} \ No newline at end of file diff --git a/ksqldb-functional-tests/src/test/resources/historical_plans/url_-_extract_a_query_from_a_URL_using_URL_EXTRACT_QUERY_-_AVRO/7.3.0_1651071676915/topology b/ksqldb-functional-tests/src/test/resources/historical_plans/url_-_extract_a_query_from_a_URL_using_URL_EXTRACT_QUERY_-_AVRO/7.3.0_1651071676915/topology new file mode 100644 index 000000000000..441a8f282644 --- /dev/null +++ b/ksqldb-functional-tests/src/test/resources/historical_plans/url_-_extract_a_query_from_a_URL_using_URL_EXTRACT_QUERY_-_AVRO/7.3.0_1651071676915/topology @@ -0,0 +1,13 @@ +Topologies: + Sub-topology: 0 + Source: KSTREAM-SOURCE-0000000000 (topics: [test_topic]) + --> KSTREAM-TRANSFORMVALUES-0000000001 + Processor: KSTREAM-TRANSFORMVALUES-0000000001 (stores: []) + --> Project + <-- KSTREAM-SOURCE-0000000000 + Processor: Project (stores: []) + --> KSTREAM-SINK-0000000003 + <-- KSTREAM-TRANSFORMVALUES-0000000001 + Sink: KSTREAM-SINK-0000000003 (topic: OUTPUT) + <-- Project + diff --git a/ksqldb-functional-tests/src/test/resources/historical_plans/url_-_extract_a_query_from_a_URL_using_URL_EXTRACT_QUERY_-_PROTOBUFs_-_PROTOBUF/7.3.0_1651071676942/plan.json b/ksqldb-functional-tests/src/test/resources/historical_plans/url_-_extract_a_query_from_a_URL_using_URL_EXTRACT_QUERY_-_PROTOBUFs_-_PROTOBUF/7.3.0_1651071676942/plan.json new file mode 100644 index 000000000000..9eae6a31186a --- /dev/null +++ b/ksqldb-functional-tests/src/test/resources/historical_plans/url_-_extract_a_query_from_a_URL_using_URL_EXTRACT_QUERY_-_PROTOBUFs_-_PROTOBUF/7.3.0_1651071676942/plan.json @@ -0,0 +1,211 @@ +{ + "plan" : [ { + "@type" : "ksqlPlanV1", + "statementText" : "CREATE STREAM TEST (K STRING KEY, URL STRING) WITH (KAFKA_TOPIC='test_topic', KEY_FORMAT='KAFKA', VALUE_FORMAT='PROTOBUF');", + "ddlCommand" : { + "@type" : "createStreamV1", + "sourceName" : "TEST", + "schema" : "`K` STRING KEY, `URL` STRING", + "timestampColumn" : null, + "topicName" : "test_topic", + "formats" : { + "keyFormat" : { + "format" : "KAFKA", + "properties" : { } + }, + "valueFormat" : { + "format" : "PROTOBUF", + "properties" : { + "unwrapPrimitives" : "true" + } + } + }, + "windowInfo" : null, + "orReplace" : false, + "isSource" : false + }, + "queryPlan" : null + }, { + "@type" : "ksqlPlanV1", + "statementText" : "CREATE STREAM OUTPUT AS SELECT\n TEST.K K,\n URL_EXTRACT_QUERY(TEST.URL) Q\nFROM TEST TEST\nEMIT CHANGES", + "ddlCommand" : { + "@type" : "createStreamV1", + "sourceName" : "OUTPUT", + "schema" : "`K` STRING KEY, `Q` STRING", + "timestampColumn" : null, + "topicName" : "OUTPUT", + "formats" : { + "keyFormat" : { + "format" : "KAFKA", + "properties" : { } + }, + "valueFormat" : { + "format" : "PROTOBUF", + "properties" : { + "unwrapPrimitives" : "true" + } + } + }, + "windowInfo" : null, + "orReplace" : false, + "isSource" : false + }, + "queryPlan" : { + "sources" : [ "TEST" ], + "sink" : "OUTPUT", + "physicalPlan" : { + "@type" : "streamSinkV1", + "properties" : { + "queryContext" : "OUTPUT" + }, + "source" : { + "@type" : "streamSelectV1", + "properties" : { + "queryContext" : "Project" + }, + "source" : { + "@type" : "streamSourceV1", + "properties" : { + "queryContext" : "KsqlTopic/Source" + }, + "topicName" : "test_topic", + "formats" : { + "keyFormat" : { + "format" : "KAFKA", + "properties" : { } + }, + "valueFormat" : { + "format" : "PROTOBUF", + "properties" : { + "unwrapPrimitives" : "true" + } + } + }, + "timestampColumn" : null, + "sourceSchema" : "`K` STRING KEY, `URL` STRING", + "pseudoColumnVersion" : 1 + }, + "keyColumnNames" : [ "K" ], + "selectedKeys" : null, + "selectExpressions" : [ "URL_EXTRACT_QUERY(URL) AS Q" ] + }, + "formats" : { + "keyFormat" : { + "format" : "KAFKA", + "properties" : { } + }, + "valueFormat" : { + "format" : "PROTOBUF", + "properties" : { + "unwrapPrimitives" : "true" + } + } + }, + "topicName" : "OUTPUT", + "timestampColumn" : null + }, + "queryId" : "CSAS_OUTPUT_0", + "runtimeId" : null + } + } ], + "configs" : { + "ksql.extension.dir" : "ext", + "ksql.streams.cache.max.bytes.buffering" : "0", + "metric.reporters" : "", + "ksql.query.status.running.threshold.seconds" : "300", + "ksql.connect.basic.auth.credentials.reload" : "false", + "ksql.output.topic.name.prefix" : "", + "ksql.query.pull.stream.enabled" : "true", + "ksql.query.push.v2.interpreter.enabled" : "true", + "ksql.queryanonymizer.logs_enabled" : "true", + "ksql.variable.substitution.enable" : "true", + "ksql.streams.shutdown.timeout.ms" : "300000", + "ksql.query.pull.max.allowed.offset.lag" : "9223372036854775807", + "ksql.query.pull.max.qps" : "2147483647", + "ksql.access.validator.enable" : "auto", + "ksql.streams.bootstrap.servers" : "localhost:0", + "ksql.query.pull.metrics.enabled" : "true", + "ksql.metrics.extension" : null, + "ksql.query.push.v2.alos.enabled" : "true", + "ksql.query.push.v2.max.hourly.bandwidth.megabytes" : "2147483647", + "ksql.query.pull.range.scan.enabled" : "true", + "ksql.transient.query.cleanup.service.initial.delay.seconds" : "600", + "ksql.hidden.topics" : "_confluent.*,__confluent.*,_schemas,__consumer_offsets,__transaction_state,connect-configs,connect-offsets,connect-status,connect-statuses", + "ksql.lambdas.enabled" : "true", + "ksql.source.table.materialization.enabled" : "true", + "ksql.query.pull.max.hourly.bandwidth.megabytes" : "2147483647", + "ksql.sink.window.change.log.additional.retention" : "1000000", + "ksql.query.persistent.active.limit" : "2147483647", + "ksql.persistence.wrap.single.values" : null, + "ksql.query.transient.max.bytes.buffering.total" : "-1", + "ksql.connect.basic.auth.credentials.source" : "NONE", + "ksql.schema.registry.url" : "schema_registry.url:0", + "ksql.properties.overrides.denylist" : "", + "ksql.service.id" : "some.ksql.service.id", + "ksql.query.push.v2.max.catchup.consumers" : "5", + "ksql.query.push.v2.enabled" : "false", + "ksql.transient.query.cleanup.service.enable" : "true", + "ksql.query.push.v2.metrics.enabled" : "true", + "ksql.rowpartition.rowoffset.enabled" : "true", + "ksql.streams.commit.interval.ms" : "2000", + "ksql.query.pull.table.scan.enabled" : "true", + "ksql.streams.auto.commit.interval.ms" : "0", + "ksql.streams.topology.optimization" : "all", + "ksql.endpoint.migrate.query" : "true", + "ksql.query.push.v2.registry.installed" : "false", + "ksql.streams.num.stream.threads" : "4", + "ksql.metrics.tags.custom" : "", + "ksql.query.push.v2.catchup.consumer.msg.window" : "50", + "ksql.runtime.feature.shared.enabled" : "false", + "ksql.udf.collect.metrics" : "false", + "ksql.new.query.planner.enabled" : "false", + "ksql.connect.request.headers.plugin" : null, + "ksql.security.extension.class" : null, + "ksql.transient.prefix" : "transient_", + "ksql.headers.columns.enabled" : "true", + "ksql.streams.default.deserialization.exception.handler" : "io.confluent.ksql.errors.LogMetricAndContinueExceptionHandler", + "ksql.connect.request.timeout.ms" : "5000", + "ksql.query.pull.enable.standby.reads" : "false", + "ksql.persistence.default.format.key" : "KAFKA", + "ksql.query.persistent.max.bytes.buffering.total" : "-1", + "ksql.query.error.max.queue.size" : "10", + "ksql.query.cleanup.shutdown.timeout.ms" : "30000", + "ksql.internal.topic.min.insync.replicas" : "1", + "ksql.internal.topic.replicas" : "1", + "ksql.insert.into.values.enabled" : "true", + "ksql.queryanonymizer.cluster_namespace" : null, + "ksql.create.or.replace.enabled" : "true", + "ksql.shared.runtimes.count" : "2", + "ksql.cast.strings.preserve.nulls" : "true", + "ksql.authorization.cache.max.entries" : "10000", + "ksql.pull.queries.enable" : "true", + "ksql.transient.query.cleanup.service.period.seconds" : "600", + "ksql.suppress.enabled" : "false", + "ksql.readonly.topics" : "_confluent.*,__confluent.*,_schemas,__consumer_offsets,__transaction_state,connect-configs,connect-offsets,connect-status,connect-statuses", + "ksql.connect.basic.auth.credentials.file" : "", + "ksql.authorization.cache.expiry.time.secs" : "30", + "ksql.query.retry.backoff.initial.ms" : "15000", + "ksql.query.pull.max.concurrent.requests" : "2147483647", + "ksql.streams.auto.offset.reset" : "earliest", + "ksql.connect.url" : "http://localhost:8083", + "ksql.query.push.v2.new.latest.delay.ms" : "5000", + "ksql.query.push.v2.latest.reset.age.ms" : "30000", + "ksql.streams.default.production.exception.handler" : "io.confluent.ksql.errors.ProductionExceptionHandlerUtil$LogAndFailProductionExceptionHandler", + "ksql.query.pull.interpreter.enabled" : "true", + "ksql.query.pull.limit.clause.enabled" : "true", + "ksql.query.pull.router.thread.pool.size" : "50", + "ksql.query.push.v2.continuation.tokens.enabled" : "false", + "ksql.query.retry.backoff.max.ms" : "900000", + "ksql.timestamp.throw.on.invalid" : "false", + "ksql.persistence.default.format.value" : null, + "ksql.udfs.enabled" : "true", + "ksql.udf.enable.security.manager" : "true", + "ksql.connect.worker.config" : "", + "ksql.nested.error.set.null" : "true", + "ksql.query.pull.thread.pool.size" : "50", + "ksql.persistent.prefix" : "query_", + "ksql.metastore.backup.location" : "", + "ksql.error.classifier.regex" : "", + "ksql.suppress.buffer.size.bytes" : "-1" + } +} \ No newline at end of file diff --git a/ksqldb-functional-tests/src/test/resources/historical_plans/url_-_extract_a_query_from_a_URL_using_URL_EXTRACT_QUERY_-_PROTOBUFs_-_PROTOBUF/7.3.0_1651071676942/spec.json b/ksqldb-functional-tests/src/test/resources/historical_plans/url_-_extract_a_query_from_a_URL_using_URL_EXTRACT_QUERY_-_PROTOBUFs_-_PROTOBUF/7.3.0_1651071676942/spec.json new file mode 100644 index 000000000000..eaf99687facb --- /dev/null +++ b/ksqldb-functional-tests/src/test/resources/historical_plans/url_-_extract_a_query_from_a_URL_using_URL_EXTRACT_QUERY_-_PROTOBUFs_-_PROTOBUF/7.3.0_1651071676942/spec.json @@ -0,0 +1,158 @@ +{ + "version" : "7.3.0", + "timestamp" : 1651071676942, + "path" : "query-validation-tests/url.json", + "schemas" : { + "CSAS_OUTPUT_0.KsqlTopic.Source" : { + "schema" : "`K` STRING KEY, `URL` STRING", + "keyFormat" : { + "format" : "KAFKA" + }, + "valueFormat" : { + "format" : "PROTOBUF", + "properties" : { + "unwrapPrimitives" : "true" + } + } + }, + "CSAS_OUTPUT_0.OUTPUT" : { + "schema" : "`K` STRING KEY, `Q` STRING", + "keyFormat" : { + "format" : "KAFKA" + }, + "valueFormat" : { + "format" : "PROTOBUF", + "properties" : { + "unwrapPrimitives" : "true" + } + } + } + }, + "testCase" : { + "name" : "extract a query from a URL using URL_EXTRACT_QUERY - PROTOBUFs - PROTOBUF", + "inputs" : [ { + "topic" : "test_topic", + "key" : "1", + "value" : { + "url" : "http://www.test.com?query#fragment" + }, + "timestamp" : 0 + }, { + "topic" : "test_topic", + "key" : "2", + "value" : { + "url" : "http://www.test.com/path?q1&q2" + }, + "timestamp" : 1 + }, { + "topic" : "test_topic", + "key" : "3", + "value" : { + "url" : "http://test@confluent.io:8080/nested/path?q=2" + }, + "timestamp" : 2 + }, { + "topic" : "test_topic", + "key" : "4", + "value" : { + "url" : "http://test@confluent.io:8080/path" + }, + "timestamp" : 3 + } ], + "outputs" : [ { + "topic" : "OUTPUT", + "key" : "1", + "value" : { + "Q" : "query" + }, + "timestamp" : 0 + }, { + "topic" : "OUTPUT", + "key" : "2", + "value" : { + "Q" : "q1&q2" + }, + "timestamp" : 1 + }, { + "topic" : "OUTPUT", + "key" : "3", + "value" : { + "Q" : "q=2" + }, + "timestamp" : 2 + }, { + "topic" : "OUTPUT", + "key" : "4", + "value" : { + "Q" : "" + }, + "timestamp" : 3 + } ], + "topics" : [ { + "name" : "OUTPUT", + "replicas" : 1, + "numPartitions" : 4 + }, { + "name" : "test_topic", + "valueSchema" : "syntax = \"proto3\";\n\nmessage ConnectDefault1 {\n string URL = 1;\n}\n", + "valueFormat" : "PROTOBUF", + "replicas" : 1, + "numPartitions" : 4 + } ], + "statements" : [ "CREATE STREAM TEST (K STRING KEY, url VARCHAR) WITH (kafka_topic='test_topic', value_format='PROTOBUF');", "CREATE STREAM OUTPUT AS SELECT K, URL_EXTRACT_QUERY(url) as Q FROM TEST;" ], + "post" : { + "sources" : [ { + "name" : "OUTPUT", + "type" : "STREAM", + "schema" : "`K` STRING KEY, `Q` STRING", + "keyFormat" : { + "format" : "KAFKA" + }, + "valueFormat" : "PROTOBUF", + "keyFeatures" : [ ], + "valueFeatures" : [ ], + "isSource" : false + }, { + "name" : "TEST", + "type" : "STREAM", + "schema" : "`K` STRING KEY, `URL` STRING", + "keyFormat" : { + "format" : "KAFKA" + }, + "valueFormat" : "PROTOBUF", + "keyFeatures" : [ ], + "valueFeatures" : [ ], + "isSource" : false + } ], + "topics" : { + "topics" : [ { + "name" : "test_topic", + "keyFormat" : { + "format" : "KAFKA" + }, + "valueFormat" : { + "format" : "PROTOBUF", + "properties" : { + "unwrapPrimitives" : "true" + } + }, + "partitions" : 4, + "valueSchema" : "syntax = \"proto3\";\n\nmessage ConnectDefault1 {\n string URL = 1;\n}\n" + }, { + "name" : "OUTPUT", + "keyFormat" : { + "format" : "KAFKA" + }, + "valueFormat" : { + "format" : "PROTOBUF", + "properties" : { + "unwrapPrimitives" : "true" + } + }, + "partitions" : 4, + "valueSchema" : "syntax = \"proto3\";\n\nmessage ConnectDefault1 {\n string Q = 1;\n}\n" + } ] + } + } + } +} \ No newline at end of file diff --git a/ksqldb-functional-tests/src/test/resources/historical_plans/url_-_extract_a_query_from_a_URL_using_URL_EXTRACT_QUERY_-_PROTOBUFs_-_PROTOBUF/7.3.0_1651071676942/topology b/ksqldb-functional-tests/src/test/resources/historical_plans/url_-_extract_a_query_from_a_URL_using_URL_EXTRACT_QUERY_-_PROTOBUFs_-_PROTOBUF/7.3.0_1651071676942/topology new file mode 100644 index 000000000000..441a8f282644 --- /dev/null +++ b/ksqldb-functional-tests/src/test/resources/historical_plans/url_-_extract_a_query_from_a_URL_using_URL_EXTRACT_QUERY_-_PROTOBUFs_-_PROTOBUF/7.3.0_1651071676942/topology @@ -0,0 +1,13 @@ +Topologies: + Sub-topology: 0 + Source: KSTREAM-SOURCE-0000000000 (topics: [test_topic]) + --> KSTREAM-TRANSFORMVALUES-0000000001 + Processor: KSTREAM-TRANSFORMVALUES-0000000001 (stores: []) + --> Project + <-- KSTREAM-SOURCE-0000000000 + Processor: Project (stores: []) + --> KSTREAM-SINK-0000000003 + <-- KSTREAM-TRANSFORMVALUES-0000000001 + Sink: KSTREAM-SINK-0000000003 (topic: OUTPUT) + <-- Project + diff --git a/ksqldb-functional-tests/src/test/resources/historical_plans/url_-_extract_a_query_from_a_URL_using_URL_EXTRACT_QUERY_-_PROTOBUFs_-_PROTOBUF_NOSR/7.3.0_1651071676966/plan.json b/ksqldb-functional-tests/src/test/resources/historical_plans/url_-_extract_a_query_from_a_URL_using_URL_EXTRACT_QUERY_-_PROTOBUFs_-_PROTOBUF_NOSR/7.3.0_1651071676966/plan.json new file mode 100644 index 000000000000..844dabcad212 --- /dev/null +++ b/ksqldb-functional-tests/src/test/resources/historical_plans/url_-_extract_a_query_from_a_URL_using_URL_EXTRACT_QUERY_-_PROTOBUFs_-_PROTOBUF_NOSR/7.3.0_1651071676966/plan.json @@ -0,0 +1,203 @@ +{ + "plan" : [ { + "@type" : "ksqlPlanV1", + "statementText" : "CREATE STREAM TEST (K STRING KEY, URL STRING) WITH (KAFKA_TOPIC='test_topic', KEY_FORMAT='KAFKA', VALUE_FORMAT='PROTOBUF_NOSR');", + "ddlCommand" : { + "@type" : "createStreamV1", + "sourceName" : "TEST", + "schema" : "`K` STRING KEY, `URL` STRING", + "timestampColumn" : null, + "topicName" : "test_topic", + "formats" : { + "keyFormat" : { + "format" : "KAFKA", + "properties" : { } + }, + "valueFormat" : { + "format" : "PROTOBUF_NOSR", + "properties" : { } + } + }, + "windowInfo" : null, + "orReplace" : false, + "isSource" : false + }, + "queryPlan" : null + }, { + "@type" : "ksqlPlanV1", + "statementText" : "CREATE STREAM OUTPUT AS SELECT\n TEST.K K,\n URL_EXTRACT_QUERY(TEST.URL) Q\nFROM TEST TEST\nEMIT CHANGES", + "ddlCommand" : { + "@type" : "createStreamV1", + "sourceName" : "OUTPUT", + "schema" : "`K` STRING KEY, `Q` STRING", + "timestampColumn" : null, + "topicName" : "OUTPUT", + "formats" : { + "keyFormat" : { + "format" : "KAFKA", + "properties" : { } + }, + "valueFormat" : { + "format" : "PROTOBUF_NOSR", + "properties" : { } + } + }, + "windowInfo" : null, + "orReplace" : false, + "isSource" : false + }, + "queryPlan" : { + "sources" : [ "TEST" ], + "sink" : "OUTPUT", + "physicalPlan" : { + "@type" : "streamSinkV1", + "properties" : { + "queryContext" : "OUTPUT" + }, + "source" : { + "@type" : "streamSelectV1", + "properties" : { + "queryContext" : "Project" + }, + "source" : { + "@type" : "streamSourceV1", + "properties" : { + "queryContext" : "KsqlTopic/Source" + }, + "topicName" : "test_topic", + "formats" : { + "keyFormat" : { + "format" : "KAFKA", + "properties" : { } + }, + "valueFormat" : { + "format" : "PROTOBUF_NOSR", + "properties" : { } + } + }, + "timestampColumn" : null, + "sourceSchema" : "`K` STRING KEY, `URL` STRING", + "pseudoColumnVersion" : 1 + }, + "keyColumnNames" : [ "K" ], + "selectedKeys" : null, + "selectExpressions" : [ "URL_EXTRACT_QUERY(URL) AS Q" ] + }, + "formats" : { + "keyFormat" : { + "format" : "KAFKA", + "properties" : { } + }, + "valueFormat" : { + "format" : "PROTOBUF_NOSR", + "properties" : { } + } + }, + "topicName" : "OUTPUT", + "timestampColumn" : null + }, + "queryId" : "CSAS_OUTPUT_0", + "runtimeId" : null + } + } ], + "configs" : { + "ksql.extension.dir" : "ext", + "ksql.streams.cache.max.bytes.buffering" : "0", + "metric.reporters" : "", + "ksql.query.status.running.threshold.seconds" : "300", + "ksql.connect.basic.auth.credentials.reload" : "false", + "ksql.output.topic.name.prefix" : "", + "ksql.query.pull.stream.enabled" : "true", + "ksql.query.push.v2.interpreter.enabled" : "true", + "ksql.queryanonymizer.logs_enabled" : "true", + "ksql.variable.substitution.enable" : "true", + "ksql.streams.shutdown.timeout.ms" : "300000", + "ksql.query.pull.max.allowed.offset.lag" : "9223372036854775807", + "ksql.query.pull.max.qps" : "2147483647", + "ksql.access.validator.enable" : "auto", + "ksql.streams.bootstrap.servers" : "localhost:0", + "ksql.query.pull.metrics.enabled" : "true", + "ksql.metrics.extension" : null, + "ksql.query.push.v2.alos.enabled" : "true", + "ksql.query.push.v2.max.hourly.bandwidth.megabytes" : "2147483647", + "ksql.query.pull.range.scan.enabled" : "true", + "ksql.transient.query.cleanup.service.initial.delay.seconds" : "600", + "ksql.hidden.topics" : "_confluent.*,__confluent.*,_schemas,__consumer_offsets,__transaction_state,connect-configs,connect-offsets,connect-status,connect-statuses", + "ksql.lambdas.enabled" : "true", + "ksql.source.table.materialization.enabled" : "true", + "ksql.query.pull.max.hourly.bandwidth.megabytes" : "2147483647", + "ksql.sink.window.change.log.additional.retention" : "1000000", + "ksql.query.persistent.active.limit" : "2147483647", + "ksql.persistence.wrap.single.values" : null, + "ksql.query.transient.max.bytes.buffering.total" : "-1", + "ksql.connect.basic.auth.credentials.source" : "NONE", + "ksql.schema.registry.url" : "schema_registry.url:0", + "ksql.properties.overrides.denylist" : "", + "ksql.service.id" : "some.ksql.service.id", + "ksql.query.push.v2.max.catchup.consumers" : "5", + "ksql.query.push.v2.enabled" : "false", + "ksql.transient.query.cleanup.service.enable" : "true", + "ksql.query.push.v2.metrics.enabled" : "true", + "ksql.rowpartition.rowoffset.enabled" : "true", + "ksql.streams.commit.interval.ms" : "2000", + "ksql.query.pull.table.scan.enabled" : "true", + "ksql.streams.auto.commit.interval.ms" : "0", + "ksql.streams.topology.optimization" : "all", + "ksql.endpoint.migrate.query" : "true", + "ksql.query.push.v2.registry.installed" : "false", + "ksql.streams.num.stream.threads" : "4", + "ksql.metrics.tags.custom" : "", + "ksql.query.push.v2.catchup.consumer.msg.window" : "50", + "ksql.runtime.feature.shared.enabled" : "false", + "ksql.udf.collect.metrics" : "false", + "ksql.new.query.planner.enabled" : "false", + "ksql.connect.request.headers.plugin" : null, + "ksql.security.extension.class" : null, + "ksql.transient.prefix" : "transient_", + "ksql.headers.columns.enabled" : "true", + "ksql.streams.default.deserialization.exception.handler" : "io.confluent.ksql.errors.LogMetricAndContinueExceptionHandler", + "ksql.connect.request.timeout.ms" : "5000", + "ksql.query.pull.enable.standby.reads" : "false", + "ksql.persistence.default.format.key" : "KAFKA", + "ksql.query.persistent.max.bytes.buffering.total" : "-1", + "ksql.query.error.max.queue.size" : "10", + "ksql.query.cleanup.shutdown.timeout.ms" : "30000", + "ksql.internal.topic.min.insync.replicas" : "1", + "ksql.internal.topic.replicas" : "1", + "ksql.insert.into.values.enabled" : "true", + "ksql.queryanonymizer.cluster_namespace" : null, + "ksql.create.or.replace.enabled" : "true", + "ksql.shared.runtimes.count" : "2", + "ksql.cast.strings.preserve.nulls" : "true", + "ksql.authorization.cache.max.entries" : "10000", + "ksql.pull.queries.enable" : "true", + "ksql.transient.query.cleanup.service.period.seconds" : "600", + "ksql.suppress.enabled" : "false", + "ksql.readonly.topics" : "_confluent.*,__confluent.*,_schemas,__consumer_offsets,__transaction_state,connect-configs,connect-offsets,connect-status,connect-statuses", + "ksql.connect.basic.auth.credentials.file" : "", + "ksql.authorization.cache.expiry.time.secs" : "30", + "ksql.query.retry.backoff.initial.ms" : "15000", + "ksql.query.pull.max.concurrent.requests" : "2147483647", + "ksql.streams.auto.offset.reset" : "earliest", + "ksql.connect.url" : "http://localhost:8083", + "ksql.query.push.v2.new.latest.delay.ms" : "5000", + "ksql.query.push.v2.latest.reset.age.ms" : "30000", + "ksql.streams.default.production.exception.handler" : "io.confluent.ksql.errors.ProductionExceptionHandlerUtil$LogAndFailProductionExceptionHandler", + "ksql.query.pull.interpreter.enabled" : "true", + "ksql.query.pull.limit.clause.enabled" : "true", + "ksql.query.pull.router.thread.pool.size" : "50", + "ksql.query.push.v2.continuation.tokens.enabled" : "false", + "ksql.query.retry.backoff.max.ms" : "900000", + "ksql.timestamp.throw.on.invalid" : "false", + "ksql.persistence.default.format.value" : null, + "ksql.udfs.enabled" : "true", + "ksql.udf.enable.security.manager" : "true", + "ksql.connect.worker.config" : "", + "ksql.nested.error.set.null" : "true", + "ksql.query.pull.thread.pool.size" : "50", + "ksql.persistent.prefix" : "query_", + "ksql.metastore.backup.location" : "", + "ksql.error.classifier.regex" : "", + "ksql.suppress.buffer.size.bytes" : "-1" + } +} \ No newline at end of file diff --git a/ksqldb-functional-tests/src/test/resources/historical_plans/url_-_extract_a_query_from_a_URL_using_URL_EXTRACT_QUERY_-_PROTOBUFs_-_PROTOBUF_NOSR/7.3.0_1651071676966/spec.json b/ksqldb-functional-tests/src/test/resources/historical_plans/url_-_extract_a_query_from_a_URL_using_URL_EXTRACT_QUERY_-_PROTOBUFs_-_PROTOBUF_NOSR/7.3.0_1651071676966/spec.json new file mode 100644 index 000000000000..2d3035e9e6d4 --- /dev/null +++ b/ksqldb-functional-tests/src/test/resources/historical_plans/url_-_extract_a_query_from_a_URL_using_URL_EXTRACT_QUERY_-_PROTOBUFs_-_PROTOBUF_NOSR/7.3.0_1651071676966/spec.json @@ -0,0 +1,142 @@ +{ + "version" : "7.3.0", + "timestamp" : 1651071676966, + "path" : "query-validation-tests/url.json", + "schemas" : { + "CSAS_OUTPUT_0.KsqlTopic.Source" : { + "schema" : "`K` STRING KEY, `URL` STRING", + "keyFormat" : { + "format" : "KAFKA" + }, + "valueFormat" : { + "format" : "PROTOBUF_NOSR" + } + }, + "CSAS_OUTPUT_0.OUTPUT" : { + "schema" : "`K` STRING KEY, `Q` STRING", + "keyFormat" : { + "format" : "KAFKA" + }, + "valueFormat" : { + "format" : "PROTOBUF_NOSR" + } + } + }, + "testCase" : { + "name" : "extract a query from a URL using URL_EXTRACT_QUERY - PROTOBUFs - PROTOBUF_NOSR", + "inputs" : [ { + "topic" : "test_topic", + "key" : "1", + "value" : { + "url" : "http://www.test.com?query#fragment" + }, + "timestamp" : 0 + }, { + "topic" : "test_topic", + "key" : "2", + "value" : { + "url" : "http://www.test.com/path?q1&q2" + }, + "timestamp" : 1 + }, { + "topic" : "test_topic", + "key" : "3", + "value" : { + "url" : "http://test@confluent.io:8080/nested/path?q=2" + }, + "timestamp" : 2 + }, { + "topic" : "test_topic", + "key" : "4", + "value" : { + "url" : "http://test@confluent.io:8080/path" + }, + "timestamp" : 3 + } ], + "outputs" : [ { + "topic" : "OUTPUT", + "key" : "1", + "value" : { + "Q" : "query" + }, + "timestamp" : 0 + }, { + "topic" : "OUTPUT", + "key" : "2", + "value" : { + "Q" : "q1&q2" + }, + "timestamp" : 1 + }, { + "topic" : "OUTPUT", + "key" : "3", + "value" : { + "Q" : "q=2" + }, + "timestamp" : 2 + }, { + "topic" : "OUTPUT", + "key" : "4", + "value" : { + "Q" : "" + }, + "timestamp" : 3 + } ], + "topics" : [ { + "name" : "OUTPUT", + "replicas" : 1, + "numPartitions" : 4 + }, { + "name" : "test_topic", + "replicas" : 1, + "numPartitions" : 4 + } ], + "statements" : [ "CREATE STREAM TEST (K STRING KEY, url VARCHAR) WITH (kafka_topic='test_topic', value_format='PROTOBUF_NOSR');", "CREATE STREAM OUTPUT AS SELECT K, URL_EXTRACT_QUERY(url) as Q FROM TEST;" ], + "post" : { + "sources" : [ { + "name" : "OUTPUT", + "type" : "STREAM", + "schema" : "`K` STRING KEY, `Q` STRING", + "keyFormat" : { + "format" : "KAFKA" + }, + "valueFormat" : "PROTOBUF_NOSR", + "keyFeatures" : [ ], + "valueFeatures" : [ ], + "isSource" : false + }, { + "name" : "TEST", + "type" : "STREAM", + "schema" : "`K` STRING KEY, `URL` STRING", + "keyFormat" : { + "format" : "KAFKA" + }, + "valueFormat" : "PROTOBUF_NOSR", + "keyFeatures" : [ ], + "valueFeatures" : [ ], + "isSource" : false + } ], + "topics" : { + "topics" : [ { + "name" : "test_topic", + "keyFormat" : { + "format" : "KAFKA" + }, + "valueFormat" : { + "format" : "PROTOBUF_NOSR" + }, + "partitions" : 4 + }, { + "name" : "OUTPUT", + "keyFormat" : { + "format" : "KAFKA" + }, + "valueFormat" : { + "format" : "PROTOBUF_NOSR" + }, + "partitions" : 4 + } ] + } + } + } +} \ No newline at end of file diff --git a/ksqldb-functional-tests/src/test/resources/historical_plans/url_-_extract_a_query_from_a_URL_using_URL_EXTRACT_QUERY_-_PROTOBUFs_-_PROTOBUF_NOSR/7.3.0_1651071676966/topology b/ksqldb-functional-tests/src/test/resources/historical_plans/url_-_extract_a_query_from_a_URL_using_URL_EXTRACT_QUERY_-_PROTOBUFs_-_PROTOBUF_NOSR/7.3.0_1651071676966/topology new file mode 100644 index 000000000000..441a8f282644 --- /dev/null +++ b/ksqldb-functional-tests/src/test/resources/historical_plans/url_-_extract_a_query_from_a_URL_using_URL_EXTRACT_QUERY_-_PROTOBUFs_-_PROTOBUF_NOSR/7.3.0_1651071676966/topology @@ -0,0 +1,13 @@ +Topologies: + Sub-topology: 0 + Source: KSTREAM-SOURCE-0000000000 (topics: [test_topic]) + --> KSTREAM-TRANSFORMVALUES-0000000001 + Processor: KSTREAM-TRANSFORMVALUES-0000000001 (stores: []) + --> Project + <-- KSTREAM-SOURCE-0000000000 + Processor: Project (stores: []) + --> KSTREAM-SINK-0000000003 + <-- KSTREAM-TRANSFORMVALUES-0000000001 + Sink: KSTREAM-SINK-0000000003 (topic: OUTPUT) + <-- Project + diff --git a/ksqldb-functional-tests/src/test/resources/query-validation-tests/asserts.json b/ksqldb-functional-tests/src/test/resources/query-validation-tests/asserts.json index a481267c6a4f..b95dbe8df861 100644 --- a/ksqldb-functional-tests/src/test/resources/query-validation-tests/asserts.json +++ b/ksqldb-functional-tests/src/test/resources/query-validation-tests/asserts.json @@ -7,7 +7,7 @@ ], "expectedException": { "type": "io.confluent.ksql.parser.exception.ParseFailedException", - "message": "mismatched input 'ASSERT'" + "message": "no viable alternative at input 'ASSERT VALUES'" } }, { diff --git a/ksqldb-functional-tests/src/test/resources/query-validation-tests/bytes.json b/ksqldb-functional-tests/src/test/resources/query-validation-tests/bytes.json index eeff9f562cd3..3249b84b8cfc 100644 --- a/ksqldb-functional-tests/src/test/resources/query-validation-tests/bytes.json +++ b/ksqldb-functional-tests/src/test/resources/query-validation-tests/bytes.json @@ -52,6 +52,19 @@ ] } }, + { + "name": "PROTOBUF_NOSR in/out", + "statements": [ + "CREATE STREAM TEST (ID STRING KEY, b BYTES) WITH (kafka_topic='test', value_format='PROTOBUF_NOSR');", + "CREATE STREAM TEST2 AS SELECT * FROM TEST;" + ], + "inputs": [ + {"topic": "test", "value": {"b": "dmFyaWF0aW9ucw=="}} + ], + "outputs": [ + {"topic": "TEST2", "value": {"B": "dmFyaWF0aW9ucw=="}} + ] + }, { "name": "AVRO in/out", "statements": [ diff --git a/ksqldb-functional-tests/src/test/resources/query-validation-tests/collect-list.json b/ksqldb-functional-tests/src/test/resources/query-validation-tests/collect-list.json index c858b7908f1b..e48500e05ab3 100644 --- a/ksqldb-functional-tests/src/test/resources/query-validation-tests/collect-list.json +++ b/ksqldb-functional-tests/src/test/resources/query-validation-tests/collect-list.json @@ -74,7 +74,7 @@ }, { "name": "collect_list struct", - "format": ["AVRO", "JSON", "PROTOBUF"], + "format": ["AVRO", "JSON", "PROTOBUF", "PROTOBUF_NOSR"], "statements": [ "CREATE STREAM TEST (ID BIGINT KEY, VALUE STRUCT) WITH (kafka_topic='test_topic',value_format='{FORMAT}');", "CREATE TABLE S2 as SELECT ID, collect_list(value) as collected FROM test group by id;" @@ -94,7 +94,7 @@ }, { "name": "collect_list int", - "format": ["AVRO", "JSON", "PROTOBUF"], + "format": ["AVRO", "JSON", "PROTOBUF", "PROTOBUF_NOSR"], "statements": [ "CREATE STREAM TEST (ID BIGINT KEY, VALUE integer) WITH (kafka_topic='test_topic',value_format='{FORMAT}');", "CREATE TABLE S2 as SELECT ID, collect_list(value) as collected FROM test group by id;" @@ -114,7 +114,7 @@ }, { "name": "collect_list long", - "format": ["AVRO", "JSON", "PROTOBUF"], + "format": ["AVRO", "JSON", "PROTOBUF", "PROTOBUF_NOSR"], "statements": [ "CREATE STREAM TEST (ID BIGINT KEY, VALUE bigint) WITH (kafka_topic='test_topic', value_format='{FORMAT}');", "CREATE TABLE S2 as SELECT ID, collect_list(value) as collected FROM test group by id;" @@ -204,7 +204,7 @@ }, { "name": "collect_list bool map", - "format": ["JSON", "PROTOBUF"], + "format": ["JSON", "PROTOBUF", "PROTOBUF_NOSR"], "statements": [ "CREATE STREAM TEST (ID BIGINT KEY, NAME varchar, VALUE map) WITH (kafka_topic='test_topic', value_format='{FORMAT}');", "CREATE TABLE S2 as SELECT ID, collect_list(value['key1']) AS collected FROM test group by id;" @@ -222,7 +222,7 @@ }, { "name": "collect_list timestamp map", - "format": ["JSON", "PROTOBUF"], + "format": ["JSON", "PROTOBUF", "PROTOBUF_NOSR"], "statements": [ "CREATE STREAM TEST (ID BIGINT KEY, NAME varchar, VALUE map) WITH (kafka_topic='test_topic', value_format='{FORMAT}');", "CREATE TABLE S2 as SELECT ID, collect_list(value['key1']) AS collected FROM test group by id;" @@ -240,7 +240,7 @@ }, { "name": "collect_list time map", - "format": ["JSON", "PROTOBUF"], + "format": ["JSON", "PROTOBUF", "PROTOBUF_NOSR"], "statements": [ "CREATE STREAM TEST (ID BIGINT KEY, NAME varchar, VALUE map) WITH (kafka_topic='test_topic', value_format='{FORMAT}');", "CREATE TABLE S2 as SELECT ID, collect_list(value['key1']) AS collected FROM test group by id;" @@ -258,7 +258,7 @@ }, { "name": "collect_list date map", - "format": ["JSON", "PROTOBUF"], + "format": ["JSON", "PROTOBUF", "PROTOBUF_NOSR"], "statements": [ "CREATE STREAM TEST (ID BIGINT KEY, NAME varchar, VALUE map) WITH (kafka_topic='test_topic', value_format='{FORMAT}');", "CREATE TABLE S2 as SELECT ID, collect_list(value['key1']) AS collected FROM test group by id;" @@ -292,7 +292,7 @@ }, { "name": "collect_list int table", - "format": ["AVRO", "JSON", "PROTOBUF"], + "format": ["AVRO", "JSON", "PROTOBUF", "PROTOBUF_NOSR"], "statements": [ "CREATE TABLE TEST (ID BIGINT PRIMARY KEY, VALUE integer) WITH (kafka_topic='test_topic',value_format='{FORMAT}');", "CREATE TABLE S2 as SELECT ID, collect_list(value) as collected FROM test group by id;" @@ -314,7 +314,7 @@ }, { "name": "collect_list long table", - "format": ["AVRO", "JSON", "PROTOBUF"], + "format": ["AVRO", "JSON", "PROTOBUF", "PROTOBUF_NOSR"], "statements": [ "CREATE TABLE TEST (ID BIGINT PRIMARY KEY, VALUE bigint) WITH (kafka_topic='test_topic', value_format='{FORMAT}');", "CREATE TABLE S2 as SELECT ID, collect_list(value) as collected FROM test group by id;" @@ -336,7 +336,7 @@ }, { "name": "collect_list double table", - "format": ["AVRO", "JSON", "PROTOBUF"], + "format": ["AVRO", "JSON", "PROTOBUF", "PROTOBUF_NOSR"], "statements": [ "CREATE TABLE TEST (ID BIGINT PRIMARY KEY, VALUE double) WITH (kafka_topic='test_topic', value_format='{FORMAT}');", "CREATE TABLE S2 as SELECT ID, collect_list(value) as collected FROM test group by id;" @@ -358,7 +358,7 @@ }, { "name": "collect_list string table", - "format": ["AVRO", "JSON", "PROTOBUF"], + "format": ["AVRO", "JSON", "PROTOBUF", "PROTOBUF_NOSR"], "statements": [ "CREATE TABLE TEST (ID BIGINT PRIMARY KEY, VALUE varchar) WITH (kafka_topic='test_topic', value_format='{FORMAT}');", "CREATE TABLE S2 as SELECT ID, collect_list(value) as collected FROM test group by id;" @@ -383,7 +383,7 @@ }, { "name": "collect_list bool map table", - "format": ["JSON", "PROTOBUF"], + "format": ["JSON", "PROTOBUF", "PROTOBUF_NOSR"], "statements": [ "CREATE TABLE TEST (ID BIGINT PRIMARY KEY, NAME varchar, VALUE map) WITH (kafka_topic='test_topic', value_format='{FORMAT}');", "CREATE TABLE S2 as SELECT ID, collect_list(value['key1']) AS collected FROM test group by id;" @@ -403,7 +403,7 @@ }, { "name": "collect_list timestamp map table", - "format": ["JSON", "PROTOBUF"], + "format": ["JSON", "PROTOBUF", "PROTOBUF_NOSR"], "statements": [ "CREATE TABLE TEST (ID BIGINT PRIMARY KEY, NAME varchar, VALUE map) WITH (kafka_topic='test_topic', value_format='{FORMAT}');", "CREATE TABLE S2 as SELECT ID, collect_list(value['key1']) AS collected FROM test group by id;" @@ -423,7 +423,7 @@ }, { "name": "collect_list time map table", - "format": ["JSON", "PROTOBUF"], + "format": ["JSON", "PROTOBUF", "PROTOBUF_NOSR"], "statements": [ "CREATE TABLE TEST (ID BIGINT PRIMARY KEY, NAME varchar, VALUE map) WITH (kafka_topic='test_topic', value_format='{FORMAT}');", "CREATE TABLE S2 as SELECT ID, collect_list(value['key1']) AS collected FROM test group by id;" @@ -443,7 +443,7 @@ }, { "name": "collect_list date map table", - "format": ["JSON", "PROTOBUF"], + "format": ["JSON", "PROTOBUF", "PROTOBUF_NOSR"], "statements": [ "CREATE TABLE TEST (ID BIGINT PRIMARY KEY, NAME varchar, VALUE map) WITH (kafka_topic='test_topic', value_format='{FORMAT}');", "CREATE TABLE S2 as SELECT ID, collect_list(value['key1']) AS collected FROM test group by id;" diff --git a/ksqldb-functional-tests/src/test/resources/query-validation-tests/collect-set.json b/ksqldb-functional-tests/src/test/resources/query-validation-tests/collect-set.json index 982854540589..d9a3f2e24603 100644 --- a/ksqldb-functional-tests/src/test/resources/query-validation-tests/collect-set.json +++ b/ksqldb-functional-tests/src/test/resources/query-validation-tests/collect-set.json @@ -74,7 +74,7 @@ }, { "name": "collect_set struct", - "format": ["AVRO", "JSON", "PROTOBUF"], + "format": ["AVRO", "JSON", "PROTOBUF", "PROTOBUF_NOSR"], "statements": [ "CREATE STREAM TEST (ID BIGINT KEY, VALUE STRUCT) WITH (kafka_topic='test_topic',value_format='{FORMAT}');", "CREATE TABLE S2 as SELECT ID, collect_set(value) as collected FROM test group by id;" @@ -94,7 +94,7 @@ }, { "name": "collect_set int", - "format": ["AVRO", "JSON", "PROTOBUF"], + "format": ["AVRO", "JSON", "PROTOBUF", "PROTOBUF_NOSR"], "statements": [ "CREATE STREAM TEST (ID BIGINT KEY, VALUE integer) WITH (kafka_topic='test_topic',value_format='{FORMAT}');", "CREATE TABLE S2 as SELECT ID, collect_set(value) as collected FROM test group by id;" @@ -114,7 +114,7 @@ }, { "name": "collect_set long", - "format": ["AVRO", "JSON", "PROTOBUF"], + "format": ["AVRO", "JSON", "PROTOBUF", "PROTOBUF_NOSR"], "statements": [ "CREATE STREAM TEST (ID BIGINT KEY, VALUE bigint) WITH (kafka_topic='test_topic', value_format='{FORMAT}');", "CREATE TABLE S2 as SELECT ID, collect_set(value) as collected FROM test group by id;" @@ -160,7 +160,7 @@ }, { "name": "collect_set double", - "format": ["AVRO", "JSON", "PROTOBUF"], + "format": ["AVRO", "JSON", "PROTOBUF", "PROTOBUF_NOSR"], "statements": [ "CREATE STREAM TEST (ID BIGINT KEY, VALUE double) WITH (kafka_topic='test_topic', value_format='{FORMAT}');", "CREATE TABLE S2 as SELECT ID, collect_set(value) as collected FROM test group by id;" @@ -202,9 +202,33 @@ {"topic": "S2", "key": 100,"value": {"COLLECTED": ["baz","foo", null]}} ] }, + { + "name": "collect_set string - PROTOBUFs", + "format": ["PROTOBUF", "PROTOBUF_NOSR"], + "statements": [ + "CREATE STREAM TEST (ID BIGINT KEY, VALUE varchar) WITH (kafka_topic='test_topic', value_format='{FORMAT}');", + "CREATE TABLE S2 as SELECT ID, collect_set(value) as collected FROM test group by id;" + ], + "inputs": [ + {"topic": "test_topic", "key": 0,"value": {"VALUE": "foo"}}, + {"topic": "test_topic", "key": 100,"value": {"VALUE": "baz"}}, + {"topic": "test_topic", "key": 0,"value": {"VALUE": "bar"}}, + {"topic": "test_topic", "key": 100,"value": {"VALUE": "baz"}}, + {"topic": "test_topic", "key": 100,"value": {"VALUE": "foo"}}, + {"topic": "test_topic", "key": 100, "value": {"VALUE": null}} + ], + "outputs": [ + {"topic": "S2", "key": 0,"value": {"COLLECTED": ["foo"]}}, + {"topic": "S2", "key": 100,"value": {"COLLECTED": ["baz"]}}, + {"topic": "S2", "key": 0,"value": {"COLLECTED": ["foo","bar"]}, "timestamp": 0}, + {"topic": "S2", "key": 100,"value": {"COLLECTED": ["baz"]}}, + {"topic": "S2", "key": 100,"value": {"COLLECTED": ["baz","foo"]}}, + {"topic": "S2", "key": 100,"value": {"COLLECTED": ["baz","foo", ""]}} + ] + }, { "name": "collect_set bool map", - "format": ["JSON", "PROTOBUF"], + "format": ["JSON", "AVRO", "PROTOBUF", "PROTOBUF_NOSR"], "statements": [ "CREATE STREAM TEST (ID BIGINT KEY, NAME varchar, VALUE map) WITH (kafka_topic='test_topic', value_format='{FORMAT}');", "CREATE TABLE S2 as SELECT ID, collect_set(value['key1']) AS collected FROM test group by id;" @@ -222,7 +246,7 @@ }, { "name": "collect_list timestamp map", - "format": ["JSON", "PROTOBUF"], + "format": ["JSON", "AVRO", "PROTOBUF", "PROTOBUF_NOSR"], "statements": [ "CREATE STREAM TEST (ID BIGINT KEY, NAME varchar, VALUE map) WITH (kafka_topic='test_topic', value_format='{FORMAT}');", "CREATE TABLE S2 as SELECT ID, collect_set(value['key1']) AS collected FROM test group by id;" @@ -240,7 +264,7 @@ }, { "name": "collect_list time map", - "format": ["JSON", "PROTOBUF"], + "format": ["JSON", "AVRO", "PROTOBUF", "PROTOBUF_NOSR"], "statements": [ "CREATE STREAM TEST (ID BIGINT KEY, NAME varchar, VALUE map) WITH (kafka_topic='test_topic', value_format='{FORMAT}');", "CREATE TABLE S2 as SELECT ID, collect_set(value['key1']) AS collected FROM test group by id;" @@ -258,7 +282,7 @@ }, { "name": "collect_list date map", - "format": ["JSON", "PROTOBUF"], + "format": ["JSON", "PROTOBUF", "PROTOBUF_NOSR"], "statements": [ "CREATE STREAM TEST (ID BIGINT KEY, NAME varchar, VALUE map) WITH (kafka_topic='test_topic', value_format='{FORMAT}');", "CREATE TABLE S2 as SELECT ID, collect_set(value['key1']) AS collected FROM test group by id;" diff --git a/ksqldb-functional-tests/src/test/resources/query-validation-tests/date.json b/ksqldb-functional-tests/src/test/resources/query-validation-tests/date.json index 21d5380999d2..83281a73708b 100644 --- a/ksqldb-functional-tests/src/test/resources/query-validation-tests/date.json +++ b/ksqldb-functional-tests/src/test/resources/query-validation-tests/date.json @@ -60,7 +60,23 @@ {"topic": "TEST2", "value": {"DATE": 10}}, {"topic": "TEST2", "value": {"DATE": -10}} ] - },{ + }, + { + "name": "PROTOBUF_NOSR in/out", + "statements": [ + "CREATE STREAM TEST (ID STRING KEY, date DATE) WITH (kafka_topic='test', value_format='PROTOBUF_NOSR');", + "CREATE STREAM TEST2 AS SELECT * FROM TEST;" + ], + "inputs": [ + {"topic": "test", "value": {"date": 10}}, + {"topic": "test", "value": {"date": -10}} + ], + "outputs": [ + {"topic": "TEST2", "value": {"DATE": 10}}, + {"topic": "TEST2", "value": {"DATE": -10}} + ] + }, + { "name": "casting - date to string", "statements": [ "CREATE STREAM TEST (ID STRING KEY, date DATE) WITH (kafka_topic='test', value_format='AVRO');", diff --git a/ksqldb-functional-tests/src/test/resources/query-validation-tests/decimal.json b/ksqldb-functional-tests/src/test/resources/query-validation-tests/decimal.json index f8f77bdb1d53..2c86a4a4008d 100644 --- a/ksqldb-functional-tests/src/test/resources/query-validation-tests/decimal.json +++ b/ksqldb-functional-tests/src/test/resources/query-validation-tests/decimal.json @@ -57,6 +57,19 @@ {"topic": "TEST2", "value": {"DEC": 10.1234512345123451234}} ] }, + { + "name": "PROTOBUF_NOSR in/out", + "statements": [ + "CREATE STREAM TEST (ID STRING KEY, dec DECIMAL(21,19)) WITH (kafka_topic='test', value_format='PROTOBUF_NOSR');", + "CREATE STREAM TEST2 AS SELECT * FROM TEST;" + ], + "inputs": [ + {"topic": "test", "value": {"DEC": 10.1234512345123451234}} + ], + "outputs": [ + {"topic": "TEST2", "value": {"DEC": 10.1234512345123451234}} + ] + }, { "name": "JSON scale in data less than scale in type", "statements": [ @@ -192,6 +205,27 @@ ] } }, + { + "name": "PROTOBUF_NOSR should not trim trailing zeros", + "statements": [ + "CREATE STREAM INPUT (ID STRING KEY, dec DECIMAL(6,4)) WITH (kafka_topic='test', value_format='PROTOBUF_NOSR');", + "CREATE STREAM OUTPUT AS SELECT * FROM INPUT;" + ], + "inputs": [ + {"topic": "test", "value": {"DEC": 10.0}}, + {"topic": "test", "value": {"DEC": 1.0000}} + ], + "outputs": [ + {"topic": "OUTPUT", "value": {"DEC": 10.0000}}, + {"topic": "OUTPUT", "value": {"DEC": 1.0000}} + ], + "post": { + "sources": [ + {"name": "INPUT", "type": "stream", "schema": "ID STRING KEY, DEC DECIMAL(6,4)"}, + {"name": "OUTPUT", "type": "stream", "schema": "ID STRING KEY, DEC DECIMAL(6,4)"} + ] + } + }, { "name": "negation", "statements": [ diff --git a/ksqldb-functional-tests/src/test/resources/query-validation-tests/delimited.json b/ksqldb-functional-tests/src/test/resources/query-validation-tests/delimited.json index d32eb858f541..b88a82b79e66 100644 --- a/ksqldb-functional-tests/src/test/resources/query-validation-tests/delimited.json +++ b/ksqldb-functional-tests/src/test/resources/query-validation-tests/delimited.json @@ -286,7 +286,7 @@ }, { "name": "select delimited value_format into another format", - "format": ["JSON", "AVRO", "PROTOBUF"], + "format": ["JSON", "AVRO", "PROTOBUF","PROTOBUF_NOSR"], "statements": [ "CREATE STREAM TEST (K STRING KEY, ID bigint, NAME varchar, VALUE integer) WITH (kafka_topic='test_topic', value_format='DELIMITED', value_delimiter=',');", "CREATE STREAM S2 WITH(value_format='{FORMAT}') as SELECT K, id, name, value FROM test;" diff --git a/ksqldb-functional-tests/src/test/resources/query-validation-tests/elements.json b/ksqldb-functional-tests/src/test/resources/query-validation-tests/elements.json index 18c5aac3668f..d1f90573f69c 100644 --- a/ksqldb-functional-tests/src/test/resources/query-validation-tests/elements.json +++ b/ksqldb-functional-tests/src/test/resources/query-validation-tests/elements.json @@ -170,6 +170,27 @@ "inputs": [{"topic": "input", "value": {"c1": 4}}], "outputs": [{"topic": "OUTPUT", "value": {"C1": 4}}] }, + { + "name": "validate without elements with non-default schema name OK - PROTOBUF", + "statements": [ + "CREATE STREAM INPUT WITH (kafka_topic='input', value_format='PROTOBUF');", + "CREATE STREAM OUTPUT WITH(PARTITIONS = 4) as SELECT * FROM input;" + ], + "topics": [ + { + "name": "input", + "valueSchema": "syntax = \"proto3\"; message ValueName { int32 c1 = 1; }", + "valueFormat": "PROTOBUF" + }, + { + "name": "OUTPUT", + "valueFormat": "PROTOBUF", + "partitions": 4 + } + ], + "inputs": [{"topic": "input", "value": {"c1": 4}}], + "outputs": [{"topic": "OUTPUT", "value": {"C1": 4}}] + }, { "name": "validate schema id without elements OK - PROTOBUF", "statements": [ @@ -191,6 +212,27 @@ "inputs": [{"topic": "input", "value": {"c1": 4}}], "outputs": [{"topic": "OUTPUT", "value": {"c1": 4}}] }, + { + "name": "validate schema id without elements with non-default schema name OK - PROTOBUF", + "statements": [ + "CREATE STREAM INPUT WITH (kafka_topic='input', value_format='PROTOBUF', value_schema_id=1);", + "CREATE STREAM OUTPUT WITH(PARTITIONS = 4) as SELECT * FROM input;" + ], + "topics": [ + { + "name": "input", + "valueSchema": "syntax = \"proto3\"; message ValueName { int32 c1 = 1; }", + "valueFormat": "PROTOBUF" + }, + { + "name": "OUTPUT", + "valueFormat": "PROTOBUF", + "partitions": 4 + } + ], + "inputs": [{"topic": "input", "value": {"c1": 4}}], + "outputs": [{"topic": "OUTPUT", "value": {"c1": 4}}] + }, { "name": "validate without elements OK - JSON_SR SCHEMA", "statements": [ @@ -412,6 +454,34 @@ ] } }, + { + "name": "validate schema id and schema full name from multiple schema definitions OK - PROTOBUF", + "statements": [ + "CREATE STREAM INPUT WITH (kafka_topic='input', format='PROTOBUF', key_schema_id=1, key_schema_full_name='KeySchema2', value_schema_id=2, value_schema_full_name='ValueSchema2');", + "CREATE STREAM OUTPUT AS SELECT * FROM input;" + ], + "topics": [ + { + "name": "input", + "valueSchema": "syntax = \"proto3\"; message ValueSchema1 { int32 C0 = 1; } message ValueSchema2 { int32 C1 = 1; }", + "valueFormat": "PROTOBUF", + "keySchema": "syntax = \"proto3\"; message KeySchema1 { int32 K0 = 1; } message KeySchema2 { int32 K1 = 1; }", + "keyFormat": "PROTOBUF" + }, + { + "name": "OUTPUT", + "valueFormat": "PROTOBUF", + "keyFormat": "PROTOBUF" + } + ], + "inputs": [{"topic": "input", "key": {"k1": 42}, "value": {"c1": 4}}], + "outputs": [{"topic": "OUTPUT", "key": {"k1": 42}, "value": {"c1": 4}}], + "post": { + "sources": [ + {"name": "OUTPUT", "type": "stream", "schema": "`K1` INT KEY, `C1` INT"} + ] + } + }, { "name": "validate without value elements OK - JSON_SR SCHEMA", "statements": [ diff --git a/ksqldb-functional-tests/src/test/resources/query-validation-tests/histogram.json b/ksqldb-functional-tests/src/test/resources/query-validation-tests/histogram.json index bf3dc4c4139c..6087721ba3fa 100644 --- a/ksqldb-functional-tests/src/test/resources/query-validation-tests/histogram.json +++ b/ksqldb-functional-tests/src/test/resources/query-validation-tests/histogram.json @@ -10,7 +10,7 @@ "tests": [ { "name": "histogram string", - "format": ["AVRO", "JSON", "PROTOBUF"], + "format": ["AVRO", "JSON", "PROTOBUF", "PROTOBUF_NOSR"], "statements": [ "CREATE STREAM TEST (ID BIGINT KEY, VALUE varchar) WITH (kafka_topic='test_topic', value_format='{FORMAT}');", "CREATE TABLE S2 as SELECT ID, histogram(value) as counts FROM test group by id;" @@ -32,7 +32,7 @@ }, { "name": "histogram on a table", - "format": ["AVRO","JSON", "PROTOBUF"], + "format": ["AVRO","JSON", "PROTOBUF", "PROTOBUF_NOSR"], "statements": [ "CREATE TABLE TEST (K STRING PRIMARY KEY, ID bigint, NAME varchar, REGION string) WITH (kafka_topic='test_topic', value_format='{FORMAT}');", "CREATE TABLE COUNT_BY_REGION AS SELECT region, histogram(name) AS COUNTS FROM TEST GROUP BY region;" diff --git a/ksqldb-functional-tests/src/test/resources/query-validation-tests/initcap.json b/ksqldb-functional-tests/src/test/resources/query-validation-tests/initcap.json index 26a6510e0173..3e4b68357aeb 100644 --- a/ksqldb-functional-tests/src/test/resources/query-validation-tests/initcap.json +++ b/ksqldb-functional-tests/src/test/resources/query-validation-tests/initcap.json @@ -5,12 +5,12 @@ "tests": [ { "name": "do initcap", - "format": ["JSON", "PROTOBUF"], + "format": ["JSON", "AVRO"], "properties": { "ksql.functions.substring.legacy.args": false }, "statements": [ - "CREATE STREAM TEST (K STRING KEY, source VARCHAR) WITH (kafka_topic='test_topic', value_format='JSON');", + "CREATE STREAM TEST (K STRING KEY, source VARCHAR) WITH (kafka_topic='test_topic', value_format='{FORMAT}');", "CREATE STREAM OUTPUT AS SELECT K, INITCAP(source) AS INITCAP FROM TEST;" ], "inputs": [ @@ -23,6 +23,27 @@ {"topic": "OUTPUT", "value": {"INITCAP":null}}, {"topic": "OUTPUT", "value": {"INITCAP":"The Quick Br0wn Fox"}} ] + }, + { + "name": "do initcap - PROTOBUFs", + "format": ["PROTOBUF", "PROTOBUF_NOSR"], + "properties": { + "ksql.functions.substring.legacy.args": false + }, + "statements": [ + "CREATE STREAM TEST (K STRING KEY, source VARCHAR) WITH (kafka_topic='test_topic', value_format='{FORMAT}');", + "CREATE STREAM OUTPUT AS SELECT K, INITCAP(source) AS INITCAP FROM TEST;" + ], + "inputs": [ + {"topic": "test_topic", "value": {"source": "some_string"}}, + {"topic": "test_topic", "value": {"source": null}}, + {"topic": "test_topic", "value": {"source": "the Quick br0wn fOx"}} + ], + "outputs": [ + {"topic": "OUTPUT", "value": {"INITCAP":"Some_string"}}, + {"topic": "OUTPUT", "value": {"INITCAP":""}}, + {"topic": "OUTPUT", "value": {"INITCAP":"The Quick Br0wn Fox"}} + ] } ] } \ No newline at end of file diff --git a/ksqldb-functional-tests/src/test/resources/query-validation-tests/insert-into.json b/ksqldb-functional-tests/src/test/resources/query-validation-tests/insert-into.json index 43f9e39fd94b..6c02227eefdc 100644 --- a/ksqldb-functional-tests/src/test/resources/query-validation-tests/insert-into.json +++ b/ksqldb-functional-tests/src/test/resources/query-validation-tests/insert-into.json @@ -173,6 +173,76 @@ {"topic": "sink", "key": "0", "value": {"A": 789, "B": "turtle"}, "timestamp": 0} ] }, + { + "name": "convert formats: JSON to PROTOBUF_NOSR", + "statements": [ + "CREATE STREAM SOURCE (K STRING KEY, A bigint, B varchar) WITH (kafka_topic='source', value_format='JSON');", + "CREATE STREAM SINK (K STRING KEY, A bigint, B varchar) WITH (kafka_topic='sink', value_format='PROTOBUF_NOSR');", + "INSERT INTO SINK SELECT * FROM SOURCE;" + ], + "inputs": [ + {"topic": "source", "key": "0", "value": {"A": 123, "B": "falcon"}, "timestamp": 0}, + {"topic": "source", "key": "0", "value": {"A": 456, "B": "giraffe"}, "timestamp": 0}, + {"topic": "source", "key": "0", "value": {"A": 789, "B": "turtle"}, "timestamp": 0} + ], + "outputs": [ + {"topic": "sink", "key": "0", "value": {"A": 123, "B": "falcon"}, "timestamp": 0}, + {"topic": "sink", "key": "0", "value": {"A": 456, "B": "giraffe"}, "timestamp": 0}, + {"topic": "sink", "key": "0", "value": {"A": 789, "B": "turtle"}, "timestamp": 0} + ] + }, + { + "name": "convert formats: PROTOBUF_NOSR to JSON", + "statements": [ + "CREATE STREAM SOURCE (K STRING KEY, A bigint, B varchar) WITH (kafka_topic='source', value_format='PROTOBUF_NOSR');", + "CREATE STREAM SINK (K STRING KEY, A bigint, B varchar) WITH (kafka_topic='sink', value_format='JSON');", + "INSERT INTO SINK SELECT * FROM SOURCE;" + ], + "inputs": [ + {"topic": "source", "key": "0", "value": {"A": 123, "B": "falcon"}, "timestamp": 0}, + {"topic": "source", "key": "0", "value": {"A": 456, "B": "giraffe"}, "timestamp": 0}, + {"topic": "source", "key": "0", "value": {"A": 789, "B": "turtle"}, "timestamp": 0} + ], + "outputs": [ + {"topic": "sink", "key": "0", "value": {"A": 123, "B": "falcon"}, "timestamp": 0}, + {"topic": "sink", "key": "0", "value": {"A": 456, "B": "giraffe"}, "timestamp": 0}, + {"topic": "sink", "key": "0", "value": {"A": 789, "B": "turtle"}, "timestamp": 0} + ] + }, + { + "name": "INSERT INTO stream with SCHEMA_ID and SCHEMA_FULL_NAME", + "statements": [ + "CREATE STREAM SOURCE WITH (kafka_topic='source', format='PROTOBUF', KEY_SCHEMA_ID=1, KEY_SCHEMA_FULL_NAME='ProtobufKey2', VALUE_SCHEMA_ID=2, VALUE_SCHEMA_FULL_NAME='ProtobufValue2');", + "CREATE STREAM SINK WITH (kafka_topic='sink', format='PROTOBUF', KEY_SCHEMA_FULL_NAME='ProtobufKey2', VALUE_SCHEMA_FULL_NAME='ProtobufValue2');", + "INSERT INTO SINK SELECT * FROM SOURCE;" + ], + "topics": [ + { + "name": "source", + "keyFormat": "PROTOBUF", + "keySchema": "syntax = \"proto3\"; message ProtobufKey1 {uint32 k1 = 1;} message ProtobufKey2 {string K = 1;}", + "valueFormat": "PROTOBUF", + "valueSchema": "syntax = \"proto3\"; message ProtobufValue1 {float c1 = 1; uint32 c2 = 2;} message ProtobufValue2 {uint64 A = 1; string B = 2;}" + }, + { + "name": "sink", + "keyFormat": "PROTOBUF", + "keySchema": "syntax = \"proto3\"; message ProtobufKey1 {uint32 k1 = 1;} message ProtobufKey2 {string K = 1;}", + "valueFormat": "PROTOBUF", + "valueSchema": "syntax = \"proto3\"; message ProtobufValue1 {float c1 = 1; uint32 c2 = 2;} message ProtobufValue2 {uint64 A = 1; string B = 2;}" + } + ], + "inputs": [ + {"topic": "source", "key": {"K": "0"}, "value": {"A": 123, "B": "falcon"}, "timestamp": 0}, + {"topic": "source", "key": {"K": "0"}, "value": {"A": 456, "B": "giraffe"}, "timestamp": 0}, + {"topic": "source", "key": {"K": "0"}, "value": {"A": 789, "B": "turtle"}, "timestamp": 0} + ], + "outputs": [ + {"topic": "sink", "key": {"K": "0"}, "value": {"A": 123, "B": "falcon"}, "timestamp": 0}, + {"topic": "sink", "key": {"K": "0"}, "value": {"A": 456, "B": "giraffe"}, "timestamp": 0}, + {"topic": "sink", "key": {"K": "0"}, "value": {"A": 789, "B": "turtle"}, "timestamp": 0} + ] + }, { "name": "join", "statements": [ diff --git a/ksqldb-functional-tests/src/test/resources/query-validation-tests/join-with-custom-timestamp.json b/ksqldb-functional-tests/src/test/resources/query-validation-tests/join-with-custom-timestamp.json index 2f6c45c79faf..19586dbad998 100644 --- a/ksqldb-functional-tests/src/test/resources/query-validation-tests/join-with-custom-timestamp.json +++ b/ksqldb-functional-tests/src/test/resources/query-validation-tests/join-with-custom-timestamp.json @@ -10,7 +10,7 @@ "tests": [ { "name": "stream stream inner join with ts", - "format": ["AVRO", "JSON", "PROTOBUF"], + "format": ["AVRO", "JSON", "PROTOBUF", "PROTOBUF_NOSR"], "statements": [ "CREATE STREAM S1 (ID BIGINT KEY, NAME varchar, TS bigint) WITH (timestamp='TS', kafka_topic='s1', value_format='{FORMAT}');", "CREATE STREAM S2 (ID BIGINT KEY, F1 varchar, F2 varchar) WITH (kafka_topic='s2', value_format='{FORMAT}');", @@ -31,7 +31,7 @@ }, { "name": "stream stream inner join with ts extractor both sides", - "format": ["AVRO", "JSON", "PROTOBUF"], + "format": ["AVRO", "JSON", "PROTOBUF", "PROTOBUF_NOSR"], "statements": [ "CREATE STREAM S1 (ID BIGINT KEY, NAME varchar, TS bigint) WITH (timestamp='TS', kafka_topic='s1', value_format='{FORMAT}');", "CREATE STREAM S2 (ID BIGINT KEY, F1 varchar, F2 varchar, RTS bigint) WITH (timestamp='RTS', kafka_topic='s2', value_format='{FORMAT}');", @@ -52,7 +52,7 @@ }, { "name": "stream table join with ts extractor both sides", - "format": ["AVRO", "JSON", "PROTOBUF"], + "format": ["AVRO", "JSON", "PROTOBUF", "PROTOBUF_NOSR"], "statements": [ "CREATE STREAM S1 (ID BIGINT KEY, NAME varchar, TS bigint) WITH (timestamp='TS', kafka_topic='s1', value_format='{FORMAT}');", "CREATE TABLE T1 (ID BIGINT PRIMARY KEY, F1 varchar, F2 varchar, RTS bigint) WITH (timestamp='RTS', kafka_topic='t1', value_format='{FORMAT}');", @@ -94,7 +94,7 @@ }, { "name": "table table inner join with ts extractor both sides", - "format": ["AVRO", "JSON", "PROTOBUF"], + "format": ["AVRO", "JSON", "PROTOBUF", "PROTOBUF_NOSR"], "statements": [ "CREATE TABLE S1 (ID BIGINT PRIMARY KEY, NAME varchar, TS bigint) WITH (timestamp='TS', kafka_topic='s1', value_format='{FORMAT}');", "CREATE TABLE S2 (ID BIGINT PRIMARY KEY, F1 varchar, F2 varchar, RTS bigint) WITH (timestamp='RTS', kafka_topic='s2', value_format='{FORMAT}');", diff --git a/ksqldb-functional-tests/src/test/resources/query-validation-tests/joins.json b/ksqldb-functional-tests/src/test/resources/query-validation-tests/joins.json index 521ab003ebd9..c2fd7640572d 100644 --- a/ksqldb-functional-tests/src/test/resources/query-validation-tests/joins.json +++ b/ksqldb-functional-tests/src/test/resources/query-validation-tests/joins.json @@ -470,7 +470,7 @@ }, { "name": "stream stream left join - PROTOBUF", - "format": ["PROTOBUF"], + "format": ["PROTOBUF", "PROTOBUF_NOSR"], "statements": [ "CREATE STREAM TEST (ID BIGINT KEY, NAME varchar, VALUE bigint) WITH (kafka_topic='left_topic', value_format='{FORMAT}');", "CREATE STREAM TEST_STREAM (ID BIGINT KEY, F1 varchar, F2 bigint) WITH (kafka_topic='right_topic', value_format='{FORMAT}');", @@ -551,7 +551,7 @@ }, { "name": "stream stream left join with key in projection - rekey", - "format": ["PROTOBUF"], + "format": ["PROTOBUF", "PROTOBUF_NOSR"], "statements": [ "CREATE STREAM TEST (K STRING KEY, ID bigint, NAME varchar, VALUE bigint) WITH (kafka_topic='left_topic', value_format='{FORMAT}');", "CREATE STREAM TEST_STREAM (K STRING KEY, ID bigint, F1 varchar, F2 bigint) WITH (kafka_topic='right_topic', value_format='{FORMAT}');", @@ -617,7 +617,7 @@ }, { "name": "stream stream left join - rekey", - "format": ["PROTOBUF"], + "format": ["PROTOBUF", "PROTOBUF_NOSR"], "statements": [ "CREATE STREAM TEST (K STRING KEY, ID bigint, NAME varchar, VALUE bigint) WITH (kafka_topic='left_topic', value_format='{FORMAT}');", "CREATE STREAM TEST_STREAM (K STRING KEY, ID bigint, F1 varchar, F2 bigint) WITH (kafka_topic='right_topic', value_format='{FORMAT}');", @@ -715,7 +715,7 @@ }, { "name": "stream stream right join - PROTOBUF", - "format": ["PROTOBUF"], + "format": ["PROTOBUF", "PROTOBUF_NOSR"], "statements": [ "CREATE STREAM TEST (id BIGINT KEY, name VARCHAR, value BIGINT) WITH (kafka_topic='left_topic', value_format='{FORMAT}');", "CREATE STREAM TEST_STREAM (id BIGINT KEY, f1 VARCHAR, f2 BIGINT) WITH (kafka_topic='right_topic', value_format='{FORMAT}');", @@ -790,7 +790,7 @@ }, { "name": "stream stream right join with key in projection - rekey", - "format": ["PROTOBUF"], + "format": ["PROTOBUF", "PROTOBUF_NOSR"], "statements": [ "CREATE STREAM TEST (K STRING KEY, ID bigint, NAME varchar, VALUE bigint) WITH (kafka_topic='left_topic', value_format='{FORMAT}');", "CREATE STREAM TEST_STREAM (K STRING KEY, ID bigint, F1 varchar, F2 bigint) WITH (kafka_topic='right_topic', value_format='{FORMAT}');", @@ -850,7 +850,7 @@ }, { "name": "stream stream right join - rekey", - "format": ["PROTOBUF"], + "format": ["PROTOBUF", "PROTOBUF_NOSR"], "statements": [ "CREATE STREAM TEST (K STRING KEY, ID bigint, NAME varchar, VALUE bigint) WITH (kafka_topic='left_topic', value_format='{FORMAT}');", "CREATE STREAM TEST_STREAM (K STRING KEY, ID bigint, F1 varchar, F2 bigint) WITH (kafka_topic='right_topic', value_format='{FORMAT}');", @@ -880,7 +880,7 @@ }, { "name": "stream stream inner join", - "format": ["AVRO", "JSON", "PROTOBUF"], + "format": ["AVRO", "JSON", "PROTOBUF", "PROTOBUF_NOSR"], "statements": [ "CREATE STREAM TEST (ID BIGINT KEY, NAME varchar, VALUE bigint) WITH (kafka_topic='left_topic', value_format='{FORMAT}');", "CREATE STREAM TEST_STREAM (ID BIGINT KEY, F1 varchar, F2 bigint) WITH (kafka_topic='right_topic', value_format='{FORMAT}');", @@ -912,7 +912,7 @@ }, { "name": "stream stream inner join all left fields some right", - "format": ["AVRO", "JSON", "PROTOBUF"], + "format": ["AVRO", "JSON", "PROTOBUF", "PROTOBUF_NOSR"], "statements": [ "CREATE STREAM TEST (ID BIGINT KEY, NAME varchar, VALUE bigint) WITH (kafka_topic='left_topic', value_format='{FORMAT}');", "CREATE STREAM TEST_STREAM (ID BIGINT KEY, F1 varchar, F2 bigint) WITH (kafka_topic='right_topic', value_format='{FORMAT}');", @@ -941,7 +941,7 @@ }, { "name": "stream stream inner join all right fields some left", - "format": ["AVRO", "JSON", "PROTOBUF"], + "format": ["AVRO", "JSON", "PROTOBUF", "PROTOBUF_NOSR"], "statements": [ "CREATE STREAM TEST (ID BIGINT KEY, NAME varchar, VALUE bigint) WITH (kafka_topic='left_topic', value_format='{FORMAT}');", "CREATE STREAM TEST_STREAM (ID BIGINT KEY, F1 varchar, F2 bigint) WITH (kafka_topic='right_topic', value_format='{FORMAT}');", @@ -998,7 +998,7 @@ }, { "name": "stream stream inner join all fields", - "format": ["AVRO", "JSON", "PROTOBUF"], + "format": ["AVRO", "JSON", "PROTOBUF", "PROTOBUF_NOSR"], "statements": [ "CREATE STREAM TEST (ID BIGINT KEY, NAME varchar) WITH (kafka_topic='left_topic', value_format='{FORMAT}');", "CREATE STREAM TEST_STREAM (ID BIGINT KEY, F1 varchar) WITH (kafka_topic='right_topic', value_format='{FORMAT}');", @@ -1027,7 +1027,7 @@ }, { "name": "stream stream inner join with different before and after windows", - "format": ["AVRO", "JSON", "PROTOBUF"], + "format": ["AVRO", "JSON", "PROTOBUF", "PROTOBUF_NOSR"], "statements": [ "CREATE STREAM TEST (ID BIGINT KEY, NAME varchar, VALUE bigint) WITH (kafka_topic='left_topic', value_format='{FORMAT}');", "CREATE STREAM TEST_STREAM (ID BIGINT KEY, F1 varchar, F2 bigint) WITH (kafka_topic='right_topic', value_format='{FORMAT}');", @@ -1055,7 +1055,7 @@ }, { "name": "stream stream inner join with out of order messages", - "format": ["AVRO", "JSON", "PROTOBUF"], + "format": ["AVRO", "JSON", "PROTOBUF", "PROTOBUF_NOSR"], "statements": [ "CREATE STREAM TEST (ID BIGINT KEY, NAME varchar, VALUE bigint) WITH (kafka_topic='left_topic', value_format='{FORMAT}');", "CREATE STREAM TEST_STREAM (ID BIGINT KEY, F1 varchar, F2 bigint) WITH (kafka_topic='right_topic', value_format='{FORMAT}');", @@ -1087,7 +1087,7 @@ }, { "name": "stream stream inner join with out of order and custom grace period", - "format": ["AVRO", "JSON", "PROTOBUF"], + "format": ["AVRO", "JSON", "PROTOBUF", "PROTOBUF_NOSR"], "statements": [ "CREATE STREAM LEFT_STREAM (id BIGINT KEY, l1 VARCHAR) WITH (kafka_topic='left_topic', value_format='{FORMAT}');", "CREATE STREAM RIGHT_STREAM (id BIGINT KEY, l2 VARCHAR) WITH (kafka_topic='right_topic', value_format='{FORMAT}');", @@ -1220,9 +1220,10 @@ }, { "name": "stream stream outer join - PROTOBUF", + "format": ["PROTOBUF", "PROTOBUF_NOSR"], "statements": [ - "CREATE STREAM TEST (ID BIGINT KEY, NAME varchar, VALUE bigint) WITH (kafka_topic='left_topic', value_format='PROTOBUF');", - "CREATE STREAM TEST_STREAM (ID BIGINT KEY, F1 varchar, F2 bigint) WITH (kafka_topic='right_topic', value_format='PROTOBUF');", + "CREATE STREAM TEST (ID BIGINT KEY, NAME varchar, VALUE bigint) WITH (kafka_topic='left_topic', value_format='{FORMAT}');", + "CREATE STREAM TEST_STREAM (ID BIGINT KEY, F1 varchar, F2 bigint) WITH (kafka_topic='right_topic', value_format='{FORMAT}');", "CREATE STREAM OUTPUT as SELECT ROWKEY as ID, name, value, f1, f2 FROM test t FULL OUTER join TEST_STREAM tt WITHIN 11 seconds on t.id = tt.id;" ], "inputs": [ @@ -1292,9 +1293,10 @@ }, { "name": "table table left join - PROTOBUF", + "format": ["PROTOBUF", "PROTOBUF_NOSR"], "statements": [ - "CREATE TABLE TEST (id BIGINT PRIMARY KEY, name VARCHAR, value BIGINT) WITH (kafka_topic='left_topic', value_format='PROTOBUF');", - "CREATE TABLE TEST_TABLE (id BIGINT PRIMARY KEY, f1 VARCHAR, f2 BIGINT) WITH (kafka_topic='right_topic', value_format='PROTOBUF');", + "CREATE TABLE TEST (id BIGINT PRIMARY KEY, name VARCHAR, value BIGINT) WITH (kafka_topic='left_topic', value_format='{FORMAT}');", + "CREATE TABLE TEST_TABLE (id BIGINT PRIMARY KEY, f1 VARCHAR, f2 BIGINT) WITH (kafka_topic='right_topic', value_format='{FORMAT}');", "CREATE TABLE OUTPUT as SELECT t.id, name, value, f1, f2 FROM test t left join TEST_TABLE tt on t.id = tt.id;" ], "inputs": [ @@ -1360,9 +1362,10 @@ }, { "name": "table table right join - PROTOBUF", + "format": ["PROTOBUF", "PROTOBUF_NOSR"], "statements": [ - "CREATE TABLE TEST (id BIGINT PRIMARY KEY, name VARCHAR, value BIGINT) WITH (kafka_topic='left_topic', value_format='PROTOBUF');", - "CREATE TABLE TEST_TABLE (id BIGINT PRIMARY KEY, f1 VARCHAR, f2 BIGINT) WITH (kafka_topic='right_topic', value_format='PROTOBUF');", + "CREATE TABLE TEST (id BIGINT PRIMARY KEY, name VARCHAR, value BIGINT) WITH (kafka_topic='left_topic', value_format='{FORMAT}');", + "CREATE TABLE TEST_TABLE (id BIGINT PRIMARY KEY, f1 VARCHAR, f2 BIGINT) WITH (kafka_topic='right_topic', value_format='{FORMAT}');", "CREATE TABLE OUTPUT AS SELECT t.id, name, value, f1, f2 FROM test t RIGHT JOIN test_table tt ON t.id = tt.id;" ], "inputs": [ @@ -1424,7 +1427,7 @@ }, { "name": "table table inner join", - "format": ["AVRO", "JSON", "PROTOBUF"], + "format": ["AVRO", "JSON", "PROTOBUF", "PROTOBUF_NOSR"], "statements": [ "CREATE TABLE TEST (id BIGINT PRIMARY KEY, name VARCHAR, value BIGINT) WITH (kafka_topic='left_topic', value_format='{FORMAT}');", "CREATE TABLE TEST_TABLE (id BIGINT PRIMARY KEY, f1 VARCHAR, f2 BIGINT) WITH (kafka_topic='right_topic', value_format='{FORMAT}');", @@ -1492,9 +1495,10 @@ }, { "name": "table table outer join - PROTOBUF", + "format": ["PROTOBUF", "PROTOBUF_NOSR"], "statements": [ - "CREATE TABLE TEST (id BIGINT PRIMARY KEY, name VARCHAR, value BIGINT) WITH (kafka_topic='left_topic', value_format='PROTOBUF');", - "CREATE TABLE TEST_TABLE (id BIGINT PRIMARY KEY, f1 VARCHAR, f2 BIGINT) WITH (kafka_topic='right_topic', value_format='PROTOBUF');", + "CREATE TABLE TEST (id BIGINT PRIMARY KEY, name VARCHAR, value BIGINT) WITH (kafka_topic='left_topic', value_format='{FORMAT}');", + "CREATE TABLE TEST_TABLE (id BIGINT PRIMARY KEY, f1 VARCHAR, f2 BIGINT) WITH (kafka_topic='right_topic', value_format='{FORMAT}');", "CREATE TABLE OUTER_JOIN as SELECT ROWKEY AS ID, t.id, tt.id, name, value, f1, f2 FROM test t FULL OUTER join TEST_TABLE tt on t.id = tt.id;" ], "inputs": [ @@ -1558,9 +1562,10 @@ }, { "name": "stream table left join - PROTOBUF", + "format": ["PROTOBUF", "PROTOBUF_NOSR"], "statements": [ - "CREATE STREAM TEST (ID BIGINT KEY, NAME varchar, VALUE bigint) WITH (kafka_topic='test_topic', value_format='PROTOBUF');", - "CREATE TABLE TEST_TABLE (id BIGINT PRIMARY KEY, f1 VARCHAR, f2 BIGINT) WITH (kafka_topic='test_table', value_format='PROTOBUF');", + "CREATE STREAM TEST (ID BIGINT KEY, NAME varchar, VALUE bigint) WITH (kafka_topic='test_topic', value_format='{FORMAT}');", + "CREATE TABLE TEST_TABLE (id BIGINT PRIMARY KEY, f1 VARCHAR, f2 BIGINT) WITH (kafka_topic='test_table', value_format='{FORMAT}');", "CREATE STREAM LEFT_JOIN as SELECT t.id, name, value, f1, f2 FROM test t left join test_table tt on t.id = tt.id;" ], "inputs": [ @@ -1589,7 +1594,7 @@ }, { "name": "stream table inner join", - "format": ["AVRO", "JSON", "PROTOBUF"], + "format": ["AVRO", "JSON", "PROTOBUF", "PROTOBUF_NOSR"], "statements": [ "CREATE STREAM TEST (ID BIGINT KEY, NAME varchar, VALUE bigint) WITH (kafka_topic='test_topic', value_format='{FORMAT}');", "CREATE TABLE TEST_TABLE (id BIGINT PRIMARY KEY, f1 VARCHAR, f2 BIGINT) WITH (kafka_topic='test_table', value_format='{FORMAT}');", @@ -2232,7 +2237,7 @@ }, { "name": "on INT column - KAFKA", - "format": ["AVRO", "JSON", "PROTOBUF"], + "format": ["AVRO", "JSON", "PROTOBUF", "PROTOBUF_NOSR"], "statements": [ "CREATE STREAM L (ID STRING KEY, l0 INT, l1 INT) WITH (kafka_topic='left_topic', value_format='{FORMAT}');", "CREATE STREAM R (ID STRING KEY, r0 INT, r1 INT) WITH (kafka_topic='right_topic', value_format='{FORMAT}');", @@ -2253,7 +2258,7 @@ }, { "name": "on BIGINT column - KAFKA", - "format": ["AVRO", "JSON", "PROTOBUF"], + "format": ["AVRO", "JSON", "PROTOBUF", "PROTOBUF_NOSR"], "statements": [ "CREATE STREAM L (ID STRING KEY, l0 BIGINT, l1 INT) WITH (kafka_topic='left_topic', value_format='{FORMAT}');", "CREATE STREAM R (ID STRING KEY, r0 BIGINT, r1 INT) WITH (kafka_topic='right_topic', value_format='{FORMAT}');", @@ -2274,7 +2279,7 @@ }, { "name": "on DOUBLE column = KAFKA", - "format": ["AVRO", "JSON", "PROTOBUF"], + "format": ["AVRO", "JSON", "PROTOBUF", "PROTOBUF_NOSR"], "statements": [ "CREATE STREAM L (ID STRING KEY, l0 DOUBLE, l1 INT) WITH (kafka_topic='left_topic', value_format='{FORMAT}');", "CREATE STREAM R (ID STRING KEY, r0 DOUBLE, r1 INT) WITH (kafka_topic='right_topic', value_format='{FORMAT}');", @@ -2295,7 +2300,7 @@ }, { "name": "on STRING column - KAFKA", - "format": ["AVRO", "JSON", "PROTOBUF"], + "format": ["AVRO", "JSON", "PROTOBUF", "PROTOBUF_NOSR"], "statements": [ "CREATE STREAM L (ID STRING KEY, l0 STRING, l1 INT) WITH (kafka_topic='left_topic', value_format='{FORMAT}');", "CREATE STREAM R (ID STRING KEY, r0 STRING, r1 INT) WITH (kafka_topic='right_topic', value_format='{FORMAT}');", diff --git a/ksqldb-functional-tests/src/test/resources/query-validation-tests/max-group-by.json b/ksqldb-functional-tests/src/test/resources/query-validation-tests/max-group-by.json index e7577b5b3073..f60ef612f492 100644 --- a/ksqldb-functional-tests/src/test/resources/query-validation-tests/max-group-by.json +++ b/ksqldb-functional-tests/src/test/resources/query-validation-tests/max-group-by.json @@ -237,6 +237,74 @@ {"topic": "S2", "key": 0,"value": {"VALUE": 1000}}, {"topic": "S2", "key": 0,"value": {"VALUE": 1000}} ] + }, + { + "name": "max string group by", + "versions": { + "min": "7.3.0" + }, + "statements": [ + "CREATE STREAM TEST (ID BIGINT KEY, VALUE STRING) WITH (kafka_topic='test_topic',value_format='AVRO');", + "CREATE TABLE S2 as SELECT ID, max(value) as value FROM test group by id;" + ], + "inputs": [ + {"topic": "test_topic", "key": 1,"value": {"value": null}}, + {"topic": "test_topic", "key": 0,"value": {"value": null}}, + {"topic": "test_topic", "key": 0,"value": {"value": "B"}}, + {"topic": "test_topic", "key": 0,"value": {"value": "A"}}, + {"topic": "test_topic", "key": 0,"value": {"value": null}}, + {"topic": "test_topic", "key": 100,"value": {"value": "E"}}, + {"topic": "test_topic", "key": 100,"value": {"value": "C"}}, + {"topic": "test_topic", "key": 100,"value": {"value": "G"}}, + {"topic": "test_topic", "key": 0,"value": {"value": "F"}}, + {"topic": "test_topic", "key": 0,"value": {"value": "D"}} + ], + "outputs": [ + {"topic": "S2", "key": 1,"value": {"VALUE": null}}, + {"topic": "S2", "key": 0,"value": {"VALUE": null}}, + {"topic": "S2", "key": 0,"value": {"VALUE": "B"}}, + {"topic": "S2", "key": 0,"value": {"VALUE": "B"}}, + {"topic": "S2", "key": 0,"value": {"VALUE": "B"}}, + {"topic": "S2", "key": 100,"value": {"VALUE": "E"}}, + {"topic": "S2", "key": 100,"value": {"VALUE": "E"}}, + {"topic": "S2", "key": 100,"value": {"VALUE": "G"}}, + {"topic": "S2", "key": 0,"value": {"VALUE": "F"}}, + {"topic": "S2", "key": 0,"value": {"VALUE": "F"}} + ] + }, + { + "name": "max bytes group by", + "versions": { + "min": "7.3.0" + }, + "statements": [ + "CREATE STREAM TEST (ID BIGINT KEY, VALUE BYTES) WITH (kafka_topic='test_topic',value_format='AVRO');", + "CREATE TABLE S2 as SELECT ID, max(value) as value FROM test group by id;" + ], + "inputs": [ + {"topic": "test_topic", "key": 1,"value": {"value": null}}, + {"topic": "test_topic", "key": 0,"value": {"value": null}}, + {"topic": "test_topic", "key": 0,"value": {"value": "Qg=="}}, + {"topic": "test_topic", "key": 0,"value": {"value": "QQ=="}}, + {"topic": "test_topic", "key": 0,"value": {"value": null}}, + {"topic": "test_topic", "key": 100,"value": {"value": "RQ=="}}, + {"topic": "test_topic", "key": 100,"value": {"value": "Qw=="}}, + {"topic": "test_topic", "key": 100,"value": {"value": "Rw=="}}, + {"topic": "test_topic", "key": 0,"value": {"value": "Rg=="}}, + {"topic": "test_topic", "key": 0,"value": {"value": "RA=="}} + ], + "outputs": [ + {"topic": "S2", "key": 1,"value": {"VALUE": null}}, + {"topic": "S2", "key": 0,"value": {"VALUE": null}}, + {"topic": "S2", "key": 0,"value": {"VALUE": "Qg=="}}, + {"topic": "S2", "key": 0,"value": {"VALUE": "Qg=="}}, + {"topic": "S2", "key": 0,"value": {"VALUE": "Qg=="}}, + {"topic": "S2", "key": 100,"value": {"VALUE": "RQ=="}}, + {"topic": "S2", "key": 100,"value": {"VALUE": "RQ=="}}, + {"topic": "S2", "key": 100,"value": {"VALUE": "Rw=="}}, + {"topic": "S2", "key": 0,"value": {"VALUE": "Rg=="}}, + {"topic": "S2", "key": 0,"value": {"VALUE": "Rg=="}} + ] } ] } \ No newline at end of file diff --git a/ksqldb-functional-tests/src/test/resources/query-validation-tests/min-group-by.json b/ksqldb-functional-tests/src/test/resources/query-validation-tests/min-group-by.json index 280e2cb6c759..13ff2ed6f7d1 100644 --- a/ksqldb-functional-tests/src/test/resources/query-validation-tests/min-group-by.json +++ b/ksqldb-functional-tests/src/test/resources/query-validation-tests/min-group-by.json @@ -221,6 +221,74 @@ {"topic": "S2", "key": 0,"value": {"MIN": 1}}, {"topic": "S2", "key": 0,"value": {"MIN": 1}} ] + }, + { + "name": "min string group by", + "versions": { + "min": "7.3.0" + }, + "statements": [ + "CREATE STREAM TEST (ID BIGINT KEY, VALUE STRING) WITH (kafka_topic='test_topic',value_format='AVRO');", + "CREATE TABLE S2 as SELECT ID, min(value) as value FROM test group by id;" + ], + "inputs": [ + {"topic": "test_topic", "key": 1,"value": {"value": null}}, + {"topic": "test_topic", "key": 0,"value": {"value": null}}, + {"topic": "test_topic", "key": 0,"value": {"value": "B"}}, + {"topic": "test_topic", "key": 0,"value": {"value": "A"}}, + {"topic": "test_topic", "key": 0,"value": {"value": null}}, + {"topic": "test_topic", "key": 100,"value": {"value": "E"}}, + {"topic": "test_topic", "key": 100,"value": {"value": "C"}}, + {"topic": "test_topic", "key": 100,"value": {"value": "G"}}, + {"topic": "test_topic", "key": 0,"value": {"value": "F"}}, + {"topic": "test_topic", "key": 0,"value": {"value": "D"}} + ], + "outputs": [ + {"topic": "S2", "key": 1,"value": {"VALUE": null}}, + {"topic": "S2", "key": 0,"value": {"VALUE": null}}, + {"topic": "S2", "key": 0,"value": {"VALUE": "B"}}, + {"topic": "S2", "key": 0,"value": {"VALUE": "A"}}, + {"topic": "S2", "key": 0,"value": {"VALUE": "A"}}, + {"topic": "S2", "key": 100,"value": {"VALUE": "E"}}, + {"topic": "S2", "key": 100,"value": {"VALUE": "C"}}, + {"topic": "S2", "key": 100,"value": {"VALUE": "C"}}, + {"topic": "S2", "key": 0,"value": {"VALUE": "A"}}, + {"topic": "S2", "key": 0,"value": {"VALUE": "A"}} + ] + }, + { + "name": "min bytes group by", + "versions": { + "min": "7.3.0" + }, + "statements": [ + "CREATE STREAM TEST (ID BIGINT KEY, VALUE BYTES) WITH (kafka_topic='test_topic',value_format='AVRO');", + "CREATE TABLE S2 as SELECT ID, min(value) as value FROM test group by id;" + ], + "inputs": [ + {"topic": "test_topic", "key": 1,"value": {"value": null}}, + {"topic": "test_topic", "key": 0,"value": {"value": null}}, + {"topic": "test_topic", "key": 0,"value": {"value": "Qg=="}}, + {"topic": "test_topic", "key": 0,"value": {"value": "QQ=="}}, + {"topic": "test_topic", "key": 0,"value": {"value": null}}, + {"topic": "test_topic", "key": 100,"value": {"value": "RQ=="}}, + {"topic": "test_topic", "key": 100,"value": {"value": "Qw=="}}, + {"topic": "test_topic", "key": 100,"value": {"value": "Rw=="}}, + {"topic": "test_topic", "key": 0,"value": {"value": "Rg=="}}, + {"topic": "test_topic", "key": 0,"value": {"value": "RA=="}} + ], + "outputs": [ + {"topic": "S2", "key": 1,"value": {"VALUE": null}}, + {"topic": "S2", "key": 0,"value": {"VALUE": null}}, + {"topic": "S2", "key": 0,"value": {"VALUE": "Qg=="}}, + {"topic": "S2", "key": 0,"value": {"VALUE": "QQ=="}}, + {"topic": "S2", "key": 0,"value": {"VALUE": "QQ=="}}, + {"topic": "S2", "key": 100,"value": {"VALUE": "RQ=="}}, + {"topic": "S2", "key": 100,"value": {"VALUE": "Qw=="}}, + {"topic": "S2", "key": 100,"value": {"VALUE": "Qw=="}}, + {"topic": "S2", "key": 0,"value": {"VALUE": "QQ=="}}, + {"topic": "S2", "key": 0,"value": {"VALUE": "QQ=="}} + ] } ] } \ No newline at end of file diff --git a/ksqldb-functional-tests/src/test/resources/query-validation-tests/sum.json b/ksqldb-functional-tests/src/test/resources/query-validation-tests/sum.json index 8794b6398ea7..18a752d5e455 100644 --- a/ksqldb-functional-tests/src/test/resources/query-validation-tests/sum.json +++ b/ksqldb-functional-tests/src/test/resources/query-validation-tests/sum.json @@ -108,6 +108,73 @@ {"topic": "S2", "key": 100,"value": {"SUM": "801.7"}} ] }, + { + "name": "sum decimal with overflow", + "statements": [ + "CREATE STREAM TEST (ID BIGINT KEY, VALUE decimal(4,1)) WITH (kafka_topic='test_topic', value_format='AVRO');", + "CREATE TABLE S2 as SELECT ID, sum(value) AS SUM FROM test group by id;" + ], + "inputs": [ + {"topic": "test_topic", "key": 0,"value": {"value": "5.4"}}, + {"topic": "test_topic", "key": 0,"value": {"value": "100.1"}}, + {"topic": "test_topic", "key": 100,"value": {"value": "500.9"}}, + {"topic": "test_topic", "key": 100,"value": {"value": "300.8"}}, + {"topic": "test_topic", "key": 100,"value": {"value": "400.8"}} + ], + "outputs": [ + {"topic": "S2", "key": 0,"value": {"SUM": "005.4"}}, + {"topic": "S2", "key": 0,"value": {"SUM": "105.5"}}, + {"topic": "S2", "key": 100,"value": {"SUM": "500.9"}}, + {"topic": "S2", "key": 100,"value": {"SUM": "801.7"}} + ], + "expectedException": { + "type": "io.confluent.ksql.function.KsqlFunctionException", + "message": "Numeric field overflow" + } + }, + { + "name": "sum decimal should not overflow with cast to higher precision", + "statements": [ + "CREATE STREAM TEST (ID BIGINT KEY, VALUE decimal(4,1)) WITH (kafka_topic='test_topic', value_format='AVRO');", + "CREATE TABLE S2 as SELECT ID, sum(CAST(value AS DECIMAL(6,2))) AS SUM FROM test group by id;" + ], + "inputs": [ + {"topic": "test_topic", "key": 0,"value": {"value": "5.4"}}, + {"topic": "test_topic", "key": 0,"value": {"value": "100.1"}}, + {"topic": "test_topic", "key": 100,"value": {"value": "500.9"}}, + {"topic": "test_topic", "key": 100,"value": {"value": "300.8"}}, + {"topic": "test_topic", "key": 100,"value": {"value": "400.8"}} + ], + "outputs": [ + {"topic": "S2", "key": 0,"value": {"SUM": "5.40"}}, + {"topic": "S2", "key": 0,"value": {"SUM": "105.50"}}, + {"topic": "S2", "key": 100,"value": {"SUM": "500.90"}}, + {"topic": "S2", "key": 100,"value": {"SUM": "801.70"}}, + {"topic": "S2", "key": 100,"value": {"SUM": "1202.50"}} + ] + }, + { + "name": "sum decimal should overflow with cast to lower precision", + "statements": [ + "CREATE STREAM TEST (ID BIGINT KEY, VALUE decimal(4,1)) WITH (kafka_topic='test_topic', value_format='AVRO');", + "CREATE TABLE S2 as SELECT ID, sum(CAST(value as DECIMAL(3,1))) AS SUM FROM test group by id;" + ], + "inputs": [ + {"topic": "test_topic", "key": 0,"value": {"value": "5.4"}}, + {"topic": "test_topic", "key": 0,"value": {"value": "10.1"}}, + {"topic": "test_topic", "key": 100,"value": {"value": "50.9"}}, + {"topic": "test_topic", "key": 100,"value": {"value": "50.8"}} + ], + "outputs": [ + {"topic": "S2", "key": 0,"value": {"SUM": "5.4"}}, + {"topic": "S2", "key": 0,"value": {"SUM": "15.5"}}, + {"topic": "S2", "key": 100,"value": {"SUM": "50.9"}} + ], + "expectedException": { + "type": "io.confluent.ksql.function.KsqlFunctionException", + "message": "Numeric field overflow" + } + }, { "name": "sum double map", "statements": [ diff --git a/ksqldb-functional-tests/src/test/resources/query-validation-tests/topk-group-by.json b/ksqldb-functional-tests/src/test/resources/query-validation-tests/topk-group-by.json index 27442707801a..d813df090c8d 100644 --- a/ksqldb-functional-tests/src/test/resources/query-validation-tests/topk-group-by.json +++ b/ksqldb-functional-tests/src/test/resources/query-validation-tests/topk-group-by.json @@ -10,7 +10,7 @@ "tests": [ { "name": "topk integer", - "format": ["AVRO", "JSON", "PROTOBUF"], + "format": ["AVRO", "JSON", "PROTOBUF", "PROTOBUF_NOSR"], "statements": [ "CREATE STREAM TEST (ID BIGINT KEY, NAME varchar, VALUE integer) WITH (kafka_topic='test_topic',value_format='{FORMAT}');", "CREATE TABLE S2 as SELECT ID, topk(value, 3) as topk FROM test group by id;" @@ -32,7 +32,7 @@ }, { "name": "topk long", - "format": ["AVRO", "JSON", "PROTOBUF"], + "format": ["AVRO", "JSON", "PROTOBUF", "PROTOBUF_NOSR"], "statements": [ "CREATE STREAM TEST (ID BIGINT KEY, NAME varchar, VALUE bigint) WITH (kafka_topic='test_topic', value_format='{FORMAT}');", "CREATE TABLE S2 as SELECT ID, topk(value, 3) as topk FROM test group by id;" @@ -54,7 +54,7 @@ }, { "name": "topk double", - "format": ["AVRO", "JSON", "PROTOBUF"], + "format": ["AVRO", "JSON", "PROTOBUF", "PROTOBUF_NOSR"], "statements": [ "CREATE STREAM TEST (ID BIGINT KEY, NAME varchar, VALUE double) WITH (kafka_topic='test_topic', value_format='{FORMAT}');", "CREATE TABLE S2 as SELECT ID, topk(value, 3) as topk FROM test group by id;" @@ -76,7 +76,7 @@ }, { "name": "topk string", - "format": ["AVRO", "JSON", "PROTOBUF"], + "format": ["AVRO", "JSON", "PROTOBUF", "PROTOBUF_NOSR"], "statements": [ "CREATE STREAM TEST (ID BIGINT KEY, NAME varchar, VALUE string) WITH (kafka_topic='test_topic', value_format='{FORMAT}');", "CREATE TABLE S2 as SELECT ID, topk(value, 3) as topk FROM test group by id;" diff --git a/ksqldb-functional-tests/src/test/resources/query-validation-tests/url.json b/ksqldb-functional-tests/src/test/resources/query-validation-tests/url.json index d955536a31a0..27e6321159f7 100644 --- a/ksqldb-functional-tests/src/test/resources/query-validation-tests/url.json +++ b/ksqldb-functional-tests/src/test/resources/query-validation-tests/url.json @@ -6,9 +6,9 @@ "tests": [ { "name": "encode a url parameter using ENCODE_URL_PARAM", - "format": ["JSON", "PROTOBUF"], + "format": ["JSON", "PROTOBUF", "PROTOBUF_NOSR"], "statements": [ - "CREATE STREAM TEST (K STRING KEY, url VARCHAR) WITH (kafka_topic='test_topic', value_format='JSON');", + "CREATE STREAM TEST (K STRING KEY, url VARCHAR) WITH (kafka_topic='test_topic', value_format='{FORMAT}');", "CREATE STREAM OUTPUT AS SELECT K, URL_ENCODE_PARAM(url) as ENCODED FROM TEST;" ], "inputs": [ @@ -24,9 +24,9 @@ }, { "name": "decode a url parameter using DECODE_URL_PARAM", - "format": ["JSON", "PROTOBUF"], + "format": ["JSON", "PROTOBUF", "PROTOBUF_NOSR"], "statements": [ - "CREATE STREAM TEST (K STRING KEY, url VARCHAR) WITH (kafka_topic='test_topic', value_format='JSON');", + "CREATE STREAM TEST (K STRING KEY, url VARCHAR) WITH (kafka_topic='test_topic', value_format='{FORMAT}');", "CREATE STREAM OUTPUT AS SELECT K, URL_DECODE_PARAM(url) as DECODED FROM TEST;" ], "inputs": [ @@ -42,9 +42,9 @@ }, { "name": "extract a fragment from a URL using URL_EXTRACT_FRAGMENT", - "format": ["JSON", "PROTOBUF"], + "format": ["JSON", "AVRO"], "statements": [ - "CREATE STREAM TEST (K STRING KEY, url VARCHAR) WITH (kafka_topic='test_topic', value_format='JSON');", + "CREATE STREAM TEST (K STRING KEY, url VARCHAR) WITH (kafka_topic='test_topic', value_format='{FORMAT}');", "CREATE STREAM OUTPUT AS SELECT K, URL_EXTRACT_FRAGMENT(url) as FRAGMENT FROM TEST;" ], "inputs": [ @@ -56,11 +56,27 @@ {"topic": "OUTPUT", "key": "2", "value": {"FRAGMENT": null}, "timestamp": 1} ] }, + { + "name": "extract a fragment from a URL using URL_EXTRACT_FRAGMENT - PROTOBUFs", + "format": ["PROTOBUF", "PROTOBUF_NOSR"], + "statements": [ + "CREATE STREAM TEST (K STRING KEY, url VARCHAR) WITH (kafka_topic='test_topic', value_format='{FORMAT}');", + "CREATE STREAM OUTPUT AS SELECT K, URL_EXTRACT_FRAGMENT(url) as FRAGMENT FROM TEST;" + ], + "inputs": [ + {"topic": "test_topic", "key": "1", "value": {"url": "http://www.test.com/?query#fragment"}, "timestamp": 0}, + {"topic": "test_topic", "key": "2", "value": {"url": "http://www.nofragment.com"}, "timestamp": 1} + ], + "outputs": [ + {"topic": "OUTPUT", "key": "1", "value": {"FRAGMENT": "fragment"}, "timestamp": 0}, + {"topic": "OUTPUT", "key": "2", "value": {"FRAGMENT": ""}, "timestamp": 1} + ] + }, { "name": "extract a host from a URL using URL_EXTRACT_HOST", - "format": ["JSON", "PROTOBUF"], + "format": ["JSON", "PROTOBUF", "PROTOBUF_NOSR"], "statements": [ - "CREATE STREAM TEST (K STRING KEY, url VARCHAR) WITH (kafka_topic='test_topic', value_format='JSON');", + "CREATE STREAM TEST (K STRING KEY, url VARCHAR) WITH (kafka_topic='test_topic', value_format='{FORMAT}');", "CREATE STREAM OUTPUT AS SELECT K, URL_EXTRACT_HOST(url) as HOST FROM TEST;" ], "inputs": [ @@ -74,9 +90,9 @@ }, { "name": "extract a parameter from a URL using URL_EXTRACT_PARAMETER", - "format": ["JSON", "PROTOBUF"], + "format": ["JSON", "PROTOBUF", "PROTOBUF_NOSR"], "statements": [ - "CREATE STREAM TEST (K STRING KEY, url VARCHAR) WITH (kafka_topic='test_topic', value_format='JSON');", + "CREATE STREAM TEST (K STRING KEY, url VARCHAR) WITH (kafka_topic='test_topic', value_format='{FORMAT}');", "CREATE STREAM OUTPUT AS SELECT K, URL_EXTRACT_PARAMETER(url,'one') as PARAM_A, URL_EXTRACT_PARAMETER(url,'two') as PARAM_B FROM TEST;" ], "inputs": [ @@ -88,9 +104,9 @@ }, { "name": "chain a call to URL_EXTRACT_PARAMETER with URL_DECODE_PARAM", - "format": ["JSON", "PROTOBUF"], + "format": ["JSON", "PROTOBUF", "PROTOBUF_NOSR"], "statements": [ - "CREATE STREAM TEST (K STRING KEY, url VARCHAR) WITH (kafka_topic='test_topic', value_format='JSON');", + "CREATE STREAM TEST (K STRING KEY, url VARCHAR) WITH (kafka_topic='test_topic', value_format='{FORMAT}');", "CREATE STREAM OUTPUT AS SELECT K, URL_DECODE_PARAM(URL_EXTRACT_PARAMETER(url,'two')) as PARAM FROM TEST;" ], "inputs": [ @@ -102,9 +118,9 @@ }, { "name": "extract a path from a URL using URL_EXTRACT_PATH", - "format": ["JSON", "PROTOBUF"], + "format": ["JSON", "PROTOBUF", "PROTOBUF_NOSR"], "statements": [ - "CREATE STREAM TEST (K STRING KEY, url VARCHAR) WITH (kafka_topic='test_topic', value_format='JSON');", + "CREATE STREAM TEST (K STRING KEY, url VARCHAR) WITH (kafka_topic='test_topic', value_format='{FORMAT}');", "CREATE STREAM OUTPUT AS SELECT K, URL_EXTRACT_PATH(url) as PATH FROM TEST;" ], "inputs": [ @@ -120,9 +136,9 @@ }, { "name": "extract a port from a URL using URL_EXTRACT_PORT", - "format": ["JSON", "PROTOBUF"], + "format": ["JSON", "AVRO"], "statements": [ - "CREATE STREAM TEST (K STRING KEY, url VARCHAR) WITH (kafka_topic='test_topic', value_format='JSON');", + "CREATE STREAM TEST (K STRING KEY, url VARCHAR) WITH (kafka_topic='test_topic', value_format='{FORMAT}');", "CREATE STREAM OUTPUT AS SELECT K, URL_EXTRACT_PORT(url) as PORT FROM TEST;" ], "inputs": [ @@ -134,11 +150,27 @@ {"topic": "OUTPUT", "key": "2", "value": {"PORT": 8080}, "timestamp": 1} ] }, + { + "name": "extract a port from a URL using URL_EXTRACT_PORT - PROTOBUFs", + "format": ["PROTOBUF", "PROTOBUF_NOSR"], + "statements": [ + "CREATE STREAM TEST (K STRING KEY, url VARCHAR) WITH (kafka_topic='test_topic', value_format='{FORMAT}');", + "CREATE STREAM OUTPUT AS SELECT K, URL_EXTRACT_PORT(url) as PORT FROM TEST;" + ], + "inputs": [ + {"topic": "test_topic", "key": "1", "value": {"url": "http://test@confluent.io"}, "timestamp": 0}, + {"topic": "test_topic", "key": "2", "value": {"url": "http://test@confluent.io:8080/nested/path?query&jobs"}, "timestamp": 1} + ], + "outputs": [ + {"topic": "OUTPUT", "key": "1", "value": {"PORT": 0}, "timestamp": 0}, + {"topic": "OUTPUT", "key": "2", "value": {"PORT": 8080}, "timestamp": 1} + ] + }, { "name": "extract a protocol from a URL using URL_EXTRACT_PROTOCOL", - "format": ["JSON", "PROTOBUF"], + "format": ["JSON", "AVRO"], "statements": [ - "CREATE STREAM TEST (K STRING KEY, url VARCHAR) WITH (kafka_topic='test_topic', value_format='JSON');", + "CREATE STREAM TEST (K STRING KEY, url VARCHAR) WITH (kafka_topic='test_topic', value_format='{FORMAT}');", "CREATE STREAM OUTPUT AS SELECT K, URL_EXTRACT_PROTOCOL(url) as PROTOCOL FROM TEST;" ], "inputs": [ @@ -152,11 +184,29 @@ {"topic": "OUTPUT", "key": "3", "value": {"PROTOCOL": null}, "timestamp": 2} ] }, + { + "name": "extract a protocol from a URL using URL_EXTRACT_PROTOCOL - PROTOBUFs", + "format": ["PROTOBUF", "PROTOBUF_NOSR"], + "statements": [ + "CREATE STREAM TEST (K STRING KEY, url VARCHAR) WITH (kafka_topic='test_topic', value_format='{FORMAT}');", + "CREATE STREAM OUTPUT AS SELECT K, URL_EXTRACT_PROTOCOL(url) as PROTOCOL FROM TEST;" + ], + "inputs": [ + {"topic": "test_topic", "key": "1", "value": {"url": "http://test@confluent.io"}, "timestamp": 0}, + {"topic": "test_topic", "key": "2", "value": {"url": "https://test@confluent.io:8080/nested/path?query&jobs"}, "timestamp": 1}, + {"topic": "test_topic", "key": "3", "value": {"url": "www.confluent.io"}, "timestamp": 2} + ], + "outputs": [ + {"topic": "OUTPUT", "key": "1", "value": {"PROTOCOL": "http"}, "timestamp": 0}, + {"topic": "OUTPUT", "key": "2", "value": {"PROTOCOL": "https"}, "timestamp": 1}, + {"topic": "OUTPUT", "key": "3", "value": {"PROTOCOL": ""}, "timestamp": 2} + ] + }, { "name": "extract a query from a URL using URL_EXTRACT_QUERY", - "format": ["JSON", "PROTOBUF"], + "format": ["JSON", "AVRO"], "statements": [ - "CREATE STREAM TEST (K STRING KEY, url VARCHAR) WITH (kafka_topic='test_topic', value_format='JSON');", + "CREATE STREAM TEST (K STRING KEY, url VARCHAR) WITH (kafka_topic='test_topic', value_format='{FORMAT}');", "CREATE STREAM OUTPUT AS SELECT K, URL_EXTRACT_QUERY(url) as Q FROM TEST;" ], "inputs": [ @@ -171,6 +221,26 @@ {"topic": "OUTPUT", "key": "3", "value": {"Q": "q=2"}, "timestamp": 2}, {"topic": "OUTPUT", "key": "4", "value": {"Q": null}, "timestamp": 3} ] + }, + { + "name": "extract a query from a URL using URL_EXTRACT_QUERY - PROTOBUFs", + "format": ["PROTOBUF", "PROTOBUF_NOSR"], + "statements": [ + "CREATE STREAM TEST (K STRING KEY, url VARCHAR) WITH (kafka_topic='test_topic', value_format='{FORMAT}');", + "CREATE STREAM OUTPUT AS SELECT K, URL_EXTRACT_QUERY(url) as Q FROM TEST;" + ], + "inputs": [ + {"topic": "test_topic", "key": "1", "value": {"url": "http://www.test.com?query#fragment"}, "timestamp": 0}, + {"topic": "test_topic", "key": "2", "value": {"url": "http://www.test.com/path?q1&q2"}, "timestamp": 1}, + {"topic": "test_topic", "key": "3", "value": {"url": "http://test@confluent.io:8080/nested/path?q=2"}, "timestamp": 2}, + {"topic": "test_topic", "key": "4", "value": {"url": "http://test@confluent.io:8080/path"}, "timestamp": 3} + ], + "outputs": [ + {"topic": "OUTPUT", "key": "1", "value": {"Q": "query"}, "timestamp": 0}, + {"topic": "OUTPUT", "key": "2", "value": {"Q": "q1&q2"}, "timestamp": 1}, + {"topic": "OUTPUT", "key": "3", "value": {"Q": "q=2"}, "timestamp": 2}, + {"topic": "OUTPUT", "key": "4", "value": {"Q": ""}, "timestamp": 3} + ] } ] } \ No newline at end of file diff --git a/ksqldb-parser/src/main/antlr4/io/confluent/ksql/parser/SqlBase.g4 b/ksqldb-parser/src/main/antlr4/io/confluent/ksql/parser/SqlBase.g4 index 1e75f1671bc0..fa74e0b421fb 100644 --- a/ksqldb-parser/src/main/antlr4/io/confluent/ksql/parser/SqlBase.g4 +++ b/ksqldb-parser/src/main/antlr4/io/confluent/ksql/parser/SqlBase.g4 @@ -59,7 +59,7 @@ statement | DESCRIBE STREAMS EXTENDED? #describeStreams | DESCRIBE FUNCTION identifier #describeFunction | DESCRIBE CONNECTOR identifier #describeConnector - | PRINT (identifier| STRING) printClause #printTopic + | PRINT resourceName printClause #printTopic | (LIST | SHOW) QUERIES EXTENDED? #listQueries | TERMINATE identifier #terminateQuery | TERMINATE ALL #terminateQuery @@ -89,6 +89,10 @@ statement | CREATE TYPE (IF NOT EXISTS)? identifier AS type #registerType | DROP TYPE (IF EXISTS)? identifier #dropType | ALTER (STREAM | TABLE) sourceName alterOption (',' alterOption)* #alterSource + | ASSERT (NOT EXISTS)? TOPIC resourceName + (WITH tableProperties)? timeout? #assertTopic + | ASSERT (NOT EXISTS)? SCHEMA + (SUBJECT resourceName)? (ID literal)? timeout? #assertSchema ; assertStatement @@ -102,6 +106,11 @@ runScript : RUN SCRIPT STRING ; +resourceName + : identifier + | STRING + ; + query : SELECT selectItem (',' selectItem)* FROM from=relation @@ -119,6 +128,10 @@ resultMaterialization | FINAL ; +timeout + : TIMEOUT number windowUnit + ; + alterOption : ADD (COLUMN)? identifier type ; @@ -414,6 +427,7 @@ nonReserved | GRACE | PERIOD | DEFINE | UNDEFINE | VARIABLES | PLUGINS | SYSTEM + | TIMEOUT | SCHEMA| SUBJECT | ID ; EMIT: 'EMIT'; @@ -553,6 +567,10 @@ PLUGINS: 'PLUGINS'; HEADERS: 'HEADERS'; HEADER: 'HEADER'; SYSTEM: 'SYSTEM'; +TIMEOUT: 'TIMEOUT'; +SCHEMA: 'SCHEMA'; +SUBJECT: 'SUBJECT'; +ID: 'ID'; IF: 'IF'; diff --git a/ksqldb-parser/src/main/java/io/confluent/ksql/parser/AstBuilder.java b/ksqldb-parser/src/main/java/io/confluent/ksql/parser/AstBuilder.java index b21ead57c774..75d97cdbe320 100644 --- a/ksqldb-parser/src/main/java/io/confluent/ksql/parser/AstBuilder.java +++ b/ksqldb-parser/src/main/java/io/confluent/ksql/parser/AstBuilder.java @@ -70,9 +70,11 @@ import io.confluent.ksql.parser.SqlBaseParser.AlterOptionContext; import io.confluent.ksql.parser.SqlBaseParser.AlterSourceContext; import io.confluent.ksql.parser.SqlBaseParser.ArrayConstructorContext; +import io.confluent.ksql.parser.SqlBaseParser.AssertSchemaContext; import io.confluent.ksql.parser.SqlBaseParser.AssertStreamContext; import io.confluent.ksql.parser.SqlBaseParser.AssertTableContext; import io.confluent.ksql.parser.SqlBaseParser.AssertTombstoneContext; +import io.confluent.ksql.parser.SqlBaseParser.AssertTopicContext; import io.confluent.ksql.parser.SqlBaseParser.AssertValuesContext; import io.confluent.ksql.parser.SqlBaseParser.CreateConnectorContext; import io.confluent.ksql.parser.SqlBaseParser.DescribeConnectorContext; @@ -103,9 +105,11 @@ import io.confluent.ksql.parser.tree.AlterOption; import io.confluent.ksql.parser.tree.AlterSource; import io.confluent.ksql.parser.tree.AlterSystemProperty; +import io.confluent.ksql.parser.tree.AssertSchema; import io.confluent.ksql.parser.tree.AssertStatement; import io.confluent.ksql.parser.tree.AssertStream; import io.confluent.ksql.parser.tree.AssertTombstone; +import io.confluent.ksql.parser.tree.AssertTopic; import io.confluent.ksql.parser.tree.AssertValues; import io.confluent.ksql.parser.tree.ColumnConstraints; import io.confluent.ksql.parser.tree.CreateConnector; @@ -829,12 +833,7 @@ public Node visitUndefineVariable(final SqlBaseParser.UndefineVariableContext co public Node visitPrintTopic(final SqlBaseParser.PrintTopicContext context) { final boolean fromBeginning = context.printClause().FROM() != null; - final String topicName; - if (context.STRING() != null) { - topicName = ParserUtil.unquote(context.STRING().getText(), "'"); - } else { - topicName = ParserUtil.getIdentifierText(true, context.identifier()); - } + final String topicName = getResourceName(context.resourceName()); final IntervalClauseContext intervalContext = context.printClause().intervalClause(); final OptionalInt interval = intervalContext == null @@ -852,6 +851,16 @@ public Node visitPrintTopic(final SqlBaseParser.PrintTopicContext context) { ); } + private String getResourceName( + final SqlBaseParser.ResourceNameContext context + ) { + if (context.STRING() != null) { + return ParserUtil.unquote(context.STRING().getText(), "'"); + } else { + return ParserUtil.getIdentifierText(true, context.identifier()); + } + } + @Override public Node visitLogicalNot(final SqlBaseParser.LogicalNotContext context) { return new NotExpression(getLocation(context), @@ -1397,6 +1406,54 @@ public Node visitRegisterType(final RegisterTypeContext context) { ); } + @Override + public Node visitAssertTopic(final AssertTopicContext context) { + return new AssertTopic( + getLocation(context), + getResourceName(context.resourceName()), + context.WITH() == null + ? ImmutableMap.of() + : processTableProperties(context.tableProperties()), + context.timeout() == null + ? Optional.empty() + : Optional.of(getTimeClause( + context.timeout().number(), context.timeout().windowUnit())), + context.EXISTS() == null + ); + } + + @Override + public Node visitAssertSchema(final AssertSchemaContext context) { + if (context.resourceName() == null && context.literal() == null) { + throw new KsqlException("ASSERT SCHEMA statements much include a subject name or an id"); + } + + final Optional id; + if (context.literal() == null) { + id = Optional.empty(); + } else { + final Object value = ((Literal) visit(context.literal())).getValue(); + if (value instanceof Integer) { + id = Optional.of((Integer) value); + } else { + throw new KsqlException("ID must be an integer"); + } + } + + return new AssertSchema( + getLocation(context), + context.resourceName() == null + ? Optional.empty() + : Optional.of(getResourceName(context.resourceName())), + id, + context.timeout() == null + ? Optional.empty() + : Optional.of(getTimeClause( + context.timeout().number(), context.timeout().windowUnit())), + context.EXISTS() == null + ); + } + @Override public Node visitAssertValues(final AssertValuesContext context) { final SourceName targetName = ParserUtil.getSourceName(context.sourceName()); diff --git a/ksqldb-parser/src/main/java/io/confluent/ksql/parser/VariableSubstitutor.java b/ksqldb-parser/src/main/java/io/confluent/ksql/parser/VariableSubstitutor.java index 35e255d47c08..e6a2703ebb6b 100644 --- a/ksqldb-parser/src/main/java/io/confluent/ksql/parser/VariableSubstitutor.java +++ b/ksqldb-parser/src/main/java/io/confluent/ksql/parser/VariableSubstitutor.java @@ -109,6 +109,17 @@ private void lookupVariables(final String text) { } } + @Override + public Void visitResourceName(final SqlBaseParser.ResourceNameContext context) { + if (context.STRING() != null) { + final String text = unquote(context.STRING().getText(), "'"); + lookupVariables(text); + } else { + visit(context.identifier()); + } + return null; + } + @Override public Void visitStringLiteral(final SqlBaseParser.StringLiteralContext context) { final String text = unquote(context.getText(), "\'"); diff --git a/ksqldb-parser/src/main/java/io/confluent/ksql/parser/tree/AssertResource.java b/ksqldb-parser/src/main/java/io/confluent/ksql/parser/tree/AssertResource.java new file mode 100644 index 000000000000..80a301f6abf1 --- /dev/null +++ b/ksqldb-parser/src/main/java/io/confluent/ksql/parser/tree/AssertResource.java @@ -0,0 +1,44 @@ +/* + * Copyright 2022 Confluent Inc. + * + * Licensed under the Confluent Community License (the "License"; you may not use + * this file except in compliance with the License. You may obtain a copy of the + * License at + * + * http://www.confluent.io/confluent-community-license + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, WITHOUT + * WARRANTIES OF ANY KIND, either express or implied. See the License for the + * specific language governing permissions and limitations under the License. + */ + +package io.confluent.ksql.parser.tree; + +import io.confluent.ksql.execution.windows.WindowTimeClause; +import io.confluent.ksql.parser.NodeLocation; +import java.util.Objects; +import java.util.Optional; + +public abstract class AssertResource extends Statement { + protected final Optional timeout; + protected final boolean exists; + + protected AssertResource( + final Optional location, + final Optional timeout, + final boolean exists + ) { + super(location); + this.timeout = Objects.requireNonNull(timeout, "timeout"); + this.exists = exists; + } + + public Optional getTimeout() { + return timeout; + } + + public boolean checkExists() { + return exists; + } +} diff --git a/ksqldb-parser/src/main/java/io/confluent/ksql/parser/tree/AssertSchema.java b/ksqldb-parser/src/main/java/io/confluent/ksql/parser/tree/AssertSchema.java new file mode 100644 index 000000000000..b8a0906521b6 --- /dev/null +++ b/ksqldb-parser/src/main/java/io/confluent/ksql/parser/tree/AssertSchema.java @@ -0,0 +1,79 @@ +/* + * Copyright 2022 Confluent Inc. + * + * Licensed under the Confluent Community License (the "License"; you may not use + * this file except in compliance with the License. You may obtain a copy of the + * License at + * + * http://www.confluent.io/confluent-community-license + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, WITHOUT + * WARRANTIES OF ANY KIND, either express or implied. See the License for the + * specific language governing permissions and limitations under the License. + */ + +package io.confluent.ksql.parser.tree; + +import com.google.errorprone.annotations.Immutable; +import io.confluent.ksql.execution.windows.WindowTimeClause; +import io.confluent.ksql.parser.NodeLocation; +import java.util.Objects; +import java.util.Optional; + +@Immutable +public class AssertSchema extends AssertResource { + + private final Optional subject; + private final Optional id; + + public AssertSchema( + final Optional location, + final Optional subject, + final Optional id, + final Optional timeout, + final boolean exists + ) { + super(location, timeout, exists); + this.subject = Objects.requireNonNull(subject, "subject"); + this.id = Objects.requireNonNull(id, "id"); + } + + public Optional getSubject() { + return subject; + } + + public Optional getId() { + return id; + } + + @Override + public boolean equals(final Object o) { + if (this == o) { + return true; + } + if (o == null || getClass() != o.getClass()) { + return false; + } + final AssertSchema that = (AssertSchema) o; + return subject.equals(that.subject) + && id.equals(that.id) + && timeout.equals(that.timeout) + && exists == that.exists; + } + + @Override + public int hashCode() { + return Objects.hash(subject, id, timeout, exists); + } + + @Override + public String toString() { + return "AssertSchema{" + + "subject=" + subject + + ",id=" + id + + ",timeout=" + timeout + + ",exists=" + exists + + '}'; + } +} diff --git a/ksqldb-parser/src/main/java/io/confluent/ksql/parser/tree/AssertTopic.java b/ksqldb-parser/src/main/java/io/confluent/ksql/parser/tree/AssertTopic.java new file mode 100644 index 000000000000..3da10a77de7c --- /dev/null +++ b/ksqldb-parser/src/main/java/io/confluent/ksql/parser/tree/AssertTopic.java @@ -0,0 +1,82 @@ +/* + * Copyright 2022 Confluent Inc. + * + * Licensed under the Confluent Community License (the "License"; you may not use + * this file except in compliance with the License. You may obtain a copy of the + * License at + * + * http://www.confluent.io/confluent-community-license + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, WITHOUT + * WARRANTIES OF ANY KIND, either express or implied. See the License for the + * specific language governing permissions and limitations under the License. + */ + +package io.confluent.ksql.parser.tree; + +import com.google.common.collect.ImmutableMap; +import com.google.errorprone.annotations.Immutable; +import io.confluent.ksql.execution.expression.tree.Literal; +import io.confluent.ksql.execution.windows.WindowTimeClause; +import io.confluent.ksql.parser.NodeLocation; +import java.util.Map; +import java.util.Objects; +import java.util.Optional; + +@Immutable +public class AssertTopic extends AssertResource { + + private final String topic; + private final ImmutableMap config; + + public AssertTopic( + final Optional location, + final String topic, + final Map config, + final Optional timeout, + final boolean exists + ) { + super(location, timeout, exists); + this.topic = Objects.requireNonNull(topic, "topic"); + this.config = ImmutableMap.copyOf(Objects.requireNonNull(config, "config")); + } + + public String getTopic() { + return topic; + } + + public Map getConfig() { + return config; + } + + @Override + public boolean equals(final Object o) { + if (this == o) { + return true; + } + if (o == null || getClass() != o.getClass()) { + return false; + } + final AssertTopic that = (AssertTopic) o; + return topic.equals(that.topic) + && Objects.equals(config, that.config) + && timeout.equals(that.timeout) + && exists == that.exists; + } + + @Override + public int hashCode() { + return Objects.hash(topic, config, timeout, exists); + } + + @Override + public String toString() { + return "AssertTopic{" + + "topic=" + topic + + ",config=" + config + + ",timeout=" + timeout + + ",exists=" + exists + + '}'; + } +} diff --git a/ksqldb-parser/src/test/java/io/confluent/ksql/parser/AstBuilderTest.java b/ksqldb-parser/src/test/java/io/confluent/ksql/parser/AstBuilderTest.java index 16c1785deec7..fe8d22aa5ab9 100644 --- a/ksqldb-parser/src/test/java/io/confluent/ksql/parser/AstBuilderTest.java +++ b/ksqldb-parser/src/test/java/io/confluent/ksql/parser/AstBuilderTest.java @@ -46,6 +46,8 @@ import io.confluent.ksql.parser.exception.ParseFailedException; import io.confluent.ksql.parser.tree.AliasedRelation; import io.confluent.ksql.parser.tree.AllColumns; +import io.confluent.ksql.parser.tree.AssertSchema; +import io.confluent.ksql.parser.tree.AssertTopic; import io.confluent.ksql.parser.tree.ColumnConstraints; import io.confluent.ksql.parser.tree.CreateStream; import io.confluent.ksql.parser.tree.CreateTable; @@ -963,4 +965,111 @@ public void shouldFailOnPersistentQueryLimitClauseTable() { assertEquals(expectedMessage, actualMessage); } + + @Test + public void shouldBuildAssertTopic() { + // Given: + final SingleStatementContext stmt + = givenQuery("ASSERT TOPIC X;"); + + // When: + final AssertTopic assertTopic = (AssertTopic) builder.buildStatement(stmt); + + // Then: + assertThat(assertTopic.getTopic(), is("X")); + assertThat(assertTopic.getConfig().size(), is(0)); + assertThat(assertTopic.getTimeout(), is(Optional.empty())); + assertThat(assertTopic.checkExists(), is(true)); + } + + @Test + public void shouldBuildAssertNotExistsTopicWithConfigsAndTimeout() { + // Given: + final SingleStatementContext stmt + = givenQuery("ASSERT NOT EXISTS TOPIC 'a-b-c' WITH (REPLICAS=1, partitions=1) TIMEOUT 10 SECONDS;"); + + // When: + final AssertTopic assertTopic = (AssertTopic) builder.buildStatement(stmt); + + // Then: + assertThat(assertTopic.getTopic(), is("a-b-c")); + assertThat(assertTopic.getConfig().get("REPLICAS").getValue(), is(1)); + assertThat(assertTopic.getConfig().get("PARTITIONS").getValue(), is(1)); + assertThat(assertTopic.getTimeout().get().getTimeUnit(), is(TimeUnit.SECONDS)); + assertThat(assertTopic.getTimeout().get().getValue(), is(10L)); + assertThat(assertTopic.checkExists(), is(false)); + } + + @Test + public void shouldBuildAssertSchemaWithSubject() { + // Given: + final SingleStatementContext stmt + = givenQuery("ASSERT SCHEMA SUBJECT X;"); + + // When: + final AssertSchema assertSchema = (AssertSchema) builder.buildStatement(stmt); + + // Then: + assertThat(assertSchema.getSubject(), is(Optional.of("X"))); + assertThat(assertSchema.getId(), is(Optional.empty())); + assertThat(assertSchema.getTimeout(), is(Optional.empty())); + assertThat(assertSchema.checkExists(), is(true)); + } + + @Test + public void shouldBuildAssertNotExistsSchemaWithIdAndTimeout() { + // Given: + final SingleStatementContext stmt + = givenQuery("ASSERT NOT EXISTS SCHEMA ID 24 TIMEOUT 10 SECONDS;"); + + // When: + final AssertSchema assertSchema = (AssertSchema) builder.buildStatement(stmt); + + // Then: + assertThat(assertSchema.getSubject(), is(Optional.empty())); + assertThat(assertSchema.getId(), is(Optional.of(24))); + assertThat(assertSchema.getTimeout().get().getTimeUnit(), is(TimeUnit.SECONDS)); + assertThat(assertSchema.checkExists(), is(false)); + } + + @Test + public void shouldBuildAssertNotExistsWithSubjectAndId() { + // Given: + final SingleStatementContext stmt + = givenQuery("ASSERT NOT EXISTS SCHEMA SUBJECT 'a-b-c' ID 33;"); + + // When: + final AssertSchema assertSchema = (AssertSchema) builder.buildStatement(stmt); + + // Then: + assertThat(assertSchema.getSubject(), is(Optional.of("a-b-c"))); + assertThat(assertSchema.getId(), is(Optional.of(33))); + assertThat(assertSchema.getTimeout(), is(Optional.empty())); + assertThat(assertSchema.checkExists(), is(false)); + } + + @Test + public void shouldThrowOnNoSubjectOrId() { + // Given: + final SingleStatementContext stmt = givenQuery("ASSERT SCHEMA TIMEOUT 10 SECONDS;"); + + // When: + final Exception e = assertThrows(KsqlException.class, () -> builder.buildStatement(stmt)); + + // Then: + assertThat(e.getMessage(), is("ASSERT SCHEMA statements much include a subject name or an id")); + } + + @Test + public void shouldThrowOnNonIntegerId() { + // Given: + final SingleStatementContext stmt = + givenQuery("ASSERT SCHEMA ID FALSE TIMEOUT 10 SECONDS;"); + + // When: + final Exception e = assertThrows(KsqlException.class, () -> builder.buildStatement(stmt)); + + // Then: + assertThat(e.getMessage(), is("ID must be an integer")); + } } \ No newline at end of file diff --git a/ksqldb-parser/src/test/java/io/confluent/ksql/parser/VariableSubstitutorTest.java b/ksqldb-parser/src/test/java/io/confluent/ksql/parser/VariableSubstitutorTest.java index 53a65f5dca96..608f78bef99c 100644 --- a/ksqldb-parser/src/test/java/io/confluent/ksql/parser/VariableSubstitutorTest.java +++ b/ksqldb-parser/src/test/java/io/confluent/ksql/parser/VariableSubstitutorTest.java @@ -254,6 +254,22 @@ public void shouldSubstituteVariablesInString() { assertThat(substituted, equalTo("Happy birthday to you!")); } + @Test + public void shouldSubstituteVariablesInAssert() { + // Given + final Map variablesMap = new ImmutableMap.Builder() {{ + put("name", "foo"); + }}.build(); + + // When + final String substitutedString = VariableSubstitutor.substitute("ASSERT TOPIC '${name}';", variablesMap); + final String substitutedIdentifier = VariableSubstitutor.substitute("ASSERT TOPIC ${name};", variablesMap); + + // Then + assertThat(substitutedString, equalTo("ASSERT TOPIC 'foo';")); + assertThat(substitutedIdentifier, equalTo("ASSERT TOPIC foo;")); + } + private void assertReplacedStatements( final List> statements, final Map variablesMap diff --git a/ksqldb-parser/src/test/java/io/confluent/ksql/parser/tree/AssertSchemaTest.java b/ksqldb-parser/src/test/java/io/confluent/ksql/parser/tree/AssertSchemaTest.java new file mode 100644 index 000000000000..68d3a301648a --- /dev/null +++ b/ksqldb-parser/src/test/java/io/confluent/ksql/parser/tree/AssertSchemaTest.java @@ -0,0 +1,59 @@ +/* + * Copyright 2022 Confluent Inc. + * + * Licensed under the Confluent Community License (the "License"; you may not use + * this file except in compliance with the License. You may obtain a copy of the + * License at + * + * http://www.confluent.io/confluent-community-license + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, WITHOUT + * WARRANTIES OF ANY KIND, either express or implied. See the License for the + * specific language governing permissions and limitations under the License. + */ + +package io.confluent.ksql.parser.tree; + +import com.google.common.testing.EqualsTester; +import io.confluent.ksql.execution.windows.WindowTimeClause; +import io.confluent.ksql.parser.NodeLocation; +import java.util.Optional; +import java.util.concurrent.TimeUnit; +import org.junit.Rule; +import org.junit.Test; +import org.mockito.junit.MockitoJUnit; + +import org.mockito.junit.MockitoRule; + +public class AssertSchemaTest { + + @Rule + public final MockitoRule mockitoRule = MockitoJUnit.rule(); + + private static final Optional SOME_ID = Optional.of(25); + private static final Optional SOME_SUBJECT = Optional.of("subject"); + private static final WindowTimeClause SOME_TIMEOUT = new WindowTimeClause(5, TimeUnit.SECONDS); + + @Test + public void shouldImplementHashCodeAndEquals() { + new EqualsTester() + .addEqualityGroup( + new AssertSchema(Optional.empty(), SOME_SUBJECT, SOME_ID, Optional.of(SOME_TIMEOUT), true), + new AssertSchema(Optional.of(new NodeLocation(1, 1)), SOME_SUBJECT, SOME_ID, Optional.of(SOME_TIMEOUT), true)) + .addEqualityGroup( + new AssertSchema(Optional.empty(), Optional.empty(), SOME_ID, Optional.of(SOME_TIMEOUT), true)) + .addEqualityGroup( + new AssertSchema(Optional.empty(), Optional.of("another subject"), SOME_ID, Optional.of(SOME_TIMEOUT), true)) + .addEqualityGroup( + new AssertSchema(Optional.empty(), SOME_SUBJECT, Optional.empty(), Optional.of(SOME_TIMEOUT), true)) + .addEqualityGroup( + new AssertSchema(Optional.empty(), SOME_SUBJECT, Optional.of(33), Optional.of(SOME_TIMEOUT), true)) + .addEqualityGroup( + new AssertSchema(Optional.empty(), SOME_SUBJECT, SOME_ID, Optional.empty(), true)) + .addEqualityGroup( + new AssertSchema(Optional.empty(), SOME_SUBJECT, SOME_ID, Optional.of(SOME_TIMEOUT), false)) + .testEquals(); + } + +} diff --git a/ksqldb-parser/src/test/java/io/confluent/ksql/parser/tree/AssertTopicTest.java b/ksqldb-parser/src/test/java/io/confluent/ksql/parser/tree/AssertTopicTest.java new file mode 100644 index 000000000000..78a73aebd9f4 --- /dev/null +++ b/ksqldb-parser/src/test/java/io/confluent/ksql/parser/tree/AssertTopicTest.java @@ -0,0 +1,57 @@ +/* + * Copyright 2022 Confluent Inc. + * + * Licensed under the Confluent Community License (the "License"; you may not use + * this file except in compliance with the License. You may obtain a copy of the + * License at + * + * http://www.confluent.io/confluent-community-license + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, WITHOUT + * WARRANTIES OF ANY KIND, either express or implied. See the License for the + * specific language governing permissions and limitations under the License. + */ + +package io.confluent.ksql.parser.tree; + +import com.google.common.collect.ImmutableMap; +import com.google.common.testing.EqualsTester; +import io.confluent.ksql.execution.windows.WindowTimeClause; +import io.confluent.ksql.parser.NodeLocation; +import java.util.Map; +import java.util.Optional; +import java.util.concurrent.TimeUnit; +import org.junit.Rule; +import org.junit.Test; +import org.mockito.junit.MockitoJUnit; + +import org.mockito.junit.MockitoRule; + +public class AssertTopicTest { + + @Rule + public final MockitoRule mockitoRule = MockitoJUnit.rule(); + + private final String SOME_TOPIC = "TOPIC"; + private final Map SOME_CONFIG = ImmutableMap.of("partitions", 1); + private final WindowTimeClause SOME_TIMEOUT = new WindowTimeClause(5, TimeUnit.SECONDS); + + @Test + public void shouldImplementHashCodeAndEquals() { + new EqualsTester() + .addEqualityGroup( + new AssertTopic(Optional.empty(), SOME_TOPIC, SOME_CONFIG, Optional.of(SOME_TIMEOUT), true), + new AssertTopic(Optional.of(new NodeLocation(1, 1)), SOME_TOPIC, SOME_CONFIG, Optional.of(SOME_TIMEOUT), true)) + .addEqualityGroup( + new AssertTopic(Optional.empty(), "another topic", SOME_CONFIG, Optional.of(SOME_TIMEOUT), true)) + .addEqualityGroup( + new AssertTopic(Optional.empty(), SOME_TOPIC, ImmutableMap.of(), Optional.of(SOME_TIMEOUT), true)) + .addEqualityGroup( + new AssertTopic(Optional.empty(), SOME_TOPIC, SOME_CONFIG, Optional.empty(), true)) + .addEqualityGroup( + new AssertTopic(Optional.empty(), SOME_TOPIC, SOME_CONFIG, Optional.of(SOME_TIMEOUT), false)) + .testEquals(); + } + +} diff --git a/ksqldb-parser/src/test/java/io/confluent/ksql/util/GrammarTokenExporterTest.java b/ksqldb-parser/src/test/java/io/confluent/ksql/util/GrammarTokenExporterTest.java index 12949a203a8f..8784b889a9d9 100644 --- a/ksqldb-parser/src/test/java/io/confluent/ksql/util/GrammarTokenExporterTest.java +++ b/ksqldb-parser/src/test/java/io/confluent/ksql/util/GrammarTokenExporterTest.java @@ -41,9 +41,9 @@ public class GrammarTokenExporterTest { "CATALOG", "PROPERTIES", "BEGINNING", "UNSET", "RUN", "SCRIPT", "DECIMAL", "KEY", "CONNECTOR", "CONNECTORS", "SINK", "SOURCE", "NAMESPACE", "MATERIALIZED", "VIEW", "PRIMARY", "REPLACE", "ASSERT", "ADD", "ALTER", "VARIABLES", "PLUGINS", "HEADERS", - "HEADER", "SYSTEM", "IF", "EQ", "NEQ", "LT", "LTE", "GT", "GTE", "PLUS", "MINUS", "ASTERISK", - "SLASH", "PERCENT", "CONCAT", "ASSIGN", "STRING", "IDENTIFIER", "VARIABLE", "EXPONENT", - "DIGIT", "LETTER", "WS")); + "HEADER", "SYSTEM", "TIMEOUT", "SCHEMA", "SUBJECT", "ID", "IF", "EQ", "NEQ", "LT", + "LTE", "GT", "GTE", "PLUS", "MINUS", "ASTERISK", "SLASH", "PERCENT", "CONCAT", "ASSIGN", + "STRING", "IDENTIFIER", "VARIABLE", "EXPONENT", "DIGIT", "LETTER", "WS")); @Test public void shouldNeedBackQuotes() { diff --git a/ksqldb-rest-app/src/main/java/io/confluent/ksql/rest/server/KsqlRestApplication.java b/ksqldb-rest-app/src/main/java/io/confluent/ksql/rest/server/KsqlRestApplication.java index a4389b1e0b2c..aaba42cec4af 100644 --- a/ksqldb-rest-app/src/main/java/io/confluent/ksql/rest/server/KsqlRestApplication.java +++ b/ksqldb-rest-app/src/main/java/io/confluent/ksql/rest/server/KsqlRestApplication.java @@ -662,8 +662,11 @@ static KsqlRestApplication buildApplication( final ProcessingLogConfig processingLogConfig = new ProcessingLogConfig(restConfig.getOriginals()); - final ProcessingLogContext processingLogContext - = ProcessingLogContext.create(processingLogConfig); + final ProcessingLogContext processingLogContext = ProcessingLogContext.create( + processingLogConfig, + metricCollectors.getMetrics(), + ksqlConfig.getStringAsMap(KsqlConfig.KSQL_CUSTOM_METRICS_TAGS) + ); final MutableFunctionRegistry functionRegistry = new InternalFunctionRegistry(); diff --git a/ksqldb-rest-app/src/main/java/io/confluent/ksql/rest/server/StandaloneExecutorFactory.java b/ksqldb-rest-app/src/main/java/io/confluent/ksql/rest/server/StandaloneExecutorFactory.java index 099d495bb38e..57de77ec0c9b 100644 --- a/ksqldb-rest-app/src/main/java/io/confluent/ksql/rest/server/StandaloneExecutorFactory.java +++ b/ksqldb-rest-app/src/main/java/io/confluent/ksql/rest/server/StandaloneExecutorFactory.java @@ -124,7 +124,11 @@ static StandaloneExecutor create( final ProcessingLogConfig processingLogConfig = new ProcessingLogConfig(properties); final ProcessingLogContext processingLogContext - = ProcessingLogContext.create(processingLogConfig); + = ProcessingLogContext.create( + processingLogConfig, + metricCollectors.getMetrics(), + ksqlConfig.getStringAsMap(KsqlConfig.KSQL_CUSTOM_METRICS_TAGS) + ); final MutableFunctionRegistry functionRegistry = new InternalFunctionRegistry(); diff --git a/ksqldb-rest-app/src/main/java/io/confluent/ksql/rest/server/computation/Command.java b/ksqldb-rest-app/src/main/java/io/confluent/ksql/rest/server/computation/Command.java index 43580872c9aa..41b6ea6a51f2 100644 --- a/ksqldb-rest-app/src/main/java/io/confluent/ksql/rest/server/computation/Command.java +++ b/ksqldb-rest-app/src/main/java/io/confluent/ksql/rest/server/computation/Command.java @@ -38,7 +38,7 @@ public class Command { @VisibleForTesting - public static final int VERSION = 14; + public static final int VERSION = 15; private final String statement; private final Map overwriteProperties; diff --git a/ksqldb-rest-app/src/main/java/io/confluent/ksql/rest/server/execution/AssertExecutor.java b/ksqldb-rest-app/src/main/java/io/confluent/ksql/rest/server/execution/AssertExecutor.java new file mode 100644 index 000000000000..beed4d6ce824 --- /dev/null +++ b/ksqldb-rest-app/src/main/java/io/confluent/ksql/rest/server/execution/AssertExecutor.java @@ -0,0 +1,64 @@ +/* + * Copyright 2022 Confluent Inc. + * + * Licensed under the Confluent Community License (the "License"); you may not use + * this file except in compliance with the License. You may obtain a copy of the + * License at + * + * http://www.confluent.io/confluent-community-license + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, WITHOUT + * WARRANTIES OF ANY KIND, either express or implied. See the License for the + * specific language governing permissions and limitations under the License. + */ + +package io.confluent.ksql.rest.server.execution; + +import static io.confluent.ksql.rest.Errors.assertionFailedError; +import static io.confluent.ksql.util.KsqlConfig.KSQL_ASSERT_TOPIC_DEFAULT_TIMEOUT_MS; + +import io.confluent.ksql.KsqlExecutionContext; +import io.confluent.ksql.parser.tree.AssertResource; +import io.confluent.ksql.rest.entity.KsqlEntity; +import io.confluent.ksql.rest.server.resources.KsqlRestException; +import io.confluent.ksql.services.ServiceContext; +import io.confluent.ksql.util.KsqlException; +import io.confluent.ksql.util.RetryUtil; +import java.util.Optional; +import java.util.function.BiConsumer; +import java.util.function.BiFunction; + +public final class AssertExecutor { + + private static final int RETRY_MS = 100; + + private AssertExecutor() { + + } + + static StatementExecutorResponse execute( + final String statementText, + final AssertResource statement, + final KsqlExecutionContext executionContext, + final ServiceContext serviceContext, + final BiConsumer assertResource, + final BiFunction createSuccessfulEntity + ) { + final int timeout = statement.getTimeout().isPresent() + ? (int) statement.getTimeout().get().toDuration().toMillis() + : executionContext.getKsqlConfig().getInt(KSQL_ASSERT_TOPIC_DEFAULT_TIMEOUT_MS); + try { + RetryUtil.retryWithBackoff( + timeout / RETRY_MS, + RETRY_MS, + RETRY_MS, + () -> assertResource.accept(statement, serviceContext) + ); + } catch (final KsqlException e) { + throw new KsqlRestException(assertionFailedError(e.getMessage())); + } + return StatementExecutorResponse.handled( + Optional.of(createSuccessfulEntity.apply(statementText, statement))); + } +} diff --git a/ksqldb-rest-app/src/main/java/io/confluent/ksql/rest/server/execution/AssertSchemaExecutor.java b/ksqldb-rest-app/src/main/java/io/confluent/ksql/rest/server/execution/AssertSchemaExecutor.java new file mode 100644 index 000000000000..98c69ab551b6 --- /dev/null +++ b/ksqldb-rest-app/src/main/java/io/confluent/ksql/rest/server/execution/AssertSchemaExecutor.java @@ -0,0 +1,114 @@ +/* + * Copyright 2022 Confluent Inc. + * + * Licensed under the Confluent Community License (the "License"); you may not use + * this file except in compliance with the License. You may obtain a copy of the + * License at + * + * http://www.confluent.io/confluent-community-license + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, WITHOUT + * WARRANTIES OF ANY KIND, either express or implied. See the License for the + * specific language governing permissions and limitations under the License. + */ + +package io.confluent.ksql.rest.server.execution; + +import io.confluent.kafka.schemaregistry.client.SchemaRegistryClient; +import io.confluent.kafka.schemaregistry.client.rest.exceptions.RestClientException; +import io.confluent.ksql.KsqlExecutionContext; +import io.confluent.ksql.parser.tree.AssertSchema; +import io.confluent.ksql.rest.SessionProperties; +import io.confluent.ksql.rest.entity.AssertSchemaEntity; +import io.confluent.ksql.services.ServiceContext; +import io.confluent.ksql.statement.ConfiguredStatement; +import io.confluent.ksql.util.KsqlException; +import java.util.Optional; + +public final class AssertSchemaExecutor { + + private static final int HTTP_NOT_FOUND = 404; + private static final int SCHEMA_NOT_FOUND_ERROR_CODE = 40403; + private static final int SUBJECT_NOT_FOUND_ERROR_CODE = 40401; + + private AssertSchemaExecutor() { + + } + + public static StatementExecutorResponse execute( + final ConfiguredStatement statement, + final SessionProperties sessionProperties, + final KsqlExecutionContext executionContext, + final ServiceContext serviceContext + ) { + return AssertExecutor.execute( + statement.getStatementText(), + statement.getStatement(), + executionContext, + serviceContext, + (stmt, sc) -> assertSchema( + sc.getSchemaRegistryClient(), + ((AssertSchema) stmt).getSubject(), + ((AssertSchema) stmt).getId(), + stmt.checkExists()), + (str, stmt) -> new AssertSchemaEntity( + str, + ((AssertSchema) stmt).getSubject(), + ((AssertSchema) stmt).getId(), + stmt.checkExists()) + ); + } + + private static void assertSchema( + final SchemaRegistryClient client, + final Optional subject, + final Optional id, + final boolean assertExists + ) { + final boolean schemaExists = checkSchemaExistence(client, subject, id); + final String subjectString = subject.isPresent() ? " subject name " + subject.get() : ""; + final String idString = id.isPresent() ? " id " + id.get() : ""; + if (!assertExists) { + if (schemaExists) { + throw new KsqlException("Schema with" + subjectString + idString + " exists"); + } + } else { + if (!schemaExists) { + throw new KsqlException("Schema with" + subjectString + idString + " does not exist"); + } + } + } + + private static boolean checkSchemaExistence( + final SchemaRegistryClient client, + final Optional subject, + final Optional id + ) { + try { + if (subject.isPresent() && id.isPresent()) { + return client.getAllSubjectsById(id.get()).contains(subject.get()); + } else if (id.isPresent()) { + client.getSchemaById(id.get()); + } else if (subject.isPresent()) { + client.getLatestSchemaMetadata(subject.get()); + } + return true; + } catch (final Exception e) { + if (isSchemaNotFoundException(e)) { + return false; + } else { + throw new KsqlException("Cannot check schema existence: " + e.getMessage()); + } + } + } + + private static boolean isSchemaNotFoundException(final Exception e) { + if (e instanceof RestClientException) { + return ((RestClientException) e).getStatus() == HTTP_NOT_FOUND + && (((RestClientException) e).getErrorCode() == SCHEMA_NOT_FOUND_ERROR_CODE + || ((RestClientException) e).getErrorCode() == SUBJECT_NOT_FOUND_ERROR_CODE); + } + return false; + } +} diff --git a/ksqldb-rest-app/src/main/java/io/confluent/ksql/rest/server/execution/AssertTopicExecutor.java b/ksqldb-rest-app/src/main/java/io/confluent/ksql/rest/server/execution/AssertTopicExecutor.java new file mode 100644 index 000000000000..5f20f84ef416 --- /dev/null +++ b/ksqldb-rest-app/src/main/java/io/confluent/ksql/rest/server/execution/AssertTopicExecutor.java @@ -0,0 +1,134 @@ +/* + * Copyright 2019 Confluent Inc. + * + * Licensed under the Confluent Community License (the "License"); you may not use + * this file except in compliance with the License. You may obtain a copy of the + * License at + * + * http://www.confluent.io/confluent-community-license + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, WITHOUT + * WARRANTIES OF ANY KIND, either express or implied. See the License for the + * specific language governing permissions and limitations under the License. + */ + +package io.confluent.ksql.rest.server.execution; + +import io.confluent.ksql.KsqlExecutionContext; +import io.confluent.ksql.execution.expression.tree.Literal; +import io.confluent.ksql.parser.tree.AssertTopic; +import io.confluent.ksql.rest.SessionProperties; +import io.confluent.ksql.rest.entity.AssertTopicEntity; +import io.confluent.ksql.services.KafkaTopicClient; +import io.confluent.ksql.services.ServiceContext; +import io.confluent.ksql.statement.ConfiguredStatement; +import io.confluent.ksql.util.KsqlException; +import java.util.ArrayList; +import java.util.List; +import java.util.Map; +import org.apache.kafka.common.TopicPartitionInfo; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +public final class AssertTopicExecutor { + + private static final Logger LOG = LoggerFactory.getLogger(AssertTopicExecutor.class); + + private AssertTopicExecutor() { + + } + + public static StatementExecutorResponse execute( + final ConfiguredStatement statement, + final SessionProperties sessionProperties, + final KsqlExecutionContext executionContext, + final ServiceContext serviceContext + ) { + return AssertExecutor.execute(statement.getStatementText(), + statement.getStatement(), + executionContext, + serviceContext, + (stmt, sc) -> assertTopic( + sc.getTopicClient(), + ((AssertTopic) stmt).getTopic(), + ((AssertTopic) stmt).getConfig(), + stmt.checkExists()), + (str, stmt) -> new AssertTopicEntity( + str, + ((AssertTopic) stmt).getTopic(), + stmt.checkExists()) + ); + } + + private static void assertTopic( + final KafkaTopicClient client, + final String topic, + final Map config, + final boolean assertExists + ) { + final boolean topicExists; + try { + topicExists = client.isTopicExists(topic); + } catch (final Exception e) { + throw new KsqlException("Cannot check topic existence: " + e.getMessage()); + } + + if (!assertExists) { + if (config.size() > 0) { + LOG.warn("Will skip topic config check for topic non-existence assertion."); + } + if (topicExists) { + throw new KsqlException("Topic " + topic + " exists"); + } + } else { + if (topicExists) { + final List partitionList = client.describeTopic(topic).partitions(); + checkConfigs(topic, config, partitionList.size(), partitionList.get(0).replicas().size()); + } else { + throw new KsqlException("Topic " + topic + " does not exist"); + } + } + } + + private static void checkConfigs( + final String topic, + final Map config, + final int partitions, + final int replicas + ) { + final List configErrors = new ArrayList<>(); + config.forEach((k, v) -> { + if (k.toLowerCase().equals("partitions")) { + if (!configMatches(v.getValue(), partitions)) { + configErrors.add( + createConfigError(topic, "partitions", v.getValue().toString(), partitions)); + } + } else if (k.toLowerCase().equals("replicas")) { + if (!configMatches(v.getValue(), replicas)) { + configErrors.add( + createConfigError(topic, "replicas", v.getValue().toString(), replicas)); + } + } else { + configErrors.add("Cannot assert unknown topic property: " + k); + } + }); + if (configErrors.size() > 0) { + throw new KsqlException(String.join("\n", configErrors)); + } + } + + private static boolean configMatches(final Object expected, final int actual) { + if (expected instanceof Integer && (Integer) expected == actual) { + return true; + } + return false; + } + + private static String createConfigError( + final String topic, final String config, final String expected, final int actual) { + return String.format( + "Mismatched configuration for topic %s: For config %s, expected %s got %d", + topic, config, expected, actual); + } +} diff --git a/ksqldb-rest-app/src/main/java/io/confluent/ksql/rest/server/execution/CustomExecutors.java b/ksqldb-rest-app/src/main/java/io/confluent/ksql/rest/server/execution/CustomExecutors.java index e04d8a4e6ab6..508e91d692e3 100644 --- a/ksqldb-rest-app/src/main/java/io/confluent/ksql/rest/server/execution/CustomExecutors.java +++ b/ksqldb-rest-app/src/main/java/io/confluent/ksql/rest/server/execution/CustomExecutors.java @@ -17,6 +17,8 @@ import com.google.common.collect.ImmutableMap; import io.confluent.ksql.KsqlExecutionContext; +import io.confluent.ksql.parser.tree.AssertSchema; +import io.confluent.ksql.parser.tree.AssertTopic; import io.confluent.ksql.parser.tree.CreateConnector; import io.confluent.ksql.parser.tree.DefineVariable; import io.confluent.ksql.parser.tree.DescribeConnector; @@ -72,6 +74,8 @@ public enum CustomExecutors { LIST_CONNECTOR_PLUGINS(ListConnectorPlugins.class, ListConnectorPluginsExecutor::execute), LIST_TYPES(ListTypes.class, ListTypesExecutor::execute), LIST_VARIABLES(ListVariables.class, ListVariablesExecutor::execute), + ASSERT_TOPIC(AssertTopic.class, AssertTopicExecutor::execute), + ASSERT_SCHEMA(AssertSchema.class, AssertSchemaExecutor::execute), SHOW_COLUMNS(ShowColumns.class, ListSourceExecutor::columns), EXPLAIN(Explain.class, ExplainExecutor::execute), diff --git a/ksqldb-rest-app/src/main/java/io/confluent/ksql/rest/server/validation/CustomValidators.java b/ksqldb-rest-app/src/main/java/io/confluent/ksql/rest/server/validation/CustomValidators.java index aa80450a3d7f..e02ec567274d 100644 --- a/ksqldb-rest-app/src/main/java/io/confluent/ksql/rest/server/validation/CustomValidators.java +++ b/ksqldb-rest-app/src/main/java/io/confluent/ksql/rest/server/validation/CustomValidators.java @@ -18,6 +18,8 @@ import com.google.common.collect.ImmutableMap; import io.confluent.ksql.KsqlExecutionContext; import io.confluent.ksql.parser.tree.AlterSystemProperty; +import io.confluent.ksql.parser.tree.AssertSchema; +import io.confluent.ksql.parser.tree.AssertTopic; import io.confluent.ksql.parser.tree.CreateConnector; import io.confluent.ksql.parser.tree.DefineVariable; import io.confluent.ksql.parser.tree.DescribeConnector; @@ -93,6 +95,8 @@ public enum CustomValidators { LIST_TYPES(ListTypes.class, StatementValidator.NO_VALIDATION), CREATE_CONNECTOR(CreateConnector.class, ConnectExecutor::validate), DROP_CONNECTOR(DropConnector.class, StatementValidator.NO_VALIDATION), + ASSERT_TOPIC(AssertTopic.class, StatementValidator.NO_VALIDATION), + ASSERT_SCHEMA(AssertSchema.class, StatementValidator.NO_VALIDATION), LIST_VARIABLES(ListVariables.class, ListVariablesExecutor::execute), INSERT_VALUES(InsertValues.class, new InsertValuesExecutor()::execute), diff --git a/ksqldb-rest-app/src/test/java/io/confluent/ksql/api/BaseApiTest.java b/ksqldb-rest-app/src/test/java/io/confluent/ksql/api/BaseApiTest.java index 35f535331cb8..67fb6b04f667 100644 --- a/ksqldb-rest-app/src/test/java/io/confluent/ksql/api/BaseApiTest.java +++ b/ksqldb-rest-app/src/test/java/io/confluent/ksql/api/BaseApiTest.java @@ -51,6 +51,7 @@ import java.util.List; import java.util.Map; import java.util.Optional; +import java.util.concurrent.TimeUnit; import java.util.function.Consumer; import java.util.stream.Collectors; import org.apache.kafka.connect.data.Schema; @@ -251,7 +252,7 @@ protected HttpResponse sendPostRequest(final WebClient client, final Str client .post(uri) .sendBuffer(requestBody, requestFuture); - return requestFuture.get(); + return requestFuture.get(10_000L, TimeUnit.MILLISECONDS); } protected void sendPostRequest(final String uri, final Consumer> requestSender) { diff --git a/ksqldb-rest-app/src/test/java/io/confluent/ksql/rest/integration/ProcessingLogErrorMetricFunctionalTest.java b/ksqldb-rest-app/src/test/java/io/confluent/ksql/rest/integration/ProcessingLogErrorMetricFunctionalTest.java new file mode 100644 index 000000000000..c78217715682 --- /dev/null +++ b/ksqldb-rest-app/src/test/java/io/confluent/ksql/rest/integration/ProcessingLogErrorMetricFunctionalTest.java @@ -0,0 +1,174 @@ +/* + * Copyright 2022 Confluent Inc. + * + * Licensed under the Confluent Community License (the "License"; you may not use + * this file except in compliance with the License. You may obtain a copy of the + * License at + * + * http://www.confluent.io/confluent-community-license + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, WITHOUT + * WARRANTIES OF ANY KIND, either express or implied. See the License for the + * specific language governing permissions and limitations under the License. + */ + +package io.confluent.ksql.rest.integration; + +import static io.confluent.ksql.test.util.AssertEventually.assertThatEventually; +import static org.hamcrest.MatcherAssert.assertThat; +import static org.hamcrest.Matchers.equalTo; + +import com.google.common.collect.ImmutableMap; +import io.confluent.ksql.engine.KsqlEngine; +import io.confluent.ksql.integration.IntegrationTestHarness; +import io.confluent.ksql.logging.processing.MeteredProcessingLoggerFactory; +import io.confluent.ksql.rest.entity.KsqlEntity; +import io.confluent.ksql.rest.entity.Queries; +import io.confluent.ksql.rest.entity.RunningQuery; +import io.confluent.ksql.rest.server.TestKsqlRestApp; +import io.confluent.ksql.util.KsqlConfig; +import java.util.Collections; +import java.util.HashMap; +import java.util.List; +import java.util.Map; +import java.util.stream.Collectors; +import org.apache.kafka.common.MetricName; +import org.apache.kafka.common.metrics.KafkaMetric; +import org.apache.kafka.common.metrics.Metrics; +import org.junit.Before; +import org.junit.BeforeClass; +import org.junit.ClassRule; +import org.junit.Ignore; +import org.junit.Rule; +import org.junit.Test; +import org.junit.rules.RuleChain; +import org.junit.rules.Timeout; + +@Ignore +public class ProcessingLogErrorMetricFunctionalTest { + + private static final String TEST_TOPIC_NAME = "test"; + private static final String TEST_TOPIC_NAME2 = "test-topic"; + private static final ImmutableMap METRICS_TAGS = ImmutableMap.of( + "cluster.id", "cluster-1" + ); + + private static final IntegrationTestHarness TEST_HARNESS = IntegrationTestHarness.build(); + + private static final TestKsqlRestApp REST_APP = TestKsqlRestApp + .builder(TEST_HARNESS::kafkaBootstrapServers) + .withProperty(KsqlConfig.KSQL_CUSTOM_METRICS_TAGS, "cluster.id:cluster-1") + .withProperty("auto.offset.reset", "earliest") + .build(); + + @ClassRule + public static final RuleChain CHAIN = RuleChain.outerRule(TEST_HARNESS).around(REST_APP); + + @Rule + public final Timeout timeout = Timeout.seconds(60); + + private Metrics metrics; + + @BeforeClass + public static void setUpClass() { + TEST_HARNESS.ensureTopics(TEST_TOPIC_NAME); + TEST_HARNESS.ensureTopics(TEST_TOPIC_NAME2); + + RestIntegrationTestUtil.makeKsqlRequest(REST_APP, + "CREATE STREAM test_stream (f BIGINT) with (KAFKA_TOPIC='" + TEST_TOPIC_NAME + "', VALUE_FORMAT='json');" + + "CREATE STREAM test_stream2 (f BIGINT) with (KAFKA_TOPIC='" + TEST_TOPIC_NAME2 + "', VALUE_FORMAT='json');" + ); + RestIntegrationTestUtil.makeKsqlRequest(REST_APP, + "create stream test_addition_10 as select f+10 from test_stream;" + + "create stream test_addition_20 as select f+20 from test_stream2;" + ); + } + + @Before + public void setUp() { + metrics = ((KsqlEngine)REST_APP.getEngine()).getEngineMetrics().getMetrics(); + } + + @Test + public void shouldVerifyMetrics() { + + // Given: + final Map metricsTagsForQuery1DeserializerLog = new HashMap<>(METRICS_TAGS); + final Map metricsTagsForQuery2ProjectLog = new HashMap<>(METRICS_TAGS); + + final List listOfQueryId = getQueryNames(REST_APP); + assertThat(listOfQueryId.size(), equalTo(2)); + for (final String queryId:listOfQueryId) { + if (queryId.toLowerCase().contains("test_addition_10")) { + metricsTagsForQuery1DeserializerLog.put("query-id", queryId); + metricsTagsForQuery1DeserializerLog.put("logger-id", queryId + ".KsqlTopic.Source.deserializer"); + } else if (queryId.toLowerCase().contains("test_addition_20")) { + metricsTagsForQuery2ProjectLog.put("query-id", queryId); + metricsTagsForQuery2ProjectLog.put("logger-id", queryId + ".Project"); + } + } + + RestIntegrationTestUtil.makeKsqlRequest(REST_APP, + "show queries;" + ); + final MetricName processingLogErrorMetricName1 = new MetricName( + MeteredProcessingLoggerFactory.PROCESSING_LOG_ERROR_METRIC_NAME, + MeteredProcessingLoggerFactory.PROCESSING_LOG_METRICS_GROUP_NAME, + MeteredProcessingLoggerFactory.PROCESSING_LOG_METRIC_DESCRIPTION, + metricsTagsForQuery1DeserializerLog + ); + final KafkaMetric processingLogErrorMetric1 = metrics.metric(processingLogErrorMetricName1); + + final MetricName processingLogErrorMetricName2 = new MetricName( + MeteredProcessingLoggerFactory.PROCESSING_LOG_ERROR_METRIC_NAME, + MeteredProcessingLoggerFactory.PROCESSING_LOG_METRICS_GROUP_NAME, + MeteredProcessingLoggerFactory.PROCESSING_LOG_METRIC_DESCRIPTION, + metricsTagsForQuery2ProjectLog + ); + final KafkaMetric processingLogErrorMetric2 = metrics.metric(processingLogErrorMetricName2); + + // When: + + // Bad records + TEST_HARNESS.produceRecord(TEST_TOPIC_NAME, null, "{\"f\":\"string_value\"}"); + TEST_HARNESS.produceRecord(TEST_TOPIC_NAME, null, "{\"f\":5"); + TEST_HARNESS.produceRecord(TEST_TOPIC_NAME2, null, "{\"f\": null}"); + + // Then: + assertThatEventually(() -> (Double) processingLogErrorMetric1.metricValue(), equalTo(2.0)); + assertThatEventually(() -> (Double) processingLogErrorMetric2.metricValue(), equalTo(1.0)); + + // Good records shouldn't change metrics + TEST_HARNESS.produceRecord(TEST_TOPIC_NAME, null, "{\"f\":5}"); + TEST_HARNESS.produceRecord(TEST_TOPIC_NAME2, null, "{\"f\":5}"); + assertThatEventually(() -> (Double) processingLogErrorMetric1.metricValue(), equalTo(2.0)); + assertThatEventually(() -> (Double) processingLogErrorMetric2.metricValue(), equalTo(1.0)); + } + + private static List getQueryNames(final TestKsqlRestApp restApp) { + final List results = RestIntegrationTestUtil.makeKsqlRequest( + restApp, + "Show Queries;" + ); + + if (results.size() != 1) { + return Collections.emptyList(); + } + + final KsqlEntity result = results.get(0); + + if (!(result instanceof Queries)) { + return Collections.emptyList(); + } + + final List runningQueries = ((Queries) result) + .getQueries(); + + if (runningQueries.size() != 2) { + return Collections.emptyList(); + } + + return runningQueries.stream().map(query -> query.getId().toString()).collect(Collectors.toList()); + } +} diff --git a/ksqldb-rest-app/src/test/java/io/confluent/ksql/rest/integration/QueryRestartMetricFunctionalTest.java b/ksqldb-rest-app/src/test/java/io/confluent/ksql/rest/integration/QueryRestartMetricFunctionalTest.java index e6233f048f15..d569e0aa5113 100644 --- a/ksqldb-rest-app/src/test/java/io/confluent/ksql/rest/integration/QueryRestartMetricFunctionalTest.java +++ b/ksqldb-rest-app/src/test/java/io/confluent/ksql/rest/integration/QueryRestartMetricFunctionalTest.java @@ -1,5 +1,5 @@ /* - * Copyright 2019 Confluent Inc. + * Copyright 2022 Confluent Inc. * * Licensed under the Confluent Community License (the "License"; you may not use * this file except in compliance with the License. You may obtain a copy of the @@ -19,6 +19,7 @@ import static org.hamcrest.MatcherAssert.assertThat; import static org.hamcrest.Matchers.equalTo; import static org.hamcrest.Matchers.greaterThan; +import static org.hamcrest.Matchers.greaterThanOrEqualTo; import com.google.common.collect.ImmutableMap; import io.confluent.ksql.engine.KsqlEngine; @@ -34,6 +35,8 @@ import java.util.Map; import java.util.stream.Collectors; +import io.confluent.ksql.util.QueryMetadataImpl; +import io.confluent.ksql.util.QueryMetricsUtil; import org.apache.kafka.clients.consumer.ConsumerConfig; import org.apache.kafka.common.MetricName; import org.apache.kafka.common.metrics.KafkaMetric; @@ -52,8 +55,6 @@ public class QueryRestartMetricFunctionalTest { private static final String TEST_TOPIC_NAME = "test"; private static final String TEST_TOPIC_NAME2 = "test-topic"; - private static final String TEST_TOPIC_NAME3 = "test-topic-3"; - private static final String TEST_TOPIC_NAME4 = "test-topic-4"; private static final ImmutableMap METRICS_TAGS = ImmutableMap.of( "cluster.id", "cluster-1" ); @@ -67,7 +68,7 @@ public class QueryRestartMetricFunctionalTest { .withProperty(ConsumerConfig.AUTO_OFFSET_RESET_CONFIG, "earliest") .withProperty(KsqlConfig.KSQL_SHARED_RUNTIME_ENABLED, false) .withProperty(KsqlConfig.KSQL_QUERY_RETRY_BACKOFF_INITIAL_MS, 0L) - .withProperty(KsqlConfig.KSQL_QUERY_RETRY_BACKOFF_MAX_MS, 3000L) + .withProperty(KsqlConfig.KSQL_QUERY_RETRY_BACKOFF_MAX_MS, 300L) .build(); private static final TestKsqlRestApp REST_APP_SHARED_RUNTIME = TestKsqlRestApp @@ -76,6 +77,8 @@ public class QueryRestartMetricFunctionalTest { .withProperty(ConsumerConfig.AUTO_OFFSET_RESET_CONFIG, "earliest") .withProperty(KsqlConfig.KSQL_SHARED_RUNTIME_ENABLED, true) .withProperty(KsqlConfig.KSQL_SERVICE_ID_CONFIG, "another-id") + .withProperty(KsqlConfig.KSQL_QUERY_RETRY_BACKOFF_INITIAL_MS, 0L) + .withProperty(KsqlConfig.KSQL_QUERY_RETRY_BACKOFF_MAX_MS, 300L) .build(); @ClassRule @@ -88,22 +91,16 @@ public class QueryRestartMetricFunctionalTest { public static void setUpClass() throws InterruptedException { TEST_HARNESS.ensureTopics(TEST_TOPIC_NAME); TEST_HARNESS.ensureTopics(TEST_TOPIC_NAME2); - TEST_HARNESS.ensureTopics(TEST_TOPIC_NAME3); - TEST_HARNESS.ensureTopics(TEST_TOPIC_NAME4); RestIntegrationTestUtil.makeKsqlRequest(REST_APP_NO_SHARED_RUNTIME, - "CREATE STREAM test_stream (f BIGINT) with (KAFKA_TOPIC='" + TEST_TOPIC_NAME + "', VALUE_FORMAT='json');" - + "CREATE STREAM test_stream2 (f BIGINT) with (KAFKA_TOPIC='" + TEST_TOPIC_NAME2 + "', VALUE_FORMAT='json');" - ); - RestIntegrationTestUtil.makeKsqlRequest(REST_APP_NO_SHARED_RUNTIME, - "create stream test_addition_5 as select f+5 from test_stream;" - + "create stream test_addition_10 as select f+10 from test_stream;" - + "create stream test_addition_20 as select f+20 from test_stream2;" + "CREATE STREAM TEST (ID BIGINT, VALUE decimal(4,1)) WITH (kafka_topic=' " + TEST_TOPIC_NAME + "', value_format='DELIMITED');" + + "CREATE TABLE S1 as SELECT ID, sum(value) AS SUM FROM test group by id;" ); RestIntegrationTestUtil.makeKsqlRequest(REST_APP_SHARED_RUNTIME, - "CREATE STREAM test_stream (f BIGINT) with (KAFKA_TOPIC='" + TEST_TOPIC_NAME3 + "', VALUE_FORMAT='json');" - + "CREATE STREAM test_stream2 (f BIGINT) with (KAFKA_TOPIC='" + TEST_TOPIC_NAME4 + "', VALUE_FORMAT='json');" + "CREATE STREAM TEST (ID BIGINT, VALUE decimal(4,1)) WITH (kafka_topic=' " + TEST_TOPIC_NAME2 + "', value_format='DELIMITED');" + + "CREATE TABLE S2 as SELECT ID, sum(value) AS SUM FROM test group by id;" + + "CREATE TABLE S3 as SELECT ID, sum(value) AS SUM FROM test group by id;" ); } @@ -114,86 +111,58 @@ public void setUp() { } @Test - public void shouldVerifyMetricsOnNonSharedRuntimeServer() throws InterruptedException { + public void shouldVerifyMetricsOnNonSharedRuntimeServer() { // Given: - final Map metricsTagsForQuery1 = new HashMap<>(METRICS_TAGS); - final Map metricsTagsForQuery2 = new HashMap<>(METRICS_TAGS); - final Map metricsTagsForQuery3 = new HashMap<>(METRICS_TAGS); + final Map metricsTagsForQuery = new HashMap<>(METRICS_TAGS); final List listOfQueryId = RestIntegrationTestUtil.getQueryIds(REST_APP_NO_SHARED_RUNTIME); - assertThat(listOfQueryId.size(), equalTo(3)); - for (final String queryId:listOfQueryId) { - if (queryId.toLowerCase().contains("test_addition_5")) { - metricsTagsForQuery1.put("query_id", queryId); - } else if (queryId.toLowerCase().contains("test_addition_10")) { - metricsTagsForQuery2.put("query_id", queryId); - } else if (queryId.toLowerCase().contains("test_addition_20")) { - metricsTagsForQuery3.put("query_id", queryId); - } - } - - TEST_HARNESS.deleteTopics(Collections.singletonList(TEST_TOPIC_NAME)); - Thread.sleep(10000); - REST_APP_NO_SHARED_RUNTIME.stop(); + assertThat(listOfQueryId.size(), equalTo(1)); + metricsTagsForQuery.put("query-id", listOfQueryId.get(0)); // When: - REST_APP_NO_SHARED_RUNTIME.start(); - Thread.sleep(15000); + TEST_HARNESS.produceRecord(TEST_TOPIC_NAME, null, "5,900.1"); + TEST_HARNESS.produceRecord(TEST_TOPIC_NAME, null, "5,900.1"); + TEST_HARNESS.produceRecord(TEST_TOPIC_NAME, null, "5,900.1"); + System.out.println("records done"); metricsNoSharedRuntime = ((KsqlEngine)REST_APP_NO_SHARED_RUNTIME.getEngine()).getEngineMetrics().getMetrics(); - final KafkaMetric restartMetric1 = getKafkaMetric(metricsNoSharedRuntime, metricsTagsForQuery1); - final KafkaMetric restartMetric2 = getKafkaMetric(metricsNoSharedRuntime, metricsTagsForQuery2); - final KafkaMetric restartMetric3 = getKafkaMetric(metricsNoSharedRuntime, metricsTagsForQuery3); + final KafkaMetric restartMetric1 = getKafkaMetric(metricsNoSharedRuntime, metricsTagsForQuery); // Then: - assertThatEventually(() -> (Double) restartMetric1.metricValue(), greaterThan(12.0)); - assertThatEventually(() -> (Double) restartMetric2.metricValue(), greaterThan(12.0)); - assertThat(restartMetric3.metricValue(), equalTo(0.0)); + assertThatEventually(() -> (Double) restartMetric1.metricValue(), greaterThan(8.0)); } - @Ignore @Test public void shouldVerifyMetricsOnSharedRuntimeServer() throws InterruptedException { -// Given: + // Given: final Map metricsTagsForQuery1 = new HashMap<>(METRICS_TAGS); final Map metricsTagsForQuery2 = new HashMap<>(METRICS_TAGS); - final Map metricsTagsForQuery3 = new HashMap<>(METRICS_TAGS); final List listOfQueryId = RestIntegrationTestUtil.getQueryIds(REST_APP_SHARED_RUNTIME); - assertThat(listOfQueryId.size(), equalTo(3)); + assertThat(listOfQueryId.size(), equalTo(2)); for (final String queryId:listOfQueryId) { - if (queryId.toLowerCase().contains("test_addition_5")) { - metricsTagsForQuery1.put("query_id", queryId); - } else if (queryId.toLowerCase().contains("test_addition_10")) { - metricsTagsForQuery2.put("query_id", queryId); - } else if (queryId.toLowerCase().contains("test_addition_20")) { - metricsTagsForQuery3.put("query_id", queryId); + if (queryId.toLowerCase().contains("s2")) { + metricsTagsForQuery1.put("query-id", queryId); + } else if (queryId.toLowerCase().contains("s3")) { + metricsTagsForQuery2.put("query-id", queryId); } } - TEST_HARNESS.deleteTopics(Collections.singletonList(TEST_TOPIC_NAME3)); - Thread.sleep(15000); - REST_APP_SHARED_RUNTIME.stop(); - Thread.sleep(30000); - // When: - REST_APP_SHARED_RUNTIME.start(); - Thread.sleep(20000); - + TEST_HARNESS.produceRecord(TEST_TOPIC_NAME2, null, "5,900.1"); + TEST_HARNESS.produceRecord(TEST_TOPIC_NAME2, null, "5,900.1"); metricsSharedRuntime = ((KsqlEngine)REST_APP_SHARED_RUNTIME.getEngine()).getEngineMetrics().getMetrics(); final KafkaMetric restartMetric1 = getKafkaMetric(metricsSharedRuntime, metricsTagsForQuery1); final KafkaMetric restartMetric2 = getKafkaMetric(metricsSharedRuntime, metricsTagsForQuery2); - final KafkaMetric restartMetric3 = getKafkaMetric(metricsSharedRuntime, metricsTagsForQuery3); // Then: - assertThatEventually(() -> (Double) restartMetric1.metricValue(), greaterThan(12.0)); - assertThatEventually(() -> (Double) restartMetric2.metricValue(), greaterThan(12.0)); - assertThat(restartMetric3.metricValue(), equalTo(0.0)); + assertThatEventually(() -> (Double) restartMetric1.metricValue(), greaterThanOrEqualTo(1.0)); + assertThatEventually(() -> (Double) restartMetric2.metricValue(), greaterThanOrEqualTo(1.0)); } private KafkaMetric getKafkaMetric(final Metrics metrics, final Map metricsTags) { final MetricName restartMetricName3 = new MetricName( - "query-restart-total", - "query-restart-metrics", - "The total number of times that a query thread has failed and then been restarted.", + QueryMetadataImpl.QUERY_RESTART_METRIC_NAME, + QueryMetadataImpl.QUERY_RESTART_METRIC_GROUP_NAME, + QueryMetadataImpl.QUERY_RESTART_METRIC_DESCRIPTION, metricsTags ); return metrics.metric(restartMetricName3); diff --git a/ksqldb-rest-app/src/test/java/io/confluent/ksql/rest/integration/RestApiTest.java b/ksqldb-rest-app/src/test/java/io/confluent/ksql/rest/integration/RestApiTest.java index ee703c46235d..81c4997c34c3 100644 --- a/ksqldb-rest-app/src/test/java/io/confluent/ksql/rest/integration/RestApiTest.java +++ b/ksqldb-rest-app/src/test/java/io/confluent/ksql/rest/integration/RestApiTest.java @@ -35,6 +35,7 @@ import static org.apache.kafka.common.resource.ResourceType.TOPIC; import static org.apache.kafka.common.resource.ResourceType.TRANSACTIONAL_ID; import static org.hamcrest.MatcherAssert.assertThat; + import static org.hamcrest.Matchers.containsString; import static org.hamcrest.Matchers.endsWith; import static org.hamcrest.Matchers.equalTo; import static org.hamcrest.Matchers.hasSize; @@ -51,6 +52,8 @@ import io.confluent.ksql.integration.IntegrationTestHarness; import io.confluent.ksql.integration.Retry; import io.confluent.ksql.rest.ApiJsonMapper; + import io.confluent.ksql.rest.entity.AssertSchemaEntity; + import io.confluent.ksql.rest.entity.AssertTopicEntity; import io.confluent.ksql.rest.entity.CommandId; import io.confluent.ksql.rest.entity.CommandId.Action; import io.confluent.ksql.rest.entity.CommandId.Type; @@ -84,6 +87,7 @@ import io.vertx.core.http.HttpMethod; import io.vertx.core.http.HttpVersion; import io.vertx.ext.web.client.HttpResponse; + import java.time.Instant; import java.util.ArrayList; import java.util.Arrays; import java.util.Collections; @@ -187,6 +191,16 @@ public class RestApiTest { resource(TOPIC, "Y"), ops(ALL) ) + .withAcl( + NORMAL_USER, + resource(TOPIC, "Z"), + ops(ALL) + ) + .withAcl( + NORMAL_USER, + resource(TOPIC, "ABC"), + ops(ALL) + ) .withAcl( NORMAL_USER, resource(TOPIC, AGG_TABLE), @@ -225,6 +239,8 @@ public class RestApiTest { .withProperty(KsqlConfig.KSQL_QUERY_PUSH_V2_ENABLED, true) .withProperty(KsqlConfig.KSQL_QUERY_PUSH_V2_NEW_LATEST_DELAY_MS, 0L) .withProperty(KsqlConfig.KSQL_QUERY_STREAM_PULL_QUERY_ENABLED, true) + .withStaticServiceContext(TEST_HARNESS::getServiceContext) + .withProperty(KsqlConfig.SCHEMA_REGISTRY_URL_PROPERTY, "http://foo:8080") .build(); @ClassRule @@ -1039,6 +1055,182 @@ public void shouldCreateStreamWithVariableSubstitution() { assertThat(query.size(), is(1)); } + @Test + public void shouldAssertTopicExists() { + // When: + List response = makeKsqlRequest("ASSERT TOPIC " + PAGE_VIEW_TOPIC + " WITH (PARTITIONS=1) TIMEOUT 2 SECONDS;"); + + // Then: + assertThat(response.size(), is(1)); + assertThat(((AssertTopicEntity) response.get(0)).getTopicName(), is(PAGE_VIEW_TOPIC)); + assertThat(((AssertTopicEntity) response.get(0)).getExists(), is(true)); + } + + @Test + public void shouldFailToAssertNonExistantTopic() { + assertThatEventually(() -> { + try { + makeKsqlRequest("ASSERT TOPIC X WITH (PARTITIONS=1) TIMEOUT 1 SECONDS;"); + return "Should have thrown 'Topic does not exist' error."; + } catch (final Throwable t) { + return t.getMessage(); + } + }, containsString("Topic X does not exist")); + } + + @Test + public void shouldFailToAssertTopicWithWrongConfigs() { + assertThatEventually(() -> { + try { + makeKsqlRequest("ASSERT TOPIC " + PAGE_VIEW_TOPIC + " WITH (PARTITIONS='apples', REPLICAS=100, FAKE_CONFIG='Hello!') TIMEOUT 2 SECONDS;"); + return "Should have thrown config mismatch error"; + } catch (final Throwable t) { + return t.getMessage(); + } + }, containsString( + "Mismatched configuration for topic PAGEVIEW_TOPIC: For config partitions, expected apples got 1\n" + + "Mismatched configuration for topic PAGEVIEW_TOPIC: For config replicas, expected 100 got 1\n" + + "Cannot assert unknown topic property: FAKE_CONFIG")); + } + + @Test + public void shouldStopScriptOnFailedAssert() { + // When: + assertThatEventually(() -> { + try { + makeKsqlRequest("ASSERT TOPIC X TIMEOUT 2 SECONDS; CREATE STREAM Z AS SELECT * FROM " + PAGE_VIEW_STREAM + ";"); + return "Should have thrown 'Topic does not exist' error."; + } catch (final Throwable t) { + return t.getMessage(); + } + }, containsString("Topic X does not exist")); + + // Then: + final List query = ((Queries) makeKsqlRequest("SHOW QUERIES;").get(0)) + .getQueries().stream().map(RunningQuery::getQueryString) + .filter(q -> q.contains("Z AS SELECT *')")) + .collect(Collectors.toList()); + assertThat(query.size(), is(0)); + } + + @Test + public void shouldAssertTopicDoesNotExists() { + // When: + List response = makeKsqlRequest("ASSERT NOT EXISTS TOPIC X WITH (PARTITIONS=1) TIMEOUT 2 SECONDS;"); + + // Then: + assertThat(response.size(), is(1)); + assertThat(((AssertTopicEntity) response.get(0)).getTopicName(), is("X")); + assertThat(((AssertTopicEntity) response.get(0)).getExists(), is(false)); + } + + @Test + public void shouldFailToAssertTopicDoesntExist() { + assertThatEventually(() -> { + try { + makeKsqlRequest("ASSERT NOT EXISTS TOPIC " + PAGE_VIEW_TOPIC + " WITH (PARTITIONS=1) TIMEOUT 1 SECONDS;"); + return "Should have thrown 'Topioc exists' error."; + } catch (final Throwable t) { + return t.getMessage(); + } + }, containsString("Topic PAGEVIEW_TOPIC exists")); + } + + @Test + public void shouldTimeoutTheCorrectAmountOfTime() { + final long start = Instant.now().getEpochSecond(); + assertThatEventually(() -> { + try { + makeKsqlRequest("ASSERT TOPIC X WITH (PARTITIONS=1) TIMEOUT 3 SECONDS;"); + return "Should have thrown 'Topic does not exist' error."; + } catch (final Throwable t) { + return t.getMessage(); + } + }, containsString("Topic X does not exist")); + final long end = Instant.now().getEpochSecond(); + + assertThat(end - start >= 3, is(true)); + } + + @Test + public void shouldTimeoutTheDefaultAmountOfTime() { + final long start = Instant.now().getEpochSecond(); + assertThatEventually(() -> { + try { + makeKsqlRequest("ASSERT TOPIC X WITH (PARTITIONS=1);"); + return "Should have thrown 'Topic does not exist' error."; + } catch (final Throwable t) { + return t.getMessage(); + } + }, containsString("Topic X does not exist")); + final long end = Instant.now().getEpochSecond(); + + assertThat(end - start >= 1, is(true)); + assertThat(end - start <= 2, is(true)); + } + + @Test + public void shouldFailToAssertTopicWithNoAcls() { + assertThatEventually(() -> { + try { + makeKsqlRequest("ASSERT TOPIC ACLESS WITH (PARTITIONS=1);"); + return "Should have thrown an error."; + } catch (final Throwable t) { + return t.getMessage(); + } + }, containsString("Cannot check topic existence: Authorization denied to Describe on topic(s): [ACLESS]")); + } + + @Test + public void shouldAssertSchemaExists() { + // When: + makeKsqlRequest("CREATE STREAM ABC (COL INT) WITH (KAFKA_TOPIC='ABC', PARTITIONS=1, VALUE_FORMAT='AVRO');"); + List response = makeKsqlRequest("ASSERT SCHEMA SUBJECT `ABC-value`;"); + + // Then: + assertThat(response.size(), is(1)); + assertThat(((AssertSchemaEntity) response.get(0)).getSubject(), is(Optional.of("ABC-value"))); + assertThat(((AssertSchemaEntity) response.get(0)).getId(), is(Optional.empty())); + assertThat(((AssertSchemaEntity) response.get(0)).getExists(), is(true)); + } + + @Test + public void shouldFailToAssertNonExistantSchema() { + assertThatEventually(() -> { + try { + makeKsqlRequest("ASSERT SCHEMA SUBJECT blahblah TIMEOUT 2 SECONDS;"); + return "Should have thrown 'Schema does not exist' error."; + } catch (final Throwable t) { + return t.getMessage(); + } + }, containsString("Schema with subject name blahblah does not exist")); + } + + @Test + public void shouldAssertSchemaDoesNotExists() { + // When: + List response = makeKsqlRequest("ASSERT NOT EXISTS SCHEMA SUBJECT blahblah ID 4;"); + + // Then: + assertThat(response.size(), is(1)); + assertThat(((AssertSchemaEntity) response.get(0)).getSubject(), is(Optional.of("blahblah"))); + assertThat(((AssertSchemaEntity) response.get(0)).getId(), is(Optional.of(4))); + assertThat(((AssertSchemaEntity) response.get(0)).getExists(), is(false)); + } + + @Test + public void shouldFailToAssertSchemaDoesntExist() { + // Then: + assertThatEventually(() -> { + try { + makeKsqlRequest("ASSERT NOT EXISTS SCHEMA SUBJECT `ABC-value` ID 1;"); + return "Should have thrown 'schema exists' error."; + } catch (final Throwable t) { + return t.getMessage(); + } + }, containsString("Schema with subject name ABC-value id 1 exists")); + } + private boolean topicExists(final String topicName) { return getServiceContext().getTopicClient().isTopicExists(topicName); } diff --git a/ksqldb-rest-app/src/test/java/io/confluent/ksql/rest/integration/RestIntegrationTestUtil.java b/ksqldb-rest-app/src/test/java/io/confluent/ksql/rest/integration/RestIntegrationTestUtil.java index dcf35005c584..76bdeaf296bd 100644 --- a/ksqldb-rest-app/src/test/java/io/confluent/ksql/rest/integration/RestIntegrationTestUtil.java +++ b/ksqldb-rest-app/src/test/java/io/confluent/ksql/rest/integration/RestIntegrationTestUtil.java @@ -18,6 +18,7 @@ import static io.netty.handler.codec.http.HttpHeaderNames.ACCEPT; import static io.netty.handler.codec.http.HttpHeaderNames.AUTHORIZATION; import static io.netty.handler.codec.http.HttpHeaderNames.CONTENT_TYPE; +import static io.netty.handler.codec.http.HttpHeaderNames.EXPIRES; import static io.vertx.core.http.HttpMethod.POST; import static io.vertx.core.http.HttpVersion.HTTP_1_1; diff --git a/ksqldb-rest-app/src/test/java/io/confluent/ksql/rest/server/execution/AssertSchemaExecutorTest.java b/ksqldb-rest-app/src/test/java/io/confluent/ksql/rest/server/execution/AssertSchemaExecutorTest.java new file mode 100644 index 000000000000..b5429badc4b5 --- /dev/null +++ b/ksqldb-rest-app/src/test/java/io/confluent/ksql/rest/server/execution/AssertSchemaExecutorTest.java @@ -0,0 +1,304 @@ +/* + * Copyright 2022 Confluent Inc. + * + * Licensed under the Confluent Community License (the "License"; you may not use + * this file except in compliance with the License. You may obtain a copy of the + * License at + * + * http://www.confluent.io/confluent-community-license + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, WITHOUT + * WARRANTIES OF ANY KIND, either express or implied. See the License for the + * specific language governing permissions and limitations under the License. + */ + +package io.confluent.ksql.rest.server.execution; + +import static org.hamcrest.MatcherAssert.assertThat; +import static org.hamcrest.Matchers.is; +import static org.junit.Assert.assertThrows; +import static org.mockito.Mockito.mock; +import static org.mockito.Mockito.when; + +import com.google.common.collect.ImmutableList; +import com.google.common.collect.ImmutableMap; +import io.confluent.kafka.schemaregistry.client.SchemaRegistryClient; +import io.confluent.kafka.schemaregistry.client.rest.exceptions.RestClientException; +import io.confluent.ksql.KsqlExecutionContext; +import io.confluent.ksql.config.SessionConfig; +import io.confluent.ksql.parser.KsqlParser; +import io.confluent.ksql.parser.tree.AssertSchema; +import io.confluent.ksql.rest.SessionProperties; +import io.confluent.ksql.rest.entity.AssertSchemaEntity; +import io.confluent.ksql.rest.entity.KsqlEntity; +import io.confluent.ksql.rest.entity.KsqlErrorMessage; +import io.confluent.ksql.rest.server.resources.KsqlRestException; +import io.confluent.ksql.services.ServiceContext; +import io.confluent.ksql.statement.ConfiguredStatement; +import io.confluent.ksql.util.KsqlConfig; +import java.io.IOException; +import java.util.Optional; +import org.junit.Before; +import org.junit.Test; +import org.junit.runner.RunWith; +import org.mockito.Mock; +import org.mockito.junit.MockitoJUnitRunner; + +@SuppressWarnings("OptionalGetWithoutIsPresent") +@RunWith(MockitoJUnitRunner.class) +public class AssertSchemaExecutorTest { + @Mock + private KsqlExecutionContext engine; + @Mock + private ServiceContext serviceContext; + @Mock + private SchemaRegistryClient srClient; + private final KsqlConfig ksqlConfig = new KsqlConfig(ImmutableMap.of()); + + @Before + public void setUp() throws IOException, RestClientException { + when(engine.getKsqlConfig()).thenReturn(ksqlConfig); + when(serviceContext.getSchemaRegistryClient()).thenReturn(srClient); + + // These are the schemas that don't exist + when(srClient.getSchemaById(100)).thenThrow(new RestClientException("", 404, 40403)); + when(srClient.getLatestSchemaMetadata("abc")).thenThrow(new RestClientException("", 404, 40403)); + when(srClient.getAllSubjectsById(100)).thenThrow(new RestClientException("", 404, 40403)); + + when(srClient.getAllSubjectsById(500)).thenReturn(ImmutableList.of("abc")); + when(srClient.getSchemaById(222)).thenThrow(new IOException("something happened!")); + } + + @Test + public void shouldAssertSchemaBySubject() { + // Given + final AssertSchema assertSchema = new AssertSchema(Optional.empty(), Optional.of("subjectName"), Optional.empty(), Optional.empty(), true); + final ConfiguredStatement statement = ConfiguredStatement + .of(KsqlParser.PreparedStatement.of("", assertSchema), + SessionConfig.of(ksqlConfig, ImmutableMap.of())); + + // When: + final Optional entity = AssertSchemaExecutor + .execute(statement, mock(SessionProperties.class), engine, serviceContext).getEntity(); + + // Then: + assertThat("expected response!", entity.isPresent()); + assertThat(((AssertSchemaEntity) entity.get()).getSubject(), is(Optional.of("subjectName"))); + assertThat(((AssertSchemaEntity) entity.get()).getId(), is(Optional.empty())); + assertThat(((AssertSchemaEntity) entity.get()).getExists(), is(true)); + } + + @Test + public void shouldAssertSchemaById() { + // Given + final AssertSchema assertSchema = new AssertSchema(Optional.empty(), Optional.empty(), Optional.of(44), Optional.empty(), true); + final ConfiguredStatement statement = ConfiguredStatement + .of(KsqlParser.PreparedStatement.of("", assertSchema), + SessionConfig.of(ksqlConfig, ImmutableMap.of())); + + // When: + final Optional entity = AssertSchemaExecutor + .execute(statement, mock(SessionProperties.class), engine, serviceContext).getEntity(); + + // Then: + assertThat("expected response!", entity.isPresent()); + assertThat(((AssertSchemaEntity) entity.get()).getSubject(), is(Optional.empty())); + assertThat(((AssertSchemaEntity) entity.get()).getId(), is(Optional.of(44))); + assertThat(((AssertSchemaEntity) entity.get()).getExists(), is(true)); + } + + @Test + public void shouldAssertSchemaBySubjectAndId() { + // Given + final AssertSchema assertSchema = new AssertSchema(Optional.empty(), Optional.of("abc"), Optional.of(500), Optional.empty(), true); + final ConfiguredStatement statement = ConfiguredStatement + .of(KsqlParser.PreparedStatement.of("", assertSchema), + SessionConfig.of(ksqlConfig, ImmutableMap.of())); + + // When: + final Optional entity = AssertSchemaExecutor + .execute(statement, mock(SessionProperties.class), engine, serviceContext).getEntity(); + + // Then: + assertThat("expected response!", entity.isPresent()); + assertThat(((AssertSchemaEntity) entity.get()).getSubject(), is(Optional.of("abc"))); + assertThat(((AssertSchemaEntity) entity.get()).getId(), is(Optional.of(500))); + assertThat(((AssertSchemaEntity) entity.get()).getExists(), is(true)); + } + + @Test + public void shouldFailToAssertSchemaBySubject() { + // Given + final AssertSchema assertSchema = new AssertSchema(Optional.empty(), Optional.of("abc"), Optional.empty(), Optional.empty(), true); + final ConfiguredStatement statement = ConfiguredStatement + .of(KsqlParser.PreparedStatement.of("", assertSchema), + SessionConfig.of(ksqlConfig, ImmutableMap.of())); + + // When: + final KsqlRestException e = assertThrows(KsqlRestException.class, () -> + AssertSchemaExecutor.execute(statement, mock(SessionProperties.class), engine, serviceContext)); + + // Then: + assertThat(e.getResponse().getStatus(), is(417)); + assertThat(((KsqlErrorMessage) e.getResponse().getEntity()).getMessage(), is("Schema with subject name abc does not exist")); + } + + @Test + public void shouldFailToAssertSchemaById() { + // Given + final AssertSchema assertSchema = new AssertSchema(Optional.empty(), Optional.empty(), Optional.of(100), Optional.empty(), true); + final ConfiguredStatement statement = ConfiguredStatement + .of(KsqlParser.PreparedStatement.of("", assertSchema), + SessionConfig.of(ksqlConfig, ImmutableMap.of())); + + // When: + final KsqlRestException e = assertThrows(KsqlRestException.class, () -> + AssertSchemaExecutor.execute(statement, mock(SessionProperties.class), engine, serviceContext)); + + // Then: + assertThat(e.getResponse().getStatus(), is(417)); + assertThat(((KsqlErrorMessage) e.getResponse().getEntity()).getMessage(), is("Schema with id 100 does not exist")); + } + + @Test + public void shouldFailToAssertSchemaBySubjectAndId() { + // Given + final AssertSchema assertSchema = new AssertSchema(Optional.empty(), Optional.of("def"), Optional.of(500), Optional.empty(), true); + final ConfiguredStatement statement = ConfiguredStatement + .of(KsqlParser.PreparedStatement.of("", assertSchema), + SessionConfig.of(ksqlConfig, ImmutableMap.of())); + + // When: + final KsqlRestException e = assertThrows(KsqlRestException.class, () -> + AssertSchemaExecutor.execute(statement, mock(SessionProperties.class), engine, serviceContext)); + + // Then: + assertThat(e.getResponse().getStatus(), is(417)); + assertThat(((KsqlErrorMessage) e.getResponse().getEntity()).getMessage(), is("Schema with subject name def id 500 does not exist")); + } + + @Test + public void shouldAssertNotExistSchemaBySubject() { + // Given + final AssertSchema assertSchema = new AssertSchema(Optional.empty(), Optional.of("abc"), Optional.empty(), Optional.empty(), false); + final ConfiguredStatement statement = ConfiguredStatement + .of(KsqlParser.PreparedStatement.of("", assertSchema), + SessionConfig.of(ksqlConfig, ImmutableMap.of())); + + // When: + final Optional entity = AssertSchemaExecutor + .execute(statement, mock(SessionProperties.class), engine, serviceContext).getEntity(); + + // Then: + assertThat("expected response!", entity.isPresent()); + assertThat(((AssertSchemaEntity) entity.get()).getSubject(), is(Optional.of("abc"))); + assertThat(((AssertSchemaEntity) entity.get()).getId(), is(Optional.empty())); + assertThat(((AssertSchemaEntity) entity.get()).getExists(), is(false)); + } + + @Test + public void shouldAssertNotExistSchemaById() { + // Given + final AssertSchema assertSchema = new AssertSchema(Optional.empty(), Optional.empty(), Optional.of(100), Optional.empty(), false); + final ConfiguredStatement statement = ConfiguredStatement + .of(KsqlParser.PreparedStatement.of("", assertSchema), + SessionConfig.of(ksqlConfig, ImmutableMap.of())); + + // When: + final Optional entity = AssertSchemaExecutor + .execute(statement, mock(SessionProperties.class), engine, serviceContext).getEntity(); + + // Then: + assertThat("expected response!", entity.isPresent()); + assertThat(((AssertSchemaEntity) entity.get()).getSubject(), is(Optional.empty())); + assertThat(((AssertSchemaEntity) entity.get()).getId(), is(Optional.of(100))); + assertThat(((AssertSchemaEntity) entity.get()).getExists(), is(false)); + } + + @Test + public void shouldAssertNotExistSchemaBySubjectAndId() { + // Given + final AssertSchema assertSchema = new AssertSchema(Optional.empty(), Optional.of("def"), Optional.of(100), Optional.empty(), false); + final ConfiguredStatement statement = ConfiguredStatement + .of(KsqlParser.PreparedStatement.of("", assertSchema), + SessionConfig.of(ksqlConfig, ImmutableMap.of())); + + // When: + final Optional entity = AssertSchemaExecutor + .execute(statement, mock(SessionProperties.class), engine, serviceContext).getEntity(); + + // Then: + assertThat("expected response!", entity.isPresent()); + assertThat(((AssertSchemaEntity) entity.get()).getSubject(), is(Optional.of("def"))); + assertThat(((AssertSchemaEntity) entity.get()).getId(), is(Optional.of(100))); + assertThat(((AssertSchemaEntity) entity.get()).getExists(), is(false)); + } + + @Test + public void shouldFailToAssertNotExistSchemaBySubject() { + // Given + final AssertSchema assertSchema = new AssertSchema(Optional.empty(), Optional.of("subjectName"), Optional.empty(), Optional.empty(), false); + final ConfiguredStatement statement = ConfiguredStatement + .of(KsqlParser.PreparedStatement.of("", assertSchema), + SessionConfig.of(ksqlConfig, ImmutableMap.of())); + + // When: + final KsqlRestException e = assertThrows(KsqlRestException.class, () -> + AssertSchemaExecutor.execute(statement, mock(SessionProperties.class), engine, serviceContext)); + + // Then: + assertThat(e.getResponse().getStatus(), is(417)); + assertThat(((KsqlErrorMessage) e.getResponse().getEntity()).getMessage(), is("Schema with subject name subjectName exists")); + } + + @Test + public void shouldFailToAssertNotExistSchemaById() { + // Given + final AssertSchema assertSchema = new AssertSchema(Optional.empty(), Optional.empty(), Optional.of(2), Optional.empty(), false); + final ConfiguredStatement statement = ConfiguredStatement + .of(KsqlParser.PreparedStatement.of("", assertSchema), + SessionConfig.of(ksqlConfig, ImmutableMap.of())); + + // When: + final KsqlRestException e = assertThrows(KsqlRestException.class, () -> + AssertSchemaExecutor.execute(statement, mock(SessionProperties.class), engine, serviceContext)); + + // Then: + assertThat(e.getResponse().getStatus(), is(417)); + assertThat(((KsqlErrorMessage) e.getResponse().getEntity()).getMessage(), is("Schema with id 2 exists")); + } + + @Test + public void shouldFailToAssertNotExistSchemaBySubjectAndId() { + // Given + final AssertSchema assertSchema = new AssertSchema(Optional.empty(), Optional.of("abc"), Optional.of(500), Optional.empty(), false); + final ConfiguredStatement statement = ConfiguredStatement + .of(KsqlParser.PreparedStatement.of("", assertSchema), + SessionConfig.of(ksqlConfig, ImmutableMap.of())); + + // When: + final KsqlRestException e = assertThrows(KsqlRestException.class, () -> + AssertSchemaExecutor.execute(statement, mock(SessionProperties.class), engine, serviceContext)); + + // Then: + assertThat(e.getResponse().getStatus(), is(417)); + assertThat(((KsqlErrorMessage) e.getResponse().getEntity()).getMessage(), is("Schema with subject name abc id 500 exists")); + } + + @Test + public void shouldFailWhenSRClientFails() { + // Given + final AssertSchema assertSchema = new AssertSchema(Optional.empty(), Optional.empty(), Optional.of(222), Optional.empty(), false); + final ConfiguredStatement statement = ConfiguredStatement + .of(KsqlParser.PreparedStatement.of("", assertSchema), + SessionConfig.of(ksqlConfig, ImmutableMap.of())); + + // When: + final KsqlRestException e = assertThrows(KsqlRestException.class, () -> + AssertSchemaExecutor.execute(statement, mock(SessionProperties.class), engine, serviceContext)); + + // Then: + assertThat(((KsqlErrorMessage) e.getResponse().getEntity()).getMessage(), is("Cannot check schema existence: something happened!")); + } +} diff --git a/ksqldb-rest-app/src/test/java/io/confluent/ksql/rest/server/execution/AssertTopicExecutorTest.java b/ksqldb-rest-app/src/test/java/io/confluent/ksql/rest/server/execution/AssertTopicExecutorTest.java new file mode 100644 index 000000000000..cbae273f45c0 --- /dev/null +++ b/ksqldb-rest-app/src/test/java/io/confluent/ksql/rest/server/execution/AssertTopicExecutorTest.java @@ -0,0 +1,178 @@ +/* + * Copyright 2022 Confluent Inc. + * + * Licensed under the Confluent Community License (the "License"; you may not use + * this file except in compliance with the License. You may obtain a copy of the + * License at + * + * http://www.confluent.io/confluent-community-license + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, WITHOUT + * WARRANTIES OF ANY KIND, either express or implied. See the License for the + * specific language governing permissions and limitations under the License. + */ + +package io.confluent.ksql.rest.server.execution; + +import static org.hamcrest.MatcherAssert.assertThat; +import static org.hamcrest.Matchers.is; +import static org.junit.Assert.assertThrows; +import static org.mockito.Mockito.mock; +import static org.mockito.Mockito.when; + +import com.google.common.collect.ImmutableList; +import com.google.common.collect.ImmutableMap; +import io.confluent.ksql.KsqlExecutionContext; +import io.confluent.ksql.config.SessionConfig; +import io.confluent.ksql.execution.expression.tree.IntegerLiteral; +import io.confluent.ksql.execution.expression.tree.Literal; +import io.confluent.ksql.parser.KsqlParser; +import io.confluent.ksql.parser.tree.AssertTopic; +import io.confluent.ksql.rest.SessionProperties; +import io.confluent.ksql.rest.entity.AssertTopicEntity; +import io.confluent.ksql.rest.entity.KsqlEntity; +import io.confluent.ksql.rest.entity.KsqlErrorMessage; +import io.confluent.ksql.rest.server.resources.KsqlRestException; +import io.confluent.ksql.services.KafkaTopicClient; +import io.confluent.ksql.services.ServiceContext; +import io.confluent.ksql.statement.ConfiguredStatement; +import io.confluent.ksql.util.KsqlConfig; +import java.util.Map; +import java.util.Optional; +import org.apache.kafka.clients.admin.TopicDescription; +import org.apache.kafka.common.TopicPartitionInfo; +import org.junit.Before; +import org.junit.Test; +import org.junit.runner.RunWith; +import org.mockito.Mock; +import org.mockito.junit.MockitoJUnitRunner; + +@SuppressWarnings("OptionalGetWithoutIsPresent") +@RunWith(MockitoJUnitRunner.class) +public class AssertTopicExecutorTest { + @Mock + private KsqlExecutionContext engine; + @Mock + private ServiceContext serviceContext; + @Mock + private KafkaTopicClient topicClient; + @Mock + private TopicDescription topicDescription; + @Mock + private TopicPartitionInfo partition; + private final KsqlConfig ksqlConfig = new KsqlConfig(ImmutableMap.of()); + + @Before + public void setUp() { + when(engine.getKsqlConfig()).thenReturn(ksqlConfig); + when(serviceContext.getTopicClient()).thenReturn(topicClient); + when(topicClient.isTopicExists("topicName")) + .thenReturn(true); + when(topicClient.describeTopic("topicName")) + .thenReturn(topicDescription); + when(topicDescription.partitions()) + .thenReturn(ImmutableList.of(partition)); + when(partition.replicas()) + .thenReturn(ImmutableList.of()); + } + + @Test + public void shouldAssertTopic() { + // Given: + final Map configs = ImmutableMap.of( + "partitions", new IntegerLiteral(1), "replicas", new IntegerLiteral(0)); + final AssertTopic assertTopic = new AssertTopic(Optional.empty(), "topicName", configs, Optional.empty(), true); + final ConfiguredStatement statement = ConfiguredStatement + .of(KsqlParser.PreparedStatement.of("", assertTopic), + SessionConfig.of(ksqlConfig, ImmutableMap.of())); + + // When: + final Optional entity = AssertTopicExecutor + .execute(statement, mock(SessionProperties.class), engine, serviceContext).getEntity(); + + // Then: + assertThat("expected response!", entity.isPresent()); + assertThat(((AssertTopicEntity) entity.get()).getTopicName(), is("topicName")); + assertThat(((AssertTopicEntity) entity.get()).getExists(), is(true)); + } + + @Test + public void shouldFailToAssertNonExistingTopic() { + // Given: + final Map configs = ImmutableMap.of( + "partitions", new IntegerLiteral(1), "replicas", new IntegerLiteral(0)); + final AssertTopic assertTopic = new AssertTopic(Optional.empty(), "fakeTopic", configs, Optional.empty(), true); + final ConfiguredStatement statement = ConfiguredStatement + .of(KsqlParser.PreparedStatement.of("", assertTopic), + SessionConfig.of(ksqlConfig, ImmutableMap.of())); + + // When: + final KsqlRestException e = assertThrows(KsqlRestException.class, + () -> AssertTopicExecutor.execute(statement, mock(SessionProperties.class), engine, serviceContext)); + + // Then: + assertThat(e.getResponse().getStatus(), is(417)); + assertThat(((KsqlErrorMessage) e.getResponse().getEntity()).getMessage(), is("Topic fakeTopic does not exist")); + } + + @Test + public void shouldFailToAssertWrongConfigs() { + // Given: + final Map configs = ImmutableMap.of( + "partitions", new IntegerLiteral(10), "replicas", new IntegerLiteral(10), "abc", new IntegerLiteral(23)); + final AssertTopic assertTopic = new AssertTopic(Optional.empty(), "topicName", configs, Optional.empty(), true); + final ConfiguredStatement statement = ConfiguredStatement + .of(KsqlParser.PreparedStatement.of("", assertTopic), + SessionConfig.of(ksqlConfig, ImmutableMap.of())); + + // When: + final KsqlRestException e = assertThrows(KsqlRestException.class, + () -> AssertTopicExecutor.execute(statement, mock(SessionProperties.class), engine, serviceContext)); + + // Then: + assertThat(e.getResponse().getStatus(), is(417)); + assertThat(((KsqlErrorMessage) e.getResponse().getEntity()).getMessage(), is("Mismatched configuration for topic topicName: For config partitions, expected 10 got 1\n" + + "Mismatched configuration for topic topicName: For config replicas, expected 10 got 0\n" + + "Cannot assert unknown topic property: abc")); + } + + @Test + public void shouldAssertTopicNotExists() { + // Given: + final Map configs = ImmutableMap.of( + "partitions", new IntegerLiteral(1), "replicas", new IntegerLiteral(0)); + final AssertTopic assertTopic = new AssertTopic(Optional.empty(), "fakeTopic", configs, Optional.empty(), false); + final ConfiguredStatement statement = ConfiguredStatement + .of(KsqlParser.PreparedStatement.of("", assertTopic), + SessionConfig.of(ksqlConfig, ImmutableMap.of())); + + // When: + final Optional entity = AssertTopicExecutor + .execute(statement, mock(SessionProperties.class), engine, serviceContext).getEntity(); + + // Then: + assertThat("expected response!", entity.isPresent()); + assertThat(((AssertTopicEntity) entity.get()).getTopicName(), is("fakeTopic")); + assertThat(((AssertTopicEntity) entity.get()).getExists(), is(false)); + } + + @Test + public void shouldFailToAssertTopicNotExists() { + // Given: + final Map configs = ImmutableMap.of( + "partitions", new IntegerLiteral(1), "replicas", new IntegerLiteral(0)); + final AssertTopic assertTopic = new AssertTopic(Optional.empty(), "topicName", configs, Optional.empty(), false); + final ConfiguredStatement statement = ConfiguredStatement + .of(KsqlParser.PreparedStatement.of("", assertTopic), + SessionConfig.of(ksqlConfig, ImmutableMap.of())); + + // When: + final KsqlRestException e = assertThrows(KsqlRestException.class, + () -> AssertTopicExecutor.execute(statement, mock(SessionProperties.class), engine, serviceContext)); + + // Then: + assertThat(e.getResponse().getStatus(), is(417)); + assertThat(((KsqlErrorMessage) e.getResponse().getEntity()).getMessage(), is("Topic topicName exists")); + } +} diff --git a/ksqldb-rest-client/src/main/java/io/confluent/ksql/rest/client/KsqlTargetUtil.java b/ksqldb-rest-client/src/main/java/io/confluent/ksql/rest/client/KsqlTargetUtil.java index f589a6bd0348..e71937dd0e55 100644 --- a/ksqldb-rest-client/src/main/java/io/confluent/ksql/rest/client/KsqlTargetUtil.java +++ b/ksqldb-rest-client/src/main/java/io/confluent/ksql/rest/client/KsqlTargetUtil.java @@ -16,6 +16,7 @@ package io.confluent.ksql.rest.client; import static io.confluent.ksql.rest.client.KsqlClientUtil.deserialize; +import static io.confluent.ksql.util.BytesUtils.toJsonMsg; import com.google.common.base.Strings; import com.google.common.collect.Streams; @@ -55,7 +56,7 @@ public static List toRows(final Buffer buff) { || (i < buff.length() && buff.getByte(i) == (byte) '\n')) { if (begin != i) { // Ignore random newlines - the server can send these final Buffer sliced = buff.slice(begin, i); - final Buffer tidied = StreamPublisher.toJsonMsg(sliced, true); + final Buffer tidied = toJsonMsg(sliced, true); if (tidied.length() > 0) { final StreamedRow row = deserialize(tidied, StreamedRow.class); rows.add(row); diff --git a/ksqldb-rest-client/src/main/java/io/confluent/ksql/rest/client/StreamPublisher.java b/ksqldb-rest-client/src/main/java/io/confluent/ksql/rest/client/StreamPublisher.java index 9bc1d92a84f0..f2de4f43fa60 100644 --- a/ksqldb-rest-client/src/main/java/io/confluent/ksql/rest/client/StreamPublisher.java +++ b/ksqldb-rest-client/src/main/java/io/confluent/ksql/rest/client/StreamPublisher.java @@ -15,6 +15,8 @@ package io.confluent.ksql.rest.client; +import static io.confluent.ksql.util.BytesUtils.toJsonMsg; + import io.confluent.ksql.reactive.BufferedPublisher; import io.vertx.core.Context; import io.vertx.core.buffer.Buffer; @@ -28,24 +30,6 @@ public class StreamPublisher extends BufferedPublisher { private final HttpClientResponse response; private boolean drainHandlerSet; - public static Buffer toJsonMsg(final Buffer responseLine, final boolean stripArray) { - - int start = 0; - int end = responseLine.length() - 1; - if (stripArray) { - if (responseLine.getByte(0) == (byte) '[') { - start = 1; - } - if (responseLine.getByte(end) == (byte) ']') { - end -= 1; - } - } - if (end > 0 && responseLine.getByte(end) == (byte) ',') { - end -= 1; - } - return responseLine.slice(start, end + 1); - } - StreamPublisher(final Context context, final HttpClientResponse response, final Function mapper, final CompletableFuture bodyFuture, diff --git a/ksqldb-rest-model/src/main/java/io/confluent/ksql/rest/Errors.java b/ksqldb-rest-model/src/main/java/io/confluent/ksql/rest/Errors.java index 89191e011d34..521b3eb84a26 100644 --- a/ksqldb-rest-model/src/main/java/io/confluent/ksql/rest/Errors.java +++ b/ksqldb-rest-model/src/main/java/io/confluent/ksql/rest/Errors.java @@ -17,6 +17,7 @@ import static io.netty.handler.codec.http.HttpHeaderNames.RETRY_AFTER; import static io.netty.handler.codec.http.HttpResponseStatus.BAD_REQUEST; +import static io.netty.handler.codec.http.HttpResponseStatus.EXPECTATION_FAILED; import static io.netty.handler.codec.http.HttpResponseStatus.FORBIDDEN; import static io.netty.handler.codec.http.HttpResponseStatus.INTERNAL_SERVER_ERROR; import static io.netty.handler.codec.http.HttpResponseStatus.MISDIRECTED_REQUEST; @@ -75,6 +76,8 @@ public final class Errors { public static final int ERROR_CODE_TOO_MANY_REQUESTS = toErrorCode(TOO_MANY_REQUESTS.code()); + public static final int ERROR_CODE_ASSERTION_FAILED = toErrorCode(EXPECTATION_FAILED.code()); + private final ErrorMessages errorMessages; public static int toStatusCode(final int errorCode) { @@ -218,6 +221,13 @@ public static EndpointResponse tooManyRequests(final String msg) { .build(); } + public static EndpointResponse assertionFailedError(final String errorMsg) { + return EndpointResponse.create() + .status(EXPECTATION_FAILED.code()) + .entity(new KsqlErrorMessage(ERROR_CODE_ASSERTION_FAILED, errorMsg)) + .build(); + } + public Errors(final ErrorMessages errorMessages) { this.errorMessages = Objects.requireNonNull(errorMessages, "errorMessages"); } diff --git a/ksqldb-rest-model/src/main/java/io/confluent/ksql/rest/entity/AssertSchemaEntity.java b/ksqldb-rest-model/src/main/java/io/confluent/ksql/rest/entity/AssertSchemaEntity.java new file mode 100644 index 000000000000..c3b84490477f --- /dev/null +++ b/ksqldb-rest-model/src/main/java/io/confluent/ksql/rest/entity/AssertSchemaEntity.java @@ -0,0 +1,61 @@ +/* + * Copyright 2022 Confluent Inc. + * + * Licensed under the Confluent Community License (the "License"); you may not use + * this file except in compliance with the License. You may obtain a copy of the + * License at + * + * http://www.confluent.io/confluent-community-license + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, WITHOUT + * WARRANTIES OF ANY KIND, either express or implied. See the License for the + * specific language governing permissions and limitations under the License. + */ + +package io.confluent.ksql.rest.entity; + +import com.fasterxml.jackson.annotation.JsonIgnoreProperties; +import com.fasterxml.jackson.annotation.JsonProperty; +import java.util.Objects; +import java.util.Optional; + +@JsonIgnoreProperties(ignoreUnknown = true) +public class AssertSchemaEntity extends KsqlEntity { + private final Optional subject; + private final Optional id; + private final boolean exists; + + public AssertSchemaEntity( + @JsonProperty("statementText") final String statementText, + @JsonProperty("subject") final Optional subject, + @JsonProperty("id") final Optional id, + @JsonProperty("exists") final boolean exists + ) { + super(statementText); + this.subject = Objects.requireNonNull(subject, "subject"); + this.id = Objects.requireNonNull(id, "id"); + this.exists = exists; + } + + public Optional getSubject() { + return subject; + } + + public Optional getId() { + return id; + } + + public boolean getExists() { + return exists; + } + + @Override + public String toString() { + return "AssertSchemaEntity{" + + "subject='" + subject + '\'' + + ", id=" + id + + ", exists=" + exists + + '}'; + } +} diff --git a/ksqldb-rest-model/src/main/java/io/confluent/ksql/rest/entity/AssertTopicEntity.java b/ksqldb-rest-model/src/main/java/io/confluent/ksql/rest/entity/AssertTopicEntity.java new file mode 100644 index 000000000000..e895616f7b00 --- /dev/null +++ b/ksqldb-rest-model/src/main/java/io/confluent/ksql/rest/entity/AssertTopicEntity.java @@ -0,0 +1,52 @@ +/* + * Copyright 2022 Confluent Inc. + * + * Licensed under the Confluent Community License (the "License"); you may not use + * this file except in compliance with the License. You may obtain a copy of the + * License at + * + * http://www.confluent.io/confluent-community-license + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, WITHOUT + * WARRANTIES OF ANY KIND, either express or implied. See the License for the + * specific language governing permissions and limitations under the License. + */ + +package io.confluent.ksql.rest.entity; + +import com.fasterxml.jackson.annotation.JsonIgnoreProperties; +import com.fasterxml.jackson.annotation.JsonProperty; +import java.util.Objects; + +@JsonIgnoreProperties(ignoreUnknown = true) +public class AssertTopicEntity extends KsqlEntity { + private final String topicName; + private final boolean exists; + + public AssertTopicEntity( + @JsonProperty("statementText") final String statementText, + @JsonProperty("topicName") final String topicName, + @JsonProperty("exists") final boolean exists + ) { + super(statementText); + this.topicName = Objects.requireNonNull(topicName, "topicName"); + this.exists = exists; + } + + public String getTopicName() { + return topicName; + } + + public boolean getExists() { + return exists; + } + + @Override + public String toString() { + return "AssertTopicEntity{" + + "topicName='" + topicName + '\'' + + ", exists=" + exists + + '}'; + } +} diff --git a/ksqldb-rest-model/src/main/java/io/confluent/ksql/rest/entity/KsqlEntity.java b/ksqldb-rest-model/src/main/java/io/confluent/ksql/rest/entity/KsqlEntity.java index 4ebcaabbfddb..d6ddfa98ad2b 100644 --- a/ksqldb-rest-model/src/main/java/io/confluent/ksql/rest/entity/KsqlEntity.java +++ b/ksqldb-rest-model/src/main/java/io/confluent/ksql/rest/entity/KsqlEntity.java @@ -51,7 +51,9 @@ @JsonSubTypes.Type(value = TypeList.class, name = "type_list"), @JsonSubTypes.Type(value = WarningEntity.class, name = "warning_entity"), @JsonSubTypes.Type(value = VariablesList.class, name = "variables"), - @JsonSubTypes.Type(value = TerminateQueryEntity.class, name = "terminate_query") + @JsonSubTypes.Type(value = TerminateQueryEntity.class, name = "terminate_query"), + @JsonSubTypes.Type(value = AssertTopicEntity.class, name = "assert_topic"), + @JsonSubTypes.Type(value = AssertSchemaEntity.class, name = "assert_schema") }) public abstract class KsqlEntity { private final String statementText; diff --git a/ksqldb-serde/src/main/java/io/confluent/ksql/serde/FormatFactory.java b/ksqldb-serde/src/main/java/io/confluent/ksql/serde/FormatFactory.java index cc02ae85f90d..c2361b5719da 100644 --- a/ksqldb-serde/src/main/java/io/confluent/ksql/serde/FormatFactory.java +++ b/ksqldb-serde/src/main/java/io/confluent/ksql/serde/FormatFactory.java @@ -22,20 +22,23 @@ import io.confluent.ksql.serde.kafka.KafkaFormat; import io.confluent.ksql.serde.none.NoneFormat; import io.confluent.ksql.serde.protobuf.ProtobufFormat; +import io.confluent.ksql.serde.protobuf.ProtobufNoSRFormat; import io.confluent.ksql.util.KsqlException; /** * A class containing the builtin supported formats in ksqlDB. */ +@SuppressWarnings("checkstyle:ClassDataAbstractionCoupling") public final class FormatFactory { - public static final Format AVRO = new AvroFormat(); - public static final Format JSON = new JsonFormat(); - public static final Format JSON_SR = new JsonSchemaFormat(); - public static final Format PROTOBUF = new ProtobufFormat(); - public static final Format KAFKA = new KafkaFormat(); - public static final Format DELIMITED = new DelimitedFormat(); - public static final Format NONE = new NoneFormat(); + public static final Format AVRO = new AvroFormat(); + public static final Format JSON = new JsonFormat(); + public static final Format JSON_SR = new JsonSchemaFormat(); + public static final Format PROTOBUF = new ProtobufFormat(); + public static final Format PROTOBUF_NOSR = new ProtobufNoSRFormat(); + public static final Format KAFKA = new KafkaFormat(); + public static final Format DELIMITED = new DelimitedFormat(); + public static final Format NONE = new NoneFormat(); private FormatFactory() { } @@ -53,13 +56,14 @@ public static Format of(final FormatInfo formatInfo) { public static Format fromName(final String name) { switch (name.toUpperCase()) { - case AvroFormat.NAME: return AVRO; - case JsonFormat.NAME: return JSON; - case JsonSchemaFormat.NAME: return JSON_SR; - case ProtobufFormat.NAME: return PROTOBUF; - case KafkaFormat.NAME: return KAFKA; - case DelimitedFormat.NAME: return DELIMITED; - case NoneFormat.NAME: return NONE; + case AvroFormat.NAME: return AVRO; + case JsonFormat.NAME: return JSON; + case JsonSchemaFormat.NAME: return JSON_SR; + case ProtobufFormat.NAME: return PROTOBUF; + case ProtobufNoSRFormat.NAME: return PROTOBUF_NOSR; + case KafkaFormat.NAME: return KAFKA; + case DelimitedFormat.NAME: return DELIMITED; + case NoneFormat.NAME: return NONE; default: throw new KsqlException("Unknown format: " + name); } diff --git a/ksqldb-serde/src/main/java/io/confluent/ksql/serde/GenericKeySerDe.java b/ksqldb-serde/src/main/java/io/confluent/ksql/serde/GenericKeySerDe.java index c7c60d9d6028..1883fe23abf3 100644 --- a/ksqldb-serde/src/main/java/io/confluent/ksql/serde/GenericKeySerDe.java +++ b/ksqldb-serde/src/main/java/io/confluent/ksql/serde/GenericKeySerDe.java @@ -47,14 +47,26 @@ public final class GenericKeySerDe implements KeySerdeFactory { private final GenericSerdeFactory innerFactory; + private Optional queryId; public GenericKeySerDe() { - this(new GenericSerdeFactory()); + this(new GenericSerdeFactory(), Optional.empty()); + } + + public GenericKeySerDe(final String queryId) { + this( + new GenericSerdeFactory(), + Optional.of(queryId) + ); } @VisibleForTesting - GenericKeySerDe(final GenericSerdeFactory innerFactory) { + GenericKeySerDe( + final GenericSerdeFactory innerFactory, + final Optional queryId + ) { this.innerFactory = Objects.requireNonNull(innerFactory, "innerFactory"); + this.queryId = queryId; } @Override @@ -121,8 +133,11 @@ private Serde createInner( final Serde genericKeySerde = toGenericKeySerde(formatSerde, schema); - final Serde loggingSerde = innerFactory - .wrapInLoggingSerde(genericKeySerde, loggerNamePrefix, processingLogContext); + final Serde loggingSerde = innerFactory.wrapInLoggingSerde( + genericKeySerde, + loggerNamePrefix, + processingLogContext, + queryId); final Serde serde = tracker .map(callback -> innerFactory.wrapInTrackingSerde(loggingSerde, callback)) diff --git a/ksqldb-serde/src/main/java/io/confluent/ksql/serde/GenericRowSerDe.java b/ksqldb-serde/src/main/java/io/confluent/ksql/serde/GenericRowSerDe.java index 8b3733a35f7c..3c3916fa7bc8 100644 --- a/ksqldb-serde/src/main/java/io/confluent/ksql/serde/GenericRowSerDe.java +++ b/ksqldb-serde/src/main/java/io/confluent/ksql/serde/GenericRowSerDe.java @@ -55,14 +55,26 @@ public final class GenericRowSerDe implements ValueSerdeFactory { private static final int ADDITIONAL_CAPACITY = 4; private final GenericSerdeFactory innerFactory; + private final Optional queryId; public GenericRowSerDe() { - this(new GenericSerdeFactory()); + this(new GenericSerdeFactory(), Optional.empty()); + } + + public GenericRowSerDe(final String queryId) { + this( + new GenericSerdeFactory(), + Optional.of(queryId) + ); } @VisibleForTesting - GenericRowSerDe(final GenericSerdeFactory innerFactory) { + GenericRowSerDe( + final GenericSerdeFactory innerFactory, + final Optional queryId + ) { this.innerFactory = Objects.requireNonNull(innerFactory, "innerFactory"); + this.queryId = queryId; } public static Serde from( @@ -98,8 +110,11 @@ public Serde create( final Serde genericRowSerde = toGenericRowSerde(formatSerde, schema); - final Serde loggingSerde = innerFactory - .wrapInLoggingSerde(genericRowSerde, loggerNamePrefix, processingLogContext); + final Serde loggingSerde = innerFactory.wrapInLoggingSerde( + genericRowSerde, + loggerNamePrefix, + processingLogContext, + queryId); final Serde serde = tracker .map(callback -> innerFactory.wrapInTrackingSerde(loggingSerde, callback)) diff --git a/ksqldb-serde/src/main/java/io/confluent/ksql/serde/GenericSerdeFactory.java b/ksqldb-serde/src/main/java/io/confluent/ksql/serde/GenericSerdeFactory.java index bea41384aca1..542d2ad4b46b 100644 --- a/ksqldb-serde/src/main/java/io/confluent/ksql/serde/GenericSerdeFactory.java +++ b/ksqldb-serde/src/main/java/io/confluent/ksql/serde/GenericSerdeFactory.java @@ -28,8 +28,11 @@ import io.confluent.ksql.serde.tracked.TrackedCallback; import io.confluent.ksql.serde.tracked.TrackedSerde; import io.confluent.ksql.util.KsqlConfig; +import io.confluent.ksql.util.MetricsTagsUtil; +import java.util.Collections; import java.util.List; import java.util.Objects; +import java.util.Optional; import java.util.function.Function; import java.util.function.Supplier; import org.apache.kafka.common.serialization.Serde; @@ -82,16 +85,43 @@ Serde> createFormatSerde( } } + Serde wrapInLoggingSerde( + final Serde formatSerde, + final String loggerNamePrefix, + final ProcessingLogContext processingLogContext + ) { + return wrapInLoggingSerde( + formatSerde, + loggerNamePrefix, + processingLogContext, + Optional.empty() + ); + } + @SuppressWarnings("MethodMayBeStatic") // Part of injected API Serde wrapInLoggingSerde( final Serde formatSerde, final String loggerNamePrefix, - final ProcessingLogContext processingLogContext + final ProcessingLogContext processingLogContext, + final Optional queryId ) { - final ProcessingLogger serializerProcessingLogger = processingLogContext.getLoggerFactory() - .getLogger(join(loggerNamePrefix, SERIALIZER_LOGGER_NAME)); - final ProcessingLogger deserializerProcessingLogger = processingLogContext.getLoggerFactory() - .getLogger(join(loggerNamePrefix, DESERIALIZER_LOGGER_NAME)); + final ProcessingLogger serializerProcessingLogger; + final ProcessingLogger deserializerProcessingLogger; + if (queryId.isPresent()) { + serializerProcessingLogger = processingLogContext.getLoggerFactory() + .getLogger( + join(loggerNamePrefix, SERIALIZER_LOGGER_NAME), + MetricsTagsUtil.getMetricsTagsWithQueryId(queryId.get(), Collections.emptyMap())); + deserializerProcessingLogger = processingLogContext.getLoggerFactory() + .getLogger( + join(loggerNamePrefix, DESERIALIZER_LOGGER_NAME), + MetricsTagsUtil.getMetricsTagsWithQueryId(queryId.get(), Collections.emptyMap())); + } else { + serializerProcessingLogger = processingLogContext.getLoggerFactory() + .getLogger(join(loggerNamePrefix, SERIALIZER_LOGGER_NAME)); + deserializerProcessingLogger = processingLogContext.getLoggerFactory() + .getLogger(join(loggerNamePrefix, DESERIALIZER_LOGGER_NAME)); + } return Serdes.serdeFrom( new LoggingSerializer<>(formatSerde.serializer(), serializerProcessingLogger), diff --git a/ksqldb-serde/src/main/java/io/confluent/ksql/serde/SerdeFactory.java b/ksqldb-serde/src/main/java/io/confluent/ksql/serde/SerdeFactory.java new file mode 100644 index 000000000000..eba516f6c441 --- /dev/null +++ b/ksqldb-serde/src/main/java/io/confluent/ksql/serde/SerdeFactory.java @@ -0,0 +1,31 @@ +/* + * Copyright 2022 Confluent Inc. + * + * Licensed under the Confluent Community License (the "License"); you may not use + * this file except in compliance with the License. You may obtain a copy of the + * License at + * + * http://www.confluent.io/confluent-community-license + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, WITHOUT + * WARRANTIES OF ANY KIND, either express or implied. See the License for the + * specific language governing permissions and limitations under the License. + */ + +package io.confluent.ksql.serde; + +import io.confluent.kafka.schemaregistry.client.SchemaRegistryClient; +import io.confluent.ksql.util.KsqlConfig; +import java.util.function.Supplier; +import org.apache.kafka.common.serialization.Serde; +import org.apache.kafka.connect.data.Schema; + +public interface SerdeFactory { + Serde createSerde( + Schema schema, + KsqlConfig ksqlConfig, + Supplier srFactory, + Class targetType, + boolean isKey); +} diff --git a/ksqldb-serde/src/main/java/io/confluent/ksql/serde/avro/KsqlAvroSerdeFactory.java b/ksqldb-serde/src/main/java/io/confluent/ksql/serde/avro/KsqlAvroSerdeFactory.java index b9ad19a01d6b..fa6aca840d5b 100644 --- a/ksqldb-serde/src/main/java/io/confluent/ksql/serde/avro/KsqlAvroSerdeFactory.java +++ b/ksqldb-serde/src/main/java/io/confluent/ksql/serde/avro/KsqlAvroSerdeFactory.java @@ -21,6 +21,7 @@ import io.confluent.connect.avro.AvroDataConfig; import io.confluent.kafka.schemaregistry.client.SchemaRegistryClient; import io.confluent.kafka.serializers.AbstractKafkaSchemaSerDeConfig; +import io.confluent.ksql.serde.SerdeFactory; import io.confluent.ksql.serde.SerdeUtils; import io.confluent.ksql.serde.connect.ConnectDataTranslator; import io.confluent.ksql.serde.connect.DataTranslator; @@ -37,12 +38,11 @@ import org.apache.kafka.common.serialization.Serde; import org.apache.kafka.common.serialization.Serdes; import org.apache.kafka.common.serialization.Serializer; -import org.apache.kafka.connect.data.ConnectSchema; import org.apache.kafka.connect.data.Schema; @Immutable @SuppressWarnings("checkstyle:ClassDataAbstractionCoupling") -class KsqlAvroSerdeFactory { +class KsqlAvroSerdeFactory implements SerdeFactory { private final String fullSchemaName; private final AvroProperties properties; @@ -60,8 +60,9 @@ class KsqlAvroSerdeFactory { this(new AvroProperties(properties)); } - Serde createSerde( - final ConnectSchema schema, + @Override + public Serde createSerde( + final Schema schema, final KsqlConfig ksqlConfig, final Supplier srFactory, final Class targetType, @@ -101,7 +102,7 @@ Serde createSerde( } private Supplier> createConnectSerializer( - final ConnectSchema schema, + final Schema schema, final KsqlConfig ksqlConfig, final Supplier srFactory, final Class targetType, @@ -127,7 +128,7 @@ private Supplier> createConnectSerializer( } private Supplier> createConnectDeserializer( - final ConnectSchema schema, + final Schema schema, final KsqlConfig ksqlConfig, final Supplier srFactory, final Class targetType, diff --git a/ksqldb-serde/src/main/java/io/confluent/ksql/serde/json/KsqlJsonDeserializer.java b/ksqldb-serde/src/main/java/io/confluent/ksql/serde/json/KsqlJsonDeserializer.java index c65b7f0332de..3bb14db99c19 100644 --- a/ksqldb-serde/src/main/java/io/confluent/ksql/serde/json/KsqlJsonDeserializer.java +++ b/ksqldb-serde/src/main/java/io/confluent/ksql/serde/json/KsqlJsonDeserializer.java @@ -50,7 +50,6 @@ import java.util.stream.Collectors; import org.apache.kafka.common.errors.SerializationException; import org.apache.kafka.common.serialization.Deserializer; -import org.apache.kafka.connect.data.ConnectSchema; import org.apache.kafka.connect.data.Date; import org.apache.kafka.connect.data.Field; import org.apache.kafka.connect.data.Schema; @@ -89,13 +88,13 @@ public class KsqlJsonDeserializer implements Deserializer { .put(Type.BYTES, KsqlJsonDeserializer::enforceValidBytes) .build(); - private final ConnectSchema schema; + private final Schema schema; private final boolean isJsonSchema; private final Class targetType; private String target = "?"; KsqlJsonDeserializer( - final ConnectSchema schema, + final Schema schema, final boolean isJsonSchema, final Class targetType ) { @@ -359,7 +358,7 @@ public String getPath() { } } - private static ConnectSchema validateSchema(final ConnectSchema schema) { + private static Schema validateSchema(final Schema schema) { class SchemaValidator implements Visitor { diff --git a/ksqldb-serde/src/main/java/io/confluent/ksql/serde/json/KsqlJsonSerdeFactory.java b/ksqldb-serde/src/main/java/io/confluent/ksql/serde/json/KsqlJsonSerdeFactory.java index 8ec9f60590ab..dd9e60f34447 100644 --- a/ksqldb-serde/src/main/java/io/confluent/ksql/serde/json/KsqlJsonSerdeFactory.java +++ b/ksqldb-serde/src/main/java/io/confluent/ksql/serde/json/KsqlJsonSerdeFactory.java @@ -21,6 +21,7 @@ import io.confluent.connect.json.JsonSchemaConverterConfig; import io.confluent.kafka.schemaregistry.client.SchemaRegistryClient; import io.confluent.kafka.serializers.AbstractKafkaSchemaSerDeConfig; +import io.confluent.ksql.serde.SerdeFactory; import io.confluent.ksql.serde.SerdeUtils; import io.confluent.ksql.serde.connect.ConnectDataTranslator; import io.confluent.ksql.serde.connect.ConnectSRSchemaDataTranslator; @@ -35,7 +36,6 @@ import org.apache.kafka.common.serialization.Serde; import org.apache.kafka.common.serialization.Serdes; import org.apache.kafka.common.serialization.Serializer; -import org.apache.kafka.connect.data.ConnectSchema; import org.apache.kafka.connect.data.Schema; import org.apache.kafka.connect.json.DecimalFormat; import org.apache.kafka.connect.json.JsonConverter; @@ -44,7 +44,7 @@ @SuppressWarnings("checkstyle:ClassDataAbstractionCoupling") @Immutable -class KsqlJsonSerdeFactory { +class KsqlJsonSerdeFactory implements SerdeFactory { private final boolean useSchemaRegistryFormat; private final JsonSchemaProperties properties; @@ -62,8 +62,9 @@ class KsqlJsonSerdeFactory { this.properties = Objects.requireNonNull(properties, "properties"); } - Serde createSerde( - final ConnectSchema schema, + @Override + public Serde createSerde( + final Schema schema, final KsqlConfig ksqlConfig, final Supplier srFactory, final Class targetType, @@ -99,7 +100,7 @@ Serde createSerde( } private Serializer createSerializer( - final ConnectSchema schema, + final Schema schema, final KsqlConfig ksqlConfig, final Supplier srFactory, final Class targetType, @@ -123,7 +124,7 @@ private Serializer createSerializer( } private Deserializer createDeserializer( - final ConnectSchema schema, + final Schema schema, final Class targetType ) { return new KsqlJsonDeserializer<>( diff --git a/ksqldb-serde/src/main/java/io/confluent/ksql/serde/protobuf/AbstractProtobufFormat.java b/ksqldb-serde/src/main/java/io/confluent/ksql/serde/protobuf/AbstractProtobufFormat.java new file mode 100644 index 000000000000..9cfff2657a86 --- /dev/null +++ b/ksqldb-serde/src/main/java/io/confluent/ksql/serde/protobuf/AbstractProtobufFormat.java @@ -0,0 +1,44 @@ +/* + * Copyright 2022 Confluent Inc. + * + * Licensed under the Confluent Community License (the "License"); you may not use + * this file except in compliance with the License. You may obtain a copy of the + * License at + * + * http://www.confluent.io/confluent-community-license + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, WITHOUT + * WARRANTIES OF ANY KIND, either express or implied. See the License for the + * specific language governing permissions and limitations under the License. + */ + +package io.confluent.ksql.serde.protobuf; + +import com.google.common.base.Joiner; +import com.google.common.collect.ImmutableList; +import com.squareup.wire.schema.internal.parser.ProtoFileElement; +import io.confluent.kafka.schemaregistry.ParsedSchema; +import io.confluent.ksql.serde.connect.ConnectFormat; +import java.util.List; +import java.util.stream.Collectors; + +/** + * This class contains common method from {@link ProtobufFormat} and {@link ProtobufNoSRFormat} + */ +public abstract class AbstractProtobufFormat extends ConnectFormat { + + @Override + public List schemaFullNames(final ParsedSchema schema) { + if (schema.rawSchema() instanceof ProtoFileElement) { + final ProtoFileElement protoFileElement = (ProtoFileElement) schema.rawSchema(); + final String packageName = protoFileElement.getPackageName(); + + return protoFileElement.getTypes().stream() + .map(typeElement -> Joiner.on(".").skipNulls().join(packageName, typeElement.getName())) + .collect(Collectors.toList()); + } + + return ImmutableList.of(); + } +} diff --git a/ksqldb-serde/src/main/java/io/confluent/ksql/serde/protobuf/ProtobufFormat.java b/ksqldb-serde/src/main/java/io/confluent/ksql/serde/protobuf/ProtobufFormat.java index 5a4f15f2919f..b78d582be5e7 100644 --- a/ksqldb-serde/src/main/java/io/confluent/ksql/serde/protobuf/ProtobufFormat.java +++ b/ksqldb-serde/src/main/java/io/confluent/ksql/serde/protobuf/ProtobufFormat.java @@ -15,27 +15,20 @@ package io.confluent.ksql.serde.protobuf; -import com.google.common.base.Joiner; -import com.google.common.collect.ImmutableList; import com.google.common.collect.ImmutableSet; -import com.squareup.wire.schema.internal.parser.ProtoFileElement; import edu.umd.cs.findbugs.annotations.SuppressFBWarnings; -import io.confluent.kafka.schemaregistry.ParsedSchema; import io.confluent.kafka.schemaregistry.client.SchemaRegistryClient; import io.confluent.ksql.serde.FormatProperties; import io.confluent.ksql.serde.SerdeFeature; -import io.confluent.ksql.serde.connect.ConnectFormat; import io.confluent.ksql.serde.connect.ConnectSchemaTranslator; import io.confluent.ksql.util.KsqlConfig; -import java.util.List; import java.util.Map; import java.util.Set; import java.util.function.Supplier; -import java.util.stream.Collectors; import org.apache.kafka.common.serialization.Serde; import org.apache.kafka.connect.data.ConnectSchema; -public class ProtobufFormat extends ConnectFormat { +public class ProtobufFormat extends AbstractProtobufFormat { static final ImmutableSet SUPPORTED_FEATURES = ImmutableSet.of( SerdeFeature.SCHEMA_INFERENCE, @@ -86,18 +79,4 @@ protected Serde getConnectSerde( return new ProtobufSerdeFactory(new ProtobufProperties(formatProps)) .createSerde(connectSchema, config, srFactory, targetType, isKey); } - - @Override - public List schemaFullNames(final ParsedSchema schema) { - if (schema.rawSchema() instanceof ProtoFileElement) { - final ProtoFileElement protoFileElement = (ProtoFileElement) schema.rawSchema(); - final String packageName = protoFileElement.getPackageName(); - - return protoFileElement.getTypes().stream() - .map(typeElement -> Joiner.on(".").skipNulls().join(packageName, typeElement.getName())) - .collect(Collectors.toList()); - } - - return ImmutableList.of(); - } } diff --git a/ksqldb-serde/src/main/java/io/confluent/ksql/serde/protobuf/ProtobufNoSRConverter.java b/ksqldb-serde/src/main/java/io/confluent/ksql/serde/protobuf/ProtobufNoSRConverter.java new file mode 100644 index 000000000000..80aaf469f913 --- /dev/null +++ b/ksqldb-serde/src/main/java/io/confluent/ksql/serde/protobuf/ProtobufNoSRConverter.java @@ -0,0 +1,159 @@ +/* + * Copyright 2022 Confluent Inc. + * + * Licensed under the Confluent Community License (the "License"); you may not use + * this file except in compliance with the License. You may obtain a copy of the + * License at + * + * http://www.confluent.io/confluent-community-license + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, WITHOUT + * WARRANTIES OF ANY KIND, either express or implied. See the License for the + * specific language governing permissions and limitations under the License. + */ + +package io.confluent.ksql.serde.protobuf; + +import com.google.common.annotations.VisibleForTesting; +import com.google.protobuf.Descriptors.Descriptor; +import com.google.protobuf.DynamicMessage; +import com.google.protobuf.Message; +import io.confluent.connect.protobuf.ProtobufData; +import io.confluent.connect.protobuf.ProtobufDataConfig; +import io.confluent.kafka.schemaregistry.protobuf.ProtobufSchema; +import io.confluent.kafka.serializers.protobuf.ProtobufSchemaAndValue; +import java.io.ByteArrayInputStream; +import java.io.ByteArrayOutputStream; +import java.io.IOException; +import java.nio.ByteBuffer; +import java.util.Map; +import org.apache.kafka.common.config.ConfigException; +import org.apache.kafka.common.errors.InvalidConfigurationException; +import org.apache.kafka.common.errors.SerializationException; +import org.apache.kafka.connect.data.Schema; +import org.apache.kafka.connect.data.SchemaAndValue; +import org.apache.kafka.connect.errors.DataException; +import org.apache.kafka.connect.storage.Converter; + +public class ProtobufNoSRConverter implements Converter { + + private static final Serializer serializer = new Serializer(); + private static final Deserializer deserializer = new Deserializer(); + private final Schema schema; + + private ProtobufData protobufData; + + public ProtobufNoSRConverter(final Schema schema) { + this.schema = schema; + } + + @Override + public void configure(final Map configs, final boolean isKey) { + this.protobufData = new ProtobufData(new ProtobufDataConfig(configs)); + } + + @Override + public byte[] fromConnectData(final String topic, final Schema schema, final Object value) { + try { + final ProtobufSchemaAndValue schemaAndValue = protobufData.fromConnectData(schema, value); + final Object v = schemaAndValue.getValue(); + if (v == null) { + return null; + } else if (v instanceof Message) { + return serializer.serialize((Message) v); + } else { + throw new DataException("Unsupported object of class " + v.getClass().getName()); + } + } catch (SerializationException e) { + throw new DataException(String.format( + "Failed to serialize Protobuf data from topic %s :", + topic + ), e); + } catch (InvalidConfigurationException e) { + throw new ConfigException( + String.format("Failed to access Protobuf data from topic %s : %s", topic, e.getMessage()) + ); + } + } + + @Override + public SchemaAndValue toConnectData(final String topic, final byte[] value) { + try { + final ProtobufSchema protobufSchema = protobufData.fromConnectSchema(schema); + final Object deserialized = deserializer.deserialize(value, protobufSchema); + + if (deserialized == null) { + return SchemaAndValue.NULL; + } else { + if (deserialized instanceof Message) { + return protobufData.toConnectData(protobufSchema, (Message) deserialized); + } + throw new DataException(String.format( + "Unsupported type %s returned during deserialization of topic %s ", + deserialized.getClass().getName(), + topic + )); + } + } catch (SerializationException e) { + throw new DataException(String.format( + "Failed to deserialize data for topic %s to Protobuf: ", + topic + ), e); + } catch (InvalidConfigurationException e) { + throw new ConfigException( + String.format("Failed to access Protobuf data from topic %s : %s", topic, e.getMessage()) + ); + } + } + + @VisibleForTesting + public static class Serializer { + public byte[] serialize(final Message value) { + if (value == null) { + return null; + } + + try { + final ByteArrayOutputStream out = new ByteArrayOutputStream(); + value.writeTo(out); + final byte[] bytes = out.toByteArray(); + out.close(); + return bytes; + } catch (IOException | RuntimeException e) { + throw new SerializationException("Error serializing Protobuf message", e); + } + } + } + + + + @VisibleForTesting + public static class Deserializer { + public Object deserialize(final byte[] payload, final ProtobufSchema schema) { + if (payload == null) { + return null; + } + + try { + final ByteBuffer buffer = ByteBuffer.wrap(payload); + + final Descriptor descriptor = schema.toDescriptor(); + if (descriptor == null) { + throw new SerializationException("Could not find descriptor with name " + schema.name()); + } + final int length = buffer.limit(); + final int start = buffer.position() + buffer.arrayOffset(); + + return DynamicMessage.parseFrom(descriptor, + new ByteArrayInputStream(buffer.array(), start, length) + ); + + } catch (IOException | RuntimeException e) { + throw new SerializationException("Error deserializing Protobuf message for schema " + + schema, e); + } + } + } + +} diff --git a/ksqldb-serde/src/main/java/io/confluent/ksql/serde/protobuf/ProtobufNoSRFormat.java b/ksqldb-serde/src/main/java/io/confluent/ksql/serde/protobuf/ProtobufNoSRFormat.java new file mode 100644 index 000000000000..70cd123d24d6 --- /dev/null +++ b/ksqldb-serde/src/main/java/io/confluent/ksql/serde/protobuf/ProtobufNoSRFormat.java @@ -0,0 +1,74 @@ +/* + * Copyright 2022 Confluent Inc. + * + * Licensed under the Confluent Community License (the "License"); you may not use + * this file except in compliance with the License. You may obtain a copy of the + * License at + * + * http://www.confluent.io/confluent-community-license + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, WITHOUT + * WARRANTIES OF ANY KIND, either express or implied. See the License for the + * specific language governing permissions and limitations under the License. + */ + +package io.confluent.ksql.serde.protobuf; + +import com.google.common.collect.ImmutableSet; +import edu.umd.cs.findbugs.annotations.SuppressFBWarnings; +import io.confluent.kafka.schemaregistry.client.SchemaRegistryClient; +import io.confluent.ksql.serde.SerdeFeature; +import io.confluent.ksql.serde.connect.ConnectSchemaTranslator; +import io.confluent.ksql.util.KsqlConfig; +import java.util.Map; +import java.util.Set; +import java.util.function.Supplier; +import org.apache.kafka.common.serialization.Serde; +import org.apache.kafka.connect.data.ConnectSchema; + +public class ProtobufNoSRFormat extends AbstractProtobufFormat { + static final ImmutableSet SUPPORTED_FEATURES = ImmutableSet.of( + SerdeFeature.WRAP_SINGLES, + SerdeFeature.UNWRAP_SINGLES + ); + + public static final String NAME = "PROTOBUF_NOSR"; + + @Override + public String name() { + return NAME; + } + + @Override + @SuppressFBWarnings(value = "EI_EXPOSE_REP", justification = "SUPPORTED_FEATURES is ImmutableSet") + public Set supportedFeatures() { + return SUPPORTED_FEATURES; + } + + + @Override + public Set getSupportedProperties() { + return ProtobufNoSRProperties.SUPPORTED_PROPERTIES; + } + + @Override + protected ConnectSchemaTranslator getConnectSchemaTranslator( + final Map formatProps + ) { + throw new UnsupportedOperationException(name() + " does not implement Schema Registry support"); + } + + @Override + protected Serde getConnectSerde( + final ConnectSchema connectSchema, + final Map formatProps, + final KsqlConfig config, + final Supplier srFactory, + final Class targetType, + final boolean isKey + ) { + return new ProtobufNoSRSerdeFactory(new ProtobufNoSRProperties(formatProps)) + .createSerde(connectSchema, config, srFactory, targetType, isKey); + } +} diff --git a/ksqldb-serde/src/main/java/io/confluent/ksql/serde/protobuf/ProtobufNoSRProperties.java b/ksqldb-serde/src/main/java/io/confluent/ksql/serde/protobuf/ProtobufNoSRProperties.java new file mode 100644 index 000000000000..3d4489556331 --- /dev/null +++ b/ksqldb-serde/src/main/java/io/confluent/ksql/serde/protobuf/ProtobufNoSRProperties.java @@ -0,0 +1,52 @@ +/* + * Copyright 2021 Confluent Inc. + * + * Licensed under the Confluent Community License (the "License"); you may not use + * this file except in compliance with the License. You may obtain a copy of the + * License at + * + * http://www.confluent.io/confluent-community-license + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, WITHOUT + * WARRANTIES OF ANY KIND, either express or implied. See the License for the + * specific language governing permissions and limitations under the License. + */ + +package io.confluent.ksql.serde.protobuf; + +import com.google.common.collect.ImmutableSet; +import edu.umd.cs.findbugs.annotations.SuppressFBWarnings; +import io.confluent.ksql.serde.connect.ConnectProperties; +import java.util.Map; + +public class ProtobufNoSRProperties extends ConnectProperties { + + public static final String UNWRAP_PRIMITIVES = "unwrapPrimitives"; + public static final String UNWRAP = "true"; + private static final String WRAP = "false"; + + static final ImmutableSet SUPPORTED_PROPERTIES = ImmutableSet.of( + UNWRAP_PRIMITIVES + ); + + public ProtobufNoSRProperties(final Map formatProps) { + super(ProtobufNoSRFormat.NAME, formatProps); + } + + @Override + @SuppressFBWarnings(value = "EI_EXPOSE_REP") + public ImmutableSet getSupportedProperties() { + return SUPPORTED_PROPERTIES; + } + + @Override + public String getDefaultFullSchemaName() { + // Return null to be backward compatible for unset schema name + return null; + } + + public boolean getUnwrapPrimitives() { + return UNWRAP.equalsIgnoreCase(properties.getOrDefault(UNWRAP_PRIMITIVES, WRAP)); + } +} diff --git a/ksqldb-serde/src/main/java/io/confluent/ksql/serde/protobuf/ProtobufNoSRSerdeFactory.java b/ksqldb-serde/src/main/java/io/confluent/ksql/serde/protobuf/ProtobufNoSRSerdeFactory.java new file mode 100644 index 000000000000..2728b258c0dc --- /dev/null +++ b/ksqldb-serde/src/main/java/io/confluent/ksql/serde/protobuf/ProtobufNoSRSerdeFactory.java @@ -0,0 +1,158 @@ +/* + * Copyright 2020 Confluent Inc. + * + * Licensed under the Confluent Community License (the "License"; you may not use + * this file except in compliance with the License. You may obtain a copy of the + * License at + * + * http://www.confluent.io/confluent-community-license + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, WITHOUT + * WARRANTIES OF ANY KIND, either express or implied. See the License for the + * specific language governing permissions and limitations under the License. + */ + +package io.confluent.ksql.serde.protobuf; + +import com.google.common.collect.ImmutableMap; +import io.confluent.connect.protobuf.ProtobufDataConfig; +import io.confluent.kafka.schemaregistry.client.SchemaRegistryClient; +import io.confluent.ksql.schema.connect.SchemaWalker; +import io.confluent.ksql.serde.SerdeFactory; +import io.confluent.ksql.serde.connect.ConnectDataTranslator; +import io.confluent.ksql.serde.connect.DataTranslator; +import io.confluent.ksql.serde.connect.KsqlConnectDeserializer; +import io.confluent.ksql.serde.connect.KsqlConnectSerializer; +import io.confluent.ksql.serde.tls.ThreadLocalDeserializer; +import io.confluent.ksql.serde.tls.ThreadLocalSerializer; +import io.confluent.ksql.util.KsqlConfig; +import io.confluent.ksql.util.KsqlException; +import java.util.HashMap; +import java.util.Map; +import java.util.Objects; +import java.util.function.Supplier; +import org.apache.kafka.common.serialization.Deserializer; +import org.apache.kafka.common.serialization.Serde; +import org.apache.kafka.common.serialization.Serdes; +import org.apache.kafka.common.serialization.Serializer; +import org.apache.kafka.connect.data.Schema; +import org.apache.kafka.connect.data.Schema.Type; + +@SuppressWarnings("checkstyle:ClassDataAbstractionCoupling") +final class ProtobufNoSRSerdeFactory implements SerdeFactory { + + private final ProtobufNoSRProperties properties; + + ProtobufNoSRSerdeFactory(final ProtobufNoSRProperties properties) { + this.properties = Objects.requireNonNull(properties, "properties"); + } + + ProtobufNoSRSerdeFactory(final ImmutableMap formatProperties) { + this(new ProtobufNoSRProperties(formatProperties)); + } + + @Override + public Serde createSerde( + final Schema schema, + final KsqlConfig ksqlConfig, + final Supplier srFactory, + final Class targetType, + final boolean isKey) { + validate(schema); + + final Supplier> serializer = () -> createSerializer( + schema, + targetType, + isKey + ); + final Supplier> deserializer = () -> createDeserializer( + schema, + targetType, + isKey + ); + + // Sanity check: + serializer.get(); + deserializer.get(); + + return Serdes.serdeFrom( + new ThreadLocalSerializer<>(serializer), + new ThreadLocalDeserializer<>(deserializer) + ); + } + + private static void validate(final Schema schema) { + SchemaWalker.visit(schema, new SchemaValidator()); + } + + private KsqlConnectSerializer createSerializer( + final Schema schema, + final Class targetType, + final boolean isKey + ) { + final ProtobufNoSRConverter converter = getConverter(schema, isKey); + final DataTranslator translator = getDataTranslator(schema); + final Schema compatibleSchema = translator instanceof ProtobufDataTranslator + ? ((ProtobufDataTranslator) translator).getSchema() + : ((ConnectDataTranslator) translator).getSchema(); + + return new KsqlConnectSerializer<>( + compatibleSchema, + translator, + converter, + targetType + ); + } + + private KsqlConnectDeserializer createDeserializer( + final Schema schema, + final Class targetType, + final boolean isKey + ) { + return new KsqlConnectDeserializer<>( + getConverter(schema, isKey), + getDataTranslator(schema), + targetType + ); + } + + private DataTranslator getDataTranslator(final Schema schema) { + // maybe switch to ProtobufSchemaTranslator or a variation + return new ConnectDataTranslator(schema); + } + + private ProtobufNoSRConverter getConverter( + final Schema schema, + final boolean isKey + ) { + final Map protobufConfig = new HashMap<>(); + + protobufConfig.put( + ProtobufDataConfig.WRAPPER_FOR_RAW_PRIMITIVES_CONFIG, + properties.getUnwrapPrimitives() + ); + + final ProtobufNoSRConverter converter = new ProtobufNoSRConverter(schema); + converter.configure(protobufConfig, isKey); + + return converter; + } + + private static class SchemaValidator implements SchemaWalker.Visitor { + + @Override + public Void visitMap(final Schema schema, final Void key, final Void value) { + if (schema.keySchema().type() != Type.STRING) { + throw new KsqlException("PROTOBUF format only supports MAP types with STRING keys. " + + "See https://github.com/confluentinc/ksql/issues/6177."); + } + return null; + } + + @Override + public Void visitSchema(final Schema schema) { + return null; + } + } +} \ No newline at end of file diff --git a/ksqldb-serde/src/main/java/io/confluent/ksql/serde/protobuf/ProtobufSerdeFactory.java b/ksqldb-serde/src/main/java/io/confluent/ksql/serde/protobuf/ProtobufSerdeFactory.java index 7a5324144860..e2f49fb57dbb 100644 --- a/ksqldb-serde/src/main/java/io/confluent/ksql/serde/protobuf/ProtobufSerdeFactory.java +++ b/ksqldb-serde/src/main/java/io/confluent/ksql/serde/protobuf/ProtobufSerdeFactory.java @@ -24,6 +24,7 @@ import io.confluent.kafka.schemaregistry.client.SchemaRegistryClient; import io.confluent.kafka.serializers.AbstractKafkaSchemaSerDeConfig; import io.confluent.ksql.schema.connect.SchemaWalker; +import io.confluent.ksql.serde.SerdeFactory; import io.confluent.ksql.serde.SerdeUtils; import io.confluent.ksql.serde.connect.ConnectDataTranslator; import io.confluent.ksql.serde.connect.DataTranslator; @@ -41,12 +42,11 @@ import org.apache.kafka.common.serialization.Serde; import org.apache.kafka.common.serialization.Serdes; import org.apache.kafka.common.serialization.Serializer; -import org.apache.kafka.connect.data.ConnectSchema; import org.apache.kafka.connect.data.Schema; import org.apache.kafka.connect.data.Schema.Type; @SuppressWarnings("checkstyle:ClassDataAbstractionCoupling") -final class ProtobufSerdeFactory { +final class ProtobufSerdeFactory implements SerdeFactory { private final ProtobufProperties properties; private final Optional fullSchemaName; @@ -107,8 +107,9 @@ private Optional getSchemaFromSR( } } - Serde createSerde( - final ConnectSchema schema, + @Override + public Serde createSerde( + final Schema schema, final KsqlConfig ksqlConfig, final Supplier srFactory, final Class targetType, @@ -165,7 +166,7 @@ private static void validate(final Schema schema) { } private KsqlConnectSerializer createSerializer( - final ConnectSchema schema, + final Schema schema, final KsqlConfig ksqlConfig, final Supplier srFactory, final Class targetType, @@ -188,7 +189,7 @@ private KsqlConnectSerializer createSerializer( } private KsqlConnectDeserializer createDeserializer( - final ConnectSchema schema, + final Schema schema, final KsqlConfig ksqlConfig, final Supplier srFactory, final Class targetType, diff --git a/ksqldb-serde/src/test/java/io/confluent/ksql/serde/FormatFactoryTest.java b/ksqldb-serde/src/test/java/io/confluent/ksql/serde/FormatFactoryTest.java index 23935224ffa1..f2eff9d2ccfa 100644 --- a/ksqldb-serde/src/test/java/io/confluent/ksql/serde/FormatFactoryTest.java +++ b/ksqldb-serde/src/test/java/io/confluent/ksql/serde/FormatFactoryTest.java @@ -33,6 +33,8 @@ public void shouldCreateFromString() { assertThat(FormatFactory.of(FormatInfo.of("JsoN")), is(FormatFactory.JSON)); assertThat(FormatFactory.of(FormatInfo.of("AvRo")), is(FormatFactory.AVRO)); assertThat(FormatFactory.of(FormatInfo.of("Delimited")), is(FormatFactory.DELIMITED)); + assertThat(FormatFactory.of(FormatInfo.of("PrOtObUf")), is(FormatFactory.PROTOBUF)); + assertThat(FormatFactory.of(FormatInfo.of("PrOtObUf_nOsR")), is(FormatFactory.PROTOBUF_NOSR)); } @Test diff --git a/ksqldb-serde/src/test/java/io/confluent/ksql/serde/GenericKeySerDeTest.java b/ksqldb-serde/src/test/java/io/confluent/ksql/serde/GenericKeySerDeTest.java index d0f394a2323f..6f35ac9f7562 100644 --- a/ksqldb-serde/src/test/java/io/confluent/ksql/serde/GenericKeySerDeTest.java +++ b/ksqldb-serde/src/test/java/io/confluent/ksql/serde/GenericKeySerDeTest.java @@ -63,6 +63,7 @@ public class GenericKeySerDeTest { private static final String LOGGER_PREFIX = "bob"; + private static final String queryId = "query"; private static final WindowInfo TIMED_WND = WindowInfo .of(WindowType.HOPPING, Optional.of(Duration.ofSeconds(10))); private static final WindowInfo SESSION_WND = WindowInfo @@ -98,10 +99,10 @@ public class GenericKeySerDeTest { @Before public void setUp() { - factory = new GenericKeySerDe(innerFactory); + factory = new GenericKeySerDe(innerFactory, Optional.of(queryId)); when(innerFactory.createFormatSerde(any(), any(), any(), any(), any(), anyBoolean())).thenReturn(innerSerde); - when(innerFactory.wrapInLoggingSerde(any(), any(), any())).thenReturn(loggingSerde); + when(innerFactory.wrapInLoggingSerde(any(), any(), any(), any())).thenReturn(loggingSerde); when(innerFactory.wrapInTrackingSerde(any(), any())).thenReturn(trackingSerde); when(innerSerde.serializer()).thenReturn(innerSerializer); @@ -136,7 +137,7 @@ public void shouldWrapInLoggingSerdeNonWindowed() { Optional.empty()); // Then: - verify(innerFactory).wrapInLoggingSerde(any(), eq(LOGGER_PREFIX), eq(processingLogCxt)); + verify(innerFactory).wrapInLoggingSerde(any(), eq(LOGGER_PREFIX), eq(processingLogCxt), eq(Optional.of(queryId))); } @Test @@ -147,7 +148,7 @@ public void shouldWrapInLoggingSerdeWindowed() { Optional.empty()); // Then: - verify(innerFactory).wrapInLoggingSerde(any(), eq(LOGGER_PREFIX), eq(processingLogCxt)); + verify(innerFactory).wrapInLoggingSerde(any(), eq(LOGGER_PREFIX), eq(processingLogCxt), eq(Optional.of(queryId))); } @Test diff --git a/ksqldb-serde/src/test/java/io/confluent/ksql/serde/GenericRowSerDeTest.java b/ksqldb-serde/src/test/java/io/confluent/ksql/serde/GenericRowSerDeTest.java index 22981aa6c040..0bac4995df1b 100644 --- a/ksqldb-serde/src/test/java/io/confluent/ksql/serde/GenericRowSerDeTest.java +++ b/ksqldb-serde/src/test/java/io/confluent/ksql/serde/GenericRowSerDeTest.java @@ -51,6 +51,7 @@ public class GenericRowSerDeTest { private static final String LOGGER_PREFIX = "bob"; + private static final String queryId = "query"; @Mock private GenericSerdeFactory innerFactory; @@ -83,10 +84,10 @@ public class GenericRowSerDeTest { @Before public void setUp() { - factory = new GenericRowSerDe(innerFactory); + factory = new GenericRowSerDe(innerFactory, Optional.of(queryId)); when(innerFactory.createFormatSerde(any(), any(), any(), any(), any(), anyBoolean())).thenReturn(innerSerde); - when(innerFactory.wrapInLoggingSerde(any(), any(), any())).thenReturn(loggingSerde); + when(innerFactory.wrapInLoggingSerde(any(), any(), any(), any())).thenReturn(loggingSerde); when(innerFactory.wrapInTrackingSerde(any(), any())).thenReturn(trackingSerde); when(innerSerde.serializer()).thenReturn(innerSerializer); when(innerSerde.deserializer()).thenReturn(innerDeserializer); @@ -109,7 +110,7 @@ public void shouldWrapInLoggingSerde() { Optional.empty()); // Then: - verify(innerFactory).wrapInLoggingSerde(any(), eq(LOGGER_PREFIX), eq(processingLogCxt)); + verify(innerFactory).wrapInLoggingSerde(any(), eq(LOGGER_PREFIX), eq(processingLogCxt), eq(Optional.of(queryId))); } @Test @@ -149,7 +150,7 @@ public void shouldWrapInGenericSerde() { Optional.empty()); // Then: - verify(innerFactory).wrapInLoggingSerde(rowSerdeCaptor.capture(), any(), any()); + verify(innerFactory).wrapInLoggingSerde(rowSerdeCaptor.capture(), any(), any(), any()); assertThat(rowSerdeCaptor.getValue().serializer(), is(instanceOf(GenericSerializer.class))); assertThat(rowSerdeCaptor.getValue().deserializer(), diff --git a/ksqldb-serde/src/test/java/io/confluent/ksql/serde/GenericSerdeFactoryTest.java b/ksqldb-serde/src/test/java/io/confluent/ksql/serde/GenericSerdeFactoryTest.java index acacb10b280f..b5b521900741 100644 --- a/ksqldb-serde/src/test/java/io/confluent/ksql/serde/GenericSerdeFactoryTest.java +++ b/ksqldb-serde/src/test/java/io/confluent/ksql/serde/GenericSerdeFactoryTest.java @@ -34,6 +34,7 @@ import io.confluent.ksql.util.KsqlConfig; import java.nio.charset.StandardCharsets; import java.util.Map; +import java.util.Optional; import java.util.function.Function; import java.util.function.Supplier; import org.apache.kafka.common.serialization.Deserializer; @@ -70,9 +71,9 @@ public class GenericSerdeFactoryTest { @Mock private ProcessingLoggerFactory processingLoggerFactory; @Mock - private ProcessingLogger processingLogger; + private ProcessingLogger processingLoggerWithoutQueryId; @Mock - private ProcessingLogger otherProcessingLogger; + private ProcessingLogger processingLoggerWithQueryId; @Mock private Serializer formatSerializer; @Mock @@ -98,7 +99,8 @@ public void setUp() { .thenThrow(new RuntimeException("deserializer error")); when(processingLogContext.getLoggerFactory()).thenReturn(processingLoggerFactory); - when(processingLoggerFactory.getLogger(any())).thenReturn(otherProcessingLogger); + when(processingLoggerFactory.getLogger(any())).thenReturn(processingLoggerWithoutQueryId); + when(processingLoggerFactory.getLogger(any(), any())).thenReturn(processingLoggerWithQueryId); } @Test @@ -163,9 +165,6 @@ public void shouldThrowIfGetSerdeThrows() { @Test public void shouldWrapSerializerWithLoggingSerializer() { // Given: - when(processingLoggerFactory.getLogger("prefix.serializer")).thenReturn(processingLogger); - - // When: final Serde result = serdeFactory .wrapInLoggingSerde(formatSerde, "prefix", processingLogContext); @@ -178,13 +177,13 @@ public void shouldWrapSerializerWithLoggingSerializer() { ) ); - verify(processingLogger).error(any()); + verify(processingLoggerWithoutQueryId).error(any()); } @Test public void shouldWrapDeserializerWithLoggingSerializer() { // Given: - when(processingLoggerFactory.getLogger("prefix.deserializer")).thenReturn(processingLogger); + when(processingLoggerFactory.getLogger("prefix.deserializer")).thenReturn(processingLoggerWithoutQueryId); // When: final Serde result = serdeFactory @@ -199,6 +198,24 @@ public void shouldWrapDeserializerWithLoggingSerializer() { ) ); - verify(processingLogger).error(any()); + verify(processingLoggerWithoutQueryId).error(any()); + } + + @Test + public void shouldReturnProcessingLoggerWithQueryId() { + // When: + final Serde result = serdeFactory + .wrapInLoggingSerde(formatSerde, "prefix", processingLogContext, Optional.of("query-id")); + + // Then: + assertThrows( + RuntimeException.class, + () -> result.deserializer().deserialize( + "topicName", + "this call will cause error to be logged".getBytes(StandardCharsets.UTF_8) + ) + ); + + verify(processingLoggerWithQueryId).error(any()); } } \ No newline at end of file diff --git a/ksqldb-serde/src/test/java/io/confluent/ksql/serde/protobuf/AbstractKsqlProtobufDeserializerTest.java b/ksqldb-serde/src/test/java/io/confluent/ksql/serde/protobuf/AbstractKsqlProtobufDeserializerTest.java new file mode 100644 index 000000000000..78b13eecb43a --- /dev/null +++ b/ksqldb-serde/src/test/java/io/confluent/ksql/serde/protobuf/AbstractKsqlProtobufDeserializerTest.java @@ -0,0 +1,155 @@ +/* + * Copyright 2022 Confluent Inc. + * + * Licensed under the Confluent Community License (the "License"); you may not use + * this file except in compliance with the License. You may obtain a copy of the + * License at + * + * http://www.confluent.io/confluent-community-license + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, WITHOUT + * WARRANTIES OF ANY KIND, either express or implied. See the License for the + * specific language governing permissions and limitations under the License. + */ + +package io.confluent.ksql.serde.protobuf; + +import static org.hamcrest.MatcherAssert.assertThat; +import static org.hamcrest.Matchers.is; + +import io.confluent.ksql.util.DecimalUtil; +import java.math.BigDecimal; +import java.nio.ByteBuffer; +import org.apache.kafka.common.serialization.Deserializer; +import org.apache.kafka.connect.data.ConnectSchema; +import org.apache.kafka.connect.data.Date; +import org.apache.kafka.connect.data.Schema; +import org.apache.kafka.connect.data.SchemaBuilder; +import org.apache.kafka.connect.data.Struct; +import org.apache.kafka.connect.data.Time; +import org.apache.kafka.connect.data.Timestamp; +import org.apache.kafka.connect.storage.Converter; +import org.junit.Test; + +public abstract class AbstractKsqlProtobufDeserializerTest { + static final String SOME_TOPIC = "bob"; + + abstract Deserializer givenDeserializerForSchema( + final ConnectSchema schema, + final Class targetType + ); + + abstract byte[] givenConnectSerialized( + final Converter converter, + final Object value, + final Schema connectSchema + ); + + abstract Converter getConverter(final ConnectSchema schema); + + @Test + public void shouldDeserializeDecimalField() { + final ConnectSchema schema = (ConnectSchema) SchemaBuilder.struct() + .field("f0", DecimalUtil.builder(10, 2)) + .build(); + final Converter converter = getConverter(schema); + + // Given: + final Deserializer deserializer = + givenDeserializerForSchema(schema, + Struct.class); + final Struct value = new Struct(schema).put("f0", new BigDecimal("12.34")); + final byte[] bytes = givenConnectSerialized(converter, value, schema); + + // When: + final Object result = deserializer.deserialize(SOME_TOPIC, bytes); + + // Then: + assertThat(result, is(value)); + } + + @Test + public void shouldDeserializeTimeField() { + final ConnectSchema schema = (ConnectSchema) SchemaBuilder.struct() + .field("f0", Time.SCHEMA) + .build(); + final Converter converter = getConverter(schema); + + // Given: + final Deserializer deserializer = + givenDeserializerForSchema(schema, + Struct.class); + final Struct value = new Struct(schema).put("f0", new java.sql.Time(2000)); + final byte[] bytes = givenConnectSerialized(converter, value, schema); + + // When: + final Object result = deserializer.deserialize(SOME_TOPIC, bytes); + + // Then: + assertThat(result, is(value)); + } + + @Test + public void shouldDeserializeDateField() { + final ConnectSchema schema = (ConnectSchema) SchemaBuilder.struct() + .field("f0", Date.SCHEMA) + .build(); + final Converter converter = getConverter(schema); + + // Given: + final Deserializer deserializer = + givenDeserializerForSchema(schema, + Struct.class); + final Struct value = new Struct(schema).put("f0", new java.sql.Date(864000000L)); + final byte[] bytes = givenConnectSerialized(converter, value, schema); + + // When: + final Object result = deserializer.deserialize(SOME_TOPIC, bytes); + + // Then: + assertThat(result, is(value)); + } + + @Test + public void shouldDeserializeTimestampField() { + final ConnectSchema schema = (ConnectSchema) SchemaBuilder.struct() + .field("f0", Timestamp.SCHEMA) + .build(); + final Converter converter = getConverter(schema); + + // Given: + final Deserializer deserializer = + givenDeserializerForSchema(schema, + Struct.class); + final Struct value = new Struct(schema).put("f0", new java.sql.Timestamp(2000)); + final byte[] bytes = givenConnectSerialized(converter, value, schema); + + // When: + final Object result = deserializer.deserialize(SOME_TOPIC, bytes); + + // Then: + assertThat(result, is(value)); + } + + @Test + public void shouldDeserializeBytesField() { + final ConnectSchema schema = (ConnectSchema) SchemaBuilder.struct() + .field("f0", Schema.BYTES_SCHEMA) + .build(); + final Converter converter = getConverter(schema); + + // Given: + final Deserializer deserializer = + givenDeserializerForSchema(schema, + Struct.class); + final Struct value = new Struct(schema).put("f0", ByteBuffer.wrap(new byte[] {123})); + final byte[] bytes = givenConnectSerialized(converter, value, schema); + + // When: + final Object result = deserializer.deserialize(SOME_TOPIC, bytes); + + // Then: + assertThat(((Struct) result).getBytes("f0"), is(value.getBytes("f0"))); + } +} diff --git a/ksqldb-serde/src/test/java/io/confluent/ksql/serde/protobuf/AbstractProtobufSerdeFactoryTest.java b/ksqldb-serde/src/test/java/io/confluent/ksql/serde/protobuf/AbstractProtobufSerdeFactoryTest.java new file mode 100644 index 000000000000..fe2b8fc78fff --- /dev/null +++ b/ksqldb-serde/src/test/java/io/confluent/ksql/serde/protobuf/AbstractProtobufSerdeFactoryTest.java @@ -0,0 +1,71 @@ +/* + * Copyright 2022 Confluent Inc. + * + * Licensed under the Confluent Community License (the "License"); you may not use + * this file except in compliance with the License. You may obtain a copy of the + * License at + * + * http://www.confluent.io/confluent-community-license + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, WITHOUT + * WARRANTIES OF ANY KIND, either express or implied. See the License for the + * specific language governing permissions and limitations under the License. + */ + +package io.confluent.ksql.serde.protobuf; + +import static org.mockito.Mockito.when; + +import io.confluent.kafka.schemaregistry.client.SchemaRegistryClient; +import io.confluent.ksql.serde.SerdeFactory; +import io.confluent.ksql.util.DecimalUtil; +import io.confluent.ksql.util.KsqlConfig; +import java.util.function.Supplier; +import org.apache.kafka.connect.data.ConnectSchema; +import org.apache.kafka.connect.data.SchemaBuilder; +import org.apache.kafka.connect.data.Struct; +import org.junit.Before; +import org.junit.Test; +import org.mockito.Mock; + +public abstract class AbstractProtobufSerdeFactoryTest { + @Mock + private KsqlConfig config; + @Mock + private Supplier srClientFactory; + + abstract SerdeFactory getSerdeFactory(); + + @Before + public void setUp() throws Exception { + when(config.getString(KsqlConfig.SCHEMA_REGISTRY_URL_PROPERTY)).thenReturn("http://localhost:8088"); + } + + @Test + public void shouldNotThrowOnDecimal() { + // Given: + final ConnectSchema schema = (ConnectSchema) SchemaBuilder.struct() + .field("f0", SchemaBuilder.array(DecimalUtil.builder(10, 2))) + .build(); + + // When: + getSerdeFactory().createSerde(schema, config, srClientFactory, Struct.class, false); + + // Then (did not throw) + } + + @Test + public void shouldNotThrowOnNonDecimal() { + // Given: + final ConnectSchema schema = (ConnectSchema) SchemaBuilder.struct() + .field("f0", SchemaBuilder.array(SchemaBuilder.OPTIONAL_STRING_SCHEMA)) + .build(); + + // When: + getSerdeFactory().createSerde(schema, config, srClientFactory, + Struct.class, false); + + // Then (did not throw) + } +} diff --git a/ksqldb-serde/src/test/java/io/confluent/ksql/serde/protobuf/KsqlProtobufDeserializerTest.java b/ksqldb-serde/src/test/java/io/confluent/ksql/serde/protobuf/KsqlProtobufDeserializerTest.java index c755c999bca8..d30c7840d485 100644 --- a/ksqldb-serde/src/test/java/io/confluent/ksql/serde/protobuf/KsqlProtobufDeserializerTest.java +++ b/ksqldb-serde/src/test/java/io/confluent/ksql/serde/protobuf/KsqlProtobufDeserializerTest.java @@ -16,37 +16,23 @@ package io.confluent.ksql.serde.protobuf; import com.google.common.collect.ImmutableMap; -import java.nio.ByteBuffer; -import java.util.Optional; +import io.confluent.connect.protobuf.ProtobufConverter; +import io.confluent.kafka.schemaregistry.client.MockSchemaRegistryClient; +import io.confluent.kafka.schemaregistry.client.SchemaRegistryClient; +import io.confluent.kafka.serializers.AbstractKafkaSchemaSerDeConfig; +import io.confluent.ksql.util.KsqlConfig; +import java.util.Collections; import org.apache.kafka.common.serialization.Deserializer; import org.apache.kafka.connect.data.ConnectSchema; -import org.apache.kafka.connect.data.Date; import org.apache.kafka.connect.data.Schema; -import org.apache.kafka.connect.data.SchemaBuilder; -import org.apache.kafka.connect.data.Struct; -import org.apache.kafka.connect.data.Time; -import org.apache.kafka.connect.data.Timestamp; +import org.apache.kafka.connect.storage.Converter; import org.junit.Before; -import org.junit.Test; import org.junit.runner.RunWith; import org.mockito.junit.MockitoJUnitRunner; -import java.math.BigDecimal; -import java.util.Collections; - -import io.confluent.connect.protobuf.ProtobufConverter; -import io.confluent.kafka.schemaregistry.client.MockSchemaRegistryClient; -import io.confluent.kafka.schemaregistry.client.SchemaRegistryClient; -import io.confluent.kafka.serializers.AbstractKafkaSchemaSerDeConfig; -import io.confluent.ksql.util.DecimalUtil; -import io.confluent.ksql.util.KsqlConfig; - -import static org.hamcrest.MatcherAssert.assertThat; -import static org.hamcrest.Matchers.is; - @SuppressWarnings("rawtypes") @RunWith(MockitoJUnitRunner.class) -public class KsqlProtobufDeserializerTest { +public class KsqlProtobufDeserializerTest extends AbstractKsqlProtobufDeserializerTest { private static final String SOME_TOPIC = "bob"; @@ -69,107 +55,14 @@ public void setUp() { converter.configure(configs, false); } - @Test - public void shouldDeserializeDecimalField() { - final ConnectSchema schema = (ConnectSchema) SchemaBuilder.struct() - .field("f0", DecimalUtil.builder(10, 2)) - .build(); - - // Given: - final Deserializer deserializer = - givenDeserializerForSchema(schema, - Struct.class); - final Struct value = new Struct(schema).put("f0", new BigDecimal("12.34")); - final byte[] bytes = givenConnectSerialized(value, schema); - - // When: - final Object result = deserializer.deserialize(SOME_TOPIC, bytes); - - // Then: - assertThat(result, is(value)); - } - - @Test - public void shouldDeserializeTimeField() { - final ConnectSchema schema = (ConnectSchema) SchemaBuilder.struct() - .field("f0", Time.SCHEMA) - .build(); - - // Given: - final Deserializer deserializer = - givenDeserializerForSchema(schema, - Struct.class); - final Struct value = new Struct(schema).put("f0", new java.sql.Time(2000)); - final byte[] bytes = givenConnectSerialized(value, schema); - - // When: - final Object result = deserializer.deserialize(SOME_TOPIC, bytes); - - // Then: - assertThat(result, is(value)); - } - - @Test - public void shouldDeserializeDateField() { - final ConnectSchema schema = (ConnectSchema) SchemaBuilder.struct() - .field("f0", Date.SCHEMA) - .build(); - - // Given: - final Deserializer deserializer = - givenDeserializerForSchema(schema, - Struct.class); - final Struct value = new Struct(schema).put("f0", new java.sql.Date(864000000L)); - final byte[] bytes = givenConnectSerialized(value, schema); - - // When: - final Object result = deserializer.deserialize(SOME_TOPIC, bytes); - - // Then: - assertThat(result, is(value)); - } - - @Test - public void shouldDeserializeTimestampField() { - final ConnectSchema schema = (ConnectSchema) SchemaBuilder.struct() - .field("f0", Timestamp.SCHEMA) - .build(); - - // Given: - final Deserializer deserializer = - givenDeserializerForSchema(schema, - Struct.class); - final Struct value = new Struct(schema).put("f0", new java.sql.Timestamp(2000)); - final byte[] bytes = givenConnectSerialized(value, schema); - - // When: - final Object result = deserializer.deserialize(SOME_TOPIC, bytes); - - // Then: - assertThat(result, is(value)); - } - - @Test - public void shouldDeserializeBytesField() { - final ConnectSchema schema = (ConnectSchema) SchemaBuilder.struct() - .field("f0", Schema.BYTES_SCHEMA) - .build(); - - // Given: - final Deserializer deserializer = - givenDeserializerForSchema(schema, - Struct.class); - final Struct value = new Struct(schema).put("f0", ByteBuffer.wrap(new byte[] {123})); - final byte[] bytes = givenConnectSerialized(value, schema); - - // When: - final Object result = deserializer.deserialize(SOME_TOPIC, bytes); - - // Then: - assertThat(((Struct) result).getBytes("f0"), is(value.getBytes("f0"))); + @Override + Converter getConverter(final ConnectSchema schema) { + return converter; } - private byte[] givenConnectSerialized( + @Override + byte[] givenConnectSerialized( + final Converter converter, final Object value, final Schema connectSchema ) { @@ -184,7 +77,8 @@ private byte[] serializeAsBinaryProtobuf( return converter.fromConnectData(topicName, schema, value); } - private Deserializer givenDeserializerForSchema( + @Override + Deserializer givenDeserializerForSchema( final ConnectSchema schema, final Class targetType ) { diff --git a/ksqldb-serde/src/test/java/io/confluent/ksql/serde/protobuf/KsqlProtobufNoSRDeserializerTest.java b/ksqldb-serde/src/test/java/io/confluent/ksql/serde/protobuf/KsqlProtobufNoSRDeserializerTest.java new file mode 100644 index 000000000000..2dd2036fe056 --- /dev/null +++ b/ksqldb-serde/src/test/java/io/confluent/ksql/serde/protobuf/KsqlProtobufNoSRDeserializerTest.java @@ -0,0 +1,65 @@ +/* + * Copyright 2022 Confluent Inc. + * + * Licensed under the Confluent Community License (the "License"); you may not use + * this file except in compliance with the License. You may obtain a copy of the + * License at + * + * http://www.confluent.io/confluent-community-license + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, WITHOUT + * WARRANTIES OF ANY KIND, either express or implied. See the License for the + * specific language governing permissions and limitations under the License. + */ + +package io.confluent.ksql.serde.protobuf; + +import com.google.common.collect.ImmutableMap; +import io.confluent.ksql.util.KsqlConfig; +import java.util.Collections; +import org.apache.kafka.common.serialization.Deserializer; +import org.apache.kafka.connect.data.ConnectSchema; +import org.apache.kafka.connect.data.Schema; +import org.apache.kafka.connect.storage.Converter; +import org.junit.runner.RunWith; +import org.mockito.junit.MockitoJUnitRunner; + +@SuppressWarnings("checkstyle:ClassDataAbstractionCoupling") +@RunWith(MockitoJUnitRunner.class) +public class KsqlProtobufNoSRDeserializerTest extends AbstractKsqlProtobufDeserializerTest { + + @Override + Converter getConverter(final ConnectSchema schema) { + final ProtobufNoSRConverter converter = new ProtobufNoSRConverter(schema); + converter.configure(Collections.emptyMap(), false); + return converter; + } + + @Override + byte[] givenConnectSerialized( + final Converter converter, + final Object value, + final Schema connectSchema + ) { + return converter.fromConnectData(SOME_TOPIC, connectSchema, value); + } + + @Override + Deserializer givenDeserializerForSchema( + final ConnectSchema schema, + final Class targetType + ) { + final Deserializer deserializer = new ProtobufNoSRSerdeFactory(ImmutableMap.of()) + .createSerde( + schema, + new KsqlConfig(ImmutableMap.of()), + () -> null, + targetType, + false).deserializer(); + + deserializer.configure(Collections.emptyMap(), false); + + return deserializer; + } +} diff --git a/ksqldb-serde/src/test/java/io/confluent/ksql/serde/protobuf/KsqlProtobufNoSRSerializerTest.java b/ksqldb-serde/src/test/java/io/confluent/ksql/serde/protobuf/KsqlProtobufNoSRSerializerTest.java new file mode 100644 index 000000000000..9c969a26749d --- /dev/null +++ b/ksqldb-serde/src/test/java/io/confluent/ksql/serde/protobuf/KsqlProtobufNoSRSerializerTest.java @@ -0,0 +1,333 @@ +/* + * Copyright 2021 Confluent Inc. + * + * Licensed under the Confluent Community License (the "License"); you may not use + * this file except in compliance with the License. You may obtain a copy of the + * License at + * + * http://www.confluent.io/confluent-community-license + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, WITHOUT + * WARRANTIES OF ANY KIND, either express or implied. See the License for the + * specific language governing permissions and limitations under the License. + */ + +package io.confluent.ksql.serde.protobuf; + +import static java.nio.charset.StandardCharsets.UTF_8; +import static org.apache.kafka.connect.data.Schema.OPTIONAL_STRING_SCHEMA; +import static org.hamcrest.MatcherAssert.assertThat; +import static org.hamcrest.Matchers.equalTo; +import static org.hamcrest.Matchers.is; + +import com.google.common.collect.ImmutableList; +import com.google.common.collect.ImmutableMap; +import com.google.protobuf.ByteString; +import com.google.protobuf.Descriptors; +import com.google.protobuf.Message; +import com.google.protobuf.Timestamp; +import com.google.type.Date; +import com.google.type.TimeOfDay; +import io.confluent.connect.protobuf.ProtobufData; +import io.confluent.connect.protobuf.ProtobufDataConfig; +import io.confluent.kafka.schemaregistry.ParsedSchema; +import io.confluent.kafka.schemaregistry.protobuf.ProtobufSchema; +import io.confluent.ksql.util.DecimalUtil; +import io.confluent.ksql.util.KsqlConfig; +import io.confluent.protobuf.type.Decimal; +import io.confluent.protobuf.type.utils.DecimalUtils; +import java.math.BigDecimal; +import java.nio.ByteBuffer; +import java.util.Collections; +import org.apache.kafka.common.serialization.Deserializer; +import org.apache.kafka.common.serialization.Serializer; +import org.apache.kafka.connect.data.ConnectSchema; +import org.apache.kafka.connect.data.Schema; +import org.apache.kafka.connect.data.SchemaBuilder; +import org.apache.kafka.connect.data.Struct; +import org.junit.Test; +import org.junit.runner.RunWith; +import org.mockito.junit.MockitoJUnitRunner; + +@SuppressWarnings({"SameParameterValue", "unchecked", "checkstyle:ClassDataAbstractionCoupling"}) +@RunWith(MockitoJUnitRunner.class) +public class KsqlProtobufNoSRSerializerTest { + + private static final ParsedSchema DECIMAL_SCHEMA = + parseProtobufSchema( + "syntax = \"proto3\";\n" + + "\n" + + "import \"confluent/meta.proto\";\n" + + "import \"confluent/type/decimal.proto\";\n" + + "\n" + + "message DecimalValue {\n" + + " confluent.type.Decimal field0 = 1 [(confluent.field_meta) = { " + + "params: [\n" + + " { key: \"precision\", value: \"4\" },\n" + + " { key: \"scale\", value: \"2\" }\n" + + " ]}];\n" + + "}\n"); + private static final ParsedSchema TIME_SCHEMA = + parseProtobufSchema( + "syntax = \"proto3\";\n" + + "\n" + + "import \"google/type/TimeOfDay.proto\";\n" + + "\n" + + "message ConnectDefault1 {google.type.TimeOfDay F1 = 1;}\n"); + private static final ParsedSchema TIMESTAMP_SCHEMA = + parseProtobufSchema( + "syntax = \"proto3\";\n" + + "\n" + + "import \"google/protobuf/timestamp.proto\";\n" + + "\n" + + "message ConnectDefault1 {google.protobuf.Timestamp F1 = 1;}\n"); + + private static final Schema STRUCT_SCHEMA = SchemaBuilder.struct() + .field("id", SchemaBuilder.OPTIONAL_INT64_SCHEMA) + .field("age", SchemaBuilder.OPTIONAL_INT32_SCHEMA) + .field("name", SchemaBuilder.OPTIONAL_STRING_SCHEMA) + .name("StructName") + .optional() + .build(); + + private static final Schema RANDOM_NAME_STRUCT_SCHEMA = SchemaBuilder.struct() + .field("id", SchemaBuilder.OPTIONAL_INT64_SCHEMA) + .field("age", SchemaBuilder.OPTIONAL_INT32_SCHEMA) + .field("name", SchemaBuilder.OPTIONAL_STRING_SCHEMA) + .name("RandomName") + .optional() + .build(); + + private static final String SOME_TOPIC = "bob"; + + @SuppressWarnings("checkstyle:Indentation") + private static final ProtobufNoSRConverter.Deserializer deserializer = + new ProtobufNoSRConverter.Deserializer(); + + @Test + public void shouldUseNestedStructNames() throws Exception { + // Given: + final String schemaName = "TestSchemaName1"; + final String schemaNamespace = "com.test.namespace"; + + final Schema internalStructSchema = SchemaBuilder.struct() + .field("field0_1", OPTIONAL_STRING_SCHEMA) + .build(); + + final Schema internalArraySchema = SchemaBuilder.array(internalStructSchema).build(); + + final Schema internalMapSchema = SchemaBuilder.map( + OPTIONAL_STRING_SCHEMA, + internalStructSchema + ).build(); + + final ConnectSchema ksqlRecordSchema = (ConnectSchema) SchemaBuilder.struct() + .field("field0", internalStructSchema) + .field("field1", internalArraySchema) + .field("field2", internalMapSchema) + .build(); + + final Struct ksqlRecord = new Struct(ksqlRecordSchema) + .put("field0", new Struct(internalStructSchema) + .put("field0_1", "foobar")) + .put("field1", ImmutableList.of( + new Struct(internalStructSchema).put("field0_1", "arrValue1"), + new Struct(internalStructSchema).put("field0_1", "arrValue2"))) + .put("field2", ImmutableMap.of( + "key1", + new Struct(internalStructSchema).put("field0_1", "mapValue1") + )); + + final Serializer serializer = + new ProtobufNoSRSerdeFactory(ImmutableMap.of()).createSerde( + ksqlRecordSchema, + new KsqlConfig(ImmutableMap.of()), + () -> null, + Struct.class, + false).serializer(); + + // When: + final byte[] bytes = serializer.serialize(SOME_TOPIC, ksqlRecord); + + // Then: + final Message deserialized = deserialize(bytes, ksqlRecordSchema); + + assertThat(deserialized.toString(), is("field0 {\n" + + " field0_1: \"foobar\"\n" + + "}\n" + + "field1 {\n" + + " field0_1: \"arrValue1\"\n" + + "}\n" + + "field1 {\n" + + " field0_1: \"arrValue2\"\n" + + "}\n" + + "field2 {\n" + + " key: \"key1\"\n" + + " value {\n" + + " field0_1: \"mapValue1\"\n" + + " }\n" + + "}\n")); + } + + @Test + public void shouldUseSchemaNameFromPropertyIfExists() throws Exception { + // Given: + final String schemaName = "TestSchemaName1"; + final String schemaNamespace = "com.test.namespace"; + + final Object ksqlValue = "foobar"; + + final Schema ksqlRecordSchema = SchemaBuilder.struct() + .field("field0", OPTIONAL_STRING_SCHEMA) + .build(); + + final Struct ksqlRecord = new Struct(ksqlRecordSchema) + .put("field0", ksqlValue); + + final Serializer serializer = + new ProtobufNoSRSerdeFactory(ImmutableMap.of()).createSerde( + ksqlRecordSchema, + new KsqlConfig(ImmutableMap.of()), + () -> null, + Struct.class, + false).serializer(); + + // When: + final byte[] bytes = serializer.serialize(SOME_TOPIC, ksqlRecord); + + // Then: + final Message deserialized = deserialize(bytes, ksqlRecordSchema); + assertThat(deserialized.toString(), is("field0: \"foobar\"\n")); + } + + @Test + public void shouldSerializeDecimalField() { + final BigDecimal decimal = new BigDecimal("12.34"); + final Decimal bytes = DecimalUtils.fromBigDecimal(decimal); + shouldSerializeFieldTypeCorrectly( + DecimalUtil.builder(4, 2).build(), + decimal, + DECIMAL_SCHEMA, + bytes + ); + } + + @Test + public void shouldSerializeTimeField() { + shouldSerializeFieldTypeCorrectly( + org.apache.kafka.connect.data.Time.SCHEMA, + new java.sql.Time(2000), + TIME_SCHEMA, + TimeOfDay.newBuilder().setSeconds(2).build() + ); + } + + @Test + public void shouldSerializeDateField() { + shouldSerializeFieldTypeCorrectly( + org.apache.kafka.connect.data.Date.SCHEMA, + new java.sql.Date(864000000L), + TIME_SCHEMA, + Date.newBuilder().setMonth(1).setDay(11).setYear(1970).build() + ); + } + + @Test + public void shouldSerializeTimestampField() { + shouldSerializeFieldTypeCorrectly( + org.apache.kafka.connect.data.Timestamp.SCHEMA, + new java.sql.Timestamp(2000), + TIMESTAMP_SCHEMA, + Timestamp.newBuilder().setSeconds(2).setNanos(0).build() + ); + } + + @Test + public void shouldSerializeBytesField() { + final Message record = serializeValue(Schema.BYTES_SCHEMA, + ByteBuffer.wrap("abc".getBytes(UTF_8))); + assertThat(record.getAllFields().size(), equalTo(1)); + final Descriptors.FieldDescriptor field = record.getDescriptorForType() + .findFieldByName("field0"); + assertThat(((ByteString) record.getField(field)).toByteArray(), equalTo("abc".getBytes(UTF_8))); + } + + @SuppressWarnings("unchecked") + private Deserializer givenDeserializerForSchema( + final Schema schema, + final Class targetType + ) { + final Deserializer deserializer = new ProtobufNoSRSerdeFactory(ImmutableMap.of()) + .createSerde( + schema, + new KsqlConfig(ImmutableMap.of()), + () -> null, + targetType, + false).deserializer(); + + deserializer.configure(Collections.emptyMap(), false); + + return deserializer; + } + + private void shouldSerializeFieldTypeCorrectly( + final Schema ksqlSchema, + final Object ksqlValue, + final ParsedSchema parsedSchema, + final Object protobufValue + ) { + final Message record = serializeValue(ksqlSchema, ksqlValue); + assertThat(record.getAllFields().size(), equalTo(1)); + final Descriptors.FieldDescriptor field = record + .getDescriptorForType() + .findFieldByName("field0"); + assertThat(record.getField(field).toString(), equalTo(protobufValue.toString())); + } + + private Message serializeValue(final Schema ksqlSchema, final Object ksqlValue) { + // Given: + final Schema schema = SchemaBuilder.struct() + .field("field0", ksqlSchema) + .build(); + + final Serializer serializer = givenSerializerForSchema(schema, Struct.class); + + final Struct ksqlRecord = new Struct(schema) + .put("field0", ksqlValue); + + // When: + final byte[] bytes = serializer.serialize(SOME_TOPIC, ksqlRecord); + + // Then: + return deserialize(bytes, schema); + } + + @SuppressWarnings("unchecked") + private T deserialize(final byte[] serializedRow, final Schema schema) { + final ProtobufSchema protobufSchema = new ProtobufData( + new ProtobufDataConfig(ImmutableMap.of())).fromConnectSchema(schema); + return (T) deserializer.deserialize(serializedRow, protobufSchema); + } + + private Serializer givenSerializerForSchema( + final Schema schema, + final Class targetType + ) { + final Serializer serializer = new ProtobufNoSRSerdeFactory(ImmutableMap.of()) + .createSerde( + schema, + new KsqlConfig(ImmutableMap.of()), + () -> null, + targetType, + false).serializer(); + + serializer.configure(Collections.emptyMap(), false); + + return serializer; + } + + private static ParsedSchema parseProtobufSchema(final String protobufSchema) { + return new ProtobufSchema(protobufSchema); + } +} diff --git a/ksqldb-serde/src/test/java/io/confluent/ksql/serde/protobuf/ProtobufFormatTest.java b/ksqldb-serde/src/test/java/io/confluent/ksql/serde/protobuf/ProtobufFormatTest.java index ab0cab83e4c3..a3e00bae18f9 100644 --- a/ksqldb-serde/src/test/java/io/confluent/ksql/serde/protobuf/ProtobufFormatTest.java +++ b/ksqldb-serde/src/test/java/io/confluent/ksql/serde/protobuf/ProtobufFormatTest.java @@ -1,3 +1,18 @@ +/* + * Copyright 2022 Confluent Inc. + * + * Licensed under the Confluent Community License (the "License"); you may not use + * this file except in compliance with the License. You may obtain a copy of the + * License at + * + * http://www.confluent.io/confluent-community-license + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, WITHOUT + * WARRANTIES OF ANY KIND, either express or implied. See the License for the + * specific language governing permissions and limitations under the License. + */ + package io.confluent.ksql.serde.protobuf; import com.google.common.collect.ImmutableList; diff --git a/ksqldb-serde/src/test/java/io/confluent/ksql/serde/protobuf/ProtobufNoSRFormatTest.java b/ksqldb-serde/src/test/java/io/confluent/ksql/serde/protobuf/ProtobufNoSRFormatTest.java new file mode 100644 index 000000000000..f5a223d974bf --- /dev/null +++ b/ksqldb-serde/src/test/java/io/confluent/ksql/serde/protobuf/ProtobufNoSRFormatTest.java @@ -0,0 +1,64 @@ +/* + * Copyright 2022 Confluent Inc. + * + * Licensed under the Confluent Community License (the "License"); you may not use + * this file except in compliance with the License. You may obtain a copy of the + * License at + * + * http://www.confluent.io/confluent-community-license + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, WITHOUT + * WARRANTIES OF ANY KIND, either express or implied. See the License for the + * specific language governing permissions and limitations under the License. + */ + +package io.confluent.ksql.serde.protobuf; + +import static org.hamcrest.MatcherAssert.assertThat; +import static org.hamcrest.Matchers.equalTo; + +import com.google.common.collect.ImmutableList; +import io.confluent.kafka.schemaregistry.protobuf.ProtobufSchema; +import java.util.List; +import org.junit.Test; + +public class ProtobufNoSRFormatTest { + private static final ProtobufNoSRFormat format = new ProtobufNoSRFormat(); + + @Test + public void shouldReturnSchemaNamesFromMultipleSchemaDefinitionsWithPackageName() { + // Given + final ProtobufSchema protoSchema = new ProtobufSchema("" + + "syntax = \"proto3\"; " + + "package examples.proto; " + + "message ProtobufKey1 {uint32 k1 = 1;} " + + "message ProtobufKey2 {string k1 = 1;}" + ); + + // When + final List schemaNames = format.schemaFullNames(protoSchema); + + // Then + assertThat(schemaNames, equalTo(ImmutableList.of( + "examples.proto.ProtobufKey1", + "examples.proto.ProtobufKey2" + ))); + } + + @Test + public void shouldReturnSchemaNamesFromMultipleSchemaDefinitionsWithoutPackageName() { + // Given + final ProtobufSchema protoSchema = new ProtobufSchema("" + + "syntax = \"proto3\"; " + + "message ProtobufKey1 {uint32 k1 = 1;} " + + "message ProtobufKey2 {string k1 = 1;}" + ); + + // When + final List schemaNames = format.schemaFullNames(protoSchema); + + // Then + assertThat(schemaNames, equalTo(ImmutableList.of("ProtobufKey1", "ProtobufKey2"))); + } +} \ No newline at end of file diff --git a/ksqldb-serde/src/test/java/io/confluent/ksql/serde/protobuf/ProtobufNoSRSerdeFactoryTest.java b/ksqldb-serde/src/test/java/io/confluent/ksql/serde/protobuf/ProtobufNoSRSerdeFactoryTest.java new file mode 100644 index 000000000000..b49274ef82fe --- /dev/null +++ b/ksqldb-serde/src/test/java/io/confluent/ksql/serde/protobuf/ProtobufNoSRSerdeFactoryTest.java @@ -0,0 +1,30 @@ +/* + * Copyright 2022 Confluent Inc. + * + * Licensed under the Confluent Community License (the "License"); you may not use + * this file except in compliance with the License. You may obtain a copy of the + * License at + * + * http://www.confluent.io/confluent-community-license + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, WITHOUT + * WARRANTIES OF ANY KIND, either express or implied. See the License for the + * specific language governing permissions and limitations under the License. + */ + +package io.confluent.ksql.serde.protobuf; + +import com.google.common.collect.ImmutableMap; +import io.confluent.ksql.serde.SerdeFactory; +import org.junit.runner.RunWith; +import org.mockito.junit.MockitoJUnitRunner; + +@RunWith(MockitoJUnitRunner.class) +public class ProtobufNoSRSerdeFactoryTest extends AbstractProtobufSerdeFactoryTest { + + @Override + SerdeFactory getSerdeFactory() { + return new ProtobufSerdeFactory(ImmutableMap.of()); + } +} \ No newline at end of file diff --git a/ksqldb-serde/src/test/java/io/confluent/ksql/serde/protobuf/ProtobufSerdeFactoryTest.java b/ksqldb-serde/src/test/java/io/confluent/ksql/serde/protobuf/ProtobufSerdeFactoryTest.java index cc9254b42b4d..18be22df2bca 100644 --- a/ksqldb-serde/src/test/java/io/confluent/ksql/serde/protobuf/ProtobufSerdeFactoryTest.java +++ b/ksqldb-serde/src/test/java/io/confluent/ksql/serde/protobuf/ProtobufSerdeFactoryTest.java @@ -15,63 +15,16 @@ package io.confluent.ksql.serde.protobuf; -import static org.mockito.Mockito.when; - import com.google.common.collect.ImmutableMap; -import io.confluent.kafka.schemaregistry.client.SchemaRegistryClient; -import io.confluent.ksql.util.DecimalUtil; -import io.confluent.ksql.util.KsqlConfig; - -import java.util.Optional; -import java.util.function.Supplier; - -import org.apache.kafka.connect.data.ConnectSchema; -import org.apache.kafka.connect.data.SchemaBuilder; -import org.apache.kafka.connect.data.Struct; -import org.junit.Before; -import org.junit.Test; +import io.confluent.ksql.serde.SerdeFactory; import org.junit.runner.RunWith; -import org.mockito.Mock; import org.mockito.junit.MockitoJUnitRunner; @RunWith(MockitoJUnitRunner.class) -public class ProtobufSerdeFactoryTest { - - @Mock - private KsqlConfig config; - @Mock - private Supplier srClientFactory; - - @Before - public void setUp() throws Exception { - when(config.getString(KsqlConfig.SCHEMA_REGISTRY_URL_PROPERTY)).thenReturn("http://localhost:8088"); - } - - @Test - public void shouldNotThrowOnDecimal() { - // Given: - final ConnectSchema schema = (ConnectSchema) SchemaBuilder.struct() - .field("f0", SchemaBuilder.array(DecimalUtil.builder(10, 2))) - .build(); - - // When: - new ProtobufSerdeFactory(ImmutableMap.of()).createSerde(schema, config, - srClientFactory, Struct.class, false); - - // Then (did not throw) - } - - @Test - public void shouldNotThrowOnNonDecimal() { - // Given: - final ConnectSchema schema = (ConnectSchema) SchemaBuilder.struct() - .field("f0", SchemaBuilder.array(SchemaBuilder.OPTIONAL_STRING_SCHEMA)) - .build(); - - // When: - new ProtobufSerdeFactory(ImmutableMap.of()).createSerde(schema, config, srClientFactory, - Struct.class, false); +public class ProtobufSerdeFactoryTest extends AbstractProtobufSerdeFactoryTest { - // Then (did not throw) + @Override + SerdeFactory getSerdeFactory() { + return new ProtobufSerdeFactory(ImmutableMap.of()); } } \ No newline at end of file diff --git a/ksqldb-streams/src/main/java/io/confluent/ksql/execution/streams/materialization/ks/KsLocator.java b/ksqldb-streams/src/main/java/io/confluent/ksql/execution/streams/materialization/ks/KsLocator.java index 416e199ac604..d6d18a2a8588 100644 --- a/ksqldb-streams/src/main/java/io/confluent/ksql/execution/streams/materialization/ks/KsLocator.java +++ b/ksqldb-streams/src/main/java/io/confluent/ksql/execution/streams/materialization/ks/KsLocator.java @@ -128,6 +128,15 @@ public List locate( metadata = getMetadataForAllPartitions(filterPartitions, keySet); } + if (metadata.isEmpty()) { + final MaterializationException materializationException = new MaterializationException( + "Cannot determine which host contains the required partitions to serve the pull query. \n" + + "The underlying persistent query may be restarting (e.g. as a result of " + + "ALTER SYSTEM) view the status of your by issuing ."); + LOG.debug(materializationException.getMessage()); + throw materializationException; + } + // Go through the metadata and group them by partition. for (PartitionMetadata partitionMetadata : metadata) { LOG.debug("Handling pull query for partition {} of state store {}.", diff --git a/ksqldb-streams/src/test/java/io/confluent/ksql/execution/streams/materialization/ks/KsLocatorTest.java b/ksqldb-streams/src/test/java/io/confluent/ksql/execution/streams/materialization/ks/KsLocatorTest.java index 625cf3a33168..52dffa9e0bde 100644 --- a/ksqldb-streams/src/test/java/io/confluent/ksql/execution/streams/materialization/ks/KsLocatorTest.java +++ b/ksqldb-streams/src/test/java/io/confluent/ksql/execution/streams/materialization/ks/KsLocatorTest.java @@ -226,6 +226,29 @@ public void shouldThrowIfMetadataNotAvailable() { "Materialized data for key [1] is not available yet. Please try again later.")); } + @Test + public void shouldThrowIfMetadataIsEmpty() { + // Given: + getActiveAndStandbyMetadata(); + when(topology.describe()).thenReturn(description); + when(description.subtopologies()).thenReturn(ImmutableSet.of(sub1)); + when(sub1.nodes()).thenReturn(ImmutableSet.of(source, processor)); + when(source.topicSet()).thenReturn(ImmutableSet.of(TOPIC_NAME)); + when(processor.stores()).thenReturn(ImmutableSet.of(STORE_NAME)); + + // When: + final Exception e = assertThrows( + MaterializationException.class, + () -> locator.locate(Collections.emptyList(), routingOptions, routingFilterFactoryActive, false) + ); + + // Then: + assertThat(e.getMessage(), is( + "Cannot determine which host contains the required partitions to serve the pull query. \n" + + "The underlying persistent query may be restarting (e.g. as a result of" + + " ALTER SYSTEM) view the status of your by issuing .")); + } + @Test public void shouldThrowIfRangeScanAndKeysEmpty() { // Given: diff --git a/mkdocs.yml b/mkdocs.yml index 62e0e76cd88b..183517168fb0 100644 --- a/mkdocs.yml +++ b/mkdocs.yml @@ -259,9 +259,9 @@ extra: # Build-related string tokens kafkaversion: 3.1 - ksqldbversion: 0.25.1 - cprelease: 7.1.0 - releasepostbranch: 7.1.0-post + ksqldbversion: 0.26.0 + cprelease: 7.2.0 + releasepostbranch: 7.2.0-post scalaversion: 2.13 voluble_version: 0.3.1 jdbc_connector_version: 10.0.0 diff --git a/pom.xml b/pom.xml index 815a9cdf0a2d..9ac539b2328e 100644 --- a/pom.xml +++ b/pom.xml @@ -155,6 +155,8 @@ 4.1.72.Final 4.1.72.Final 2.34 + + 3.0.0-M5