From 40ae0748c3e259086c97f7c03e598dbaa6601348 Mon Sep 17 00:00:00 2001 From: Adam Czajkowski Date: Mon, 13 Jan 2025 12:19:07 +0100 Subject: [PATCH 01/76] feat: Kafka Connect Cloud BigTable Sink --- .github/workflows/ci.yaml | 88 +- .../.gitignore | 3 + .../LICENSE | 201 +++++ .../assembly_package.xml | 18 + .../license.header | 15 + .../pom.xml | 279 ++++++ .../bigtable/BigtableSinkConnector.java | 83 ++ .../connect/bigtable/BigtableSinkTask.java | 471 ++++++++++ .../autocreate/BigtableSchemaManager.java | 470 ++++++++++ .../autocreate/ResourceCreationResult.java | 51 ++ .../bigtable/config/BigtableErrorMode.java | 22 + .../bigtable/config/BigtableSinkConfig.java | 548 ++++++++++++ .../config/BigtableSinkTaskConfig.java | 49 + .../connect/bigtable/config/InsertMode.java | 21 + .../bigtable/config/NullValueMode.java | 22 + .../bigtable/exception/BatchException.java | 28 + ...idBigtableSchemaModificationException.java | 29 + .../connect/bigtable/mapping/KeyMapper.java | 262 ++++++ .../bigtable/mapping/MutationData.java | 63 ++ .../bigtable/mapping/MutationDataBuilder.java | 81 ++ .../connect/bigtable/mapping/ValueMapper.java | 245 +++++ .../bigtable/version/PackageMetadata.java | 39 + .../bigtable/BigtableSinkConnectorTest.java | 84 ++ .../bigtable/BigtableSinkTaskTest.java | 612 +++++++++++++ .../autocreate/BigtableSchemaManagerTest.java | 749 ++++++++++++++++ .../autocreate/SchemaApiExceptionsTest.java | 143 +++ .../config/BigtableSinkConfigTest.java | 157 ++++ .../connect/bigtable/integration/BaseIT.java | 142 +++ .../connect/bigtable/integration/BasicIT.java | 81 ++ .../bigtable/integration/ConfigIT.java | 46 + .../bigtable/integration/VersionIT.java | 53 ++ .../bigtable/mapping/KeyMapperTest.java | 835 ++++++++++++++++++ .../mapping/MutationDataBuilderTest.java | 102 +++ .../bigtable/mapping/ValueMapperTest.java | 751 ++++++++++++++++ .../bigtable/util/ApiExceptionFactory.java | 35 + .../bigtable/util/BasicPropertiesFactory.java | 36 + .../connect/bigtable/util/FutureUtil.java | 33 + .../bigtable/util/JsonConverterFactory.java | 29 + .../kafka/connect/bigtable/util/MockUtil.java | 29 + .../kafka/connect/bigtable/util/TestId.java | 32 + .../src/test/resources/fake_service_key.json | 9 + 41 files changed, 7006 insertions(+), 40 deletions(-) create mode 100644 google-cloud-bigtable-kafka-connect-sink/.gitignore create mode 100644 google-cloud-bigtable-kafka-connect-sink/LICENSE create mode 100644 google-cloud-bigtable-kafka-connect-sink/assembly_package.xml create mode 100644 google-cloud-bigtable-kafka-connect-sink/license.header create mode 100644 google-cloud-bigtable-kafka-connect-sink/pom.xml create mode 100644 google-cloud-bigtable-kafka-connect-sink/src/main/java/com/google/cloud/kafka/connect/bigtable/BigtableSinkConnector.java create mode 100644 google-cloud-bigtable-kafka-connect-sink/src/main/java/com/google/cloud/kafka/connect/bigtable/BigtableSinkTask.java create mode 100644 google-cloud-bigtable-kafka-connect-sink/src/main/java/com/google/cloud/kafka/connect/bigtable/autocreate/BigtableSchemaManager.java create mode 100644 google-cloud-bigtable-kafka-connect-sink/src/main/java/com/google/cloud/kafka/connect/bigtable/autocreate/ResourceCreationResult.java create mode 100644 google-cloud-bigtable-kafka-connect-sink/src/main/java/com/google/cloud/kafka/connect/bigtable/config/BigtableErrorMode.java create mode 100644 google-cloud-bigtable-kafka-connect-sink/src/main/java/com/google/cloud/kafka/connect/bigtable/config/BigtableSinkConfig.java create mode 100644 google-cloud-bigtable-kafka-connect-sink/src/main/java/com/google/cloud/kafka/connect/bigtable/config/BigtableSinkTaskConfig.java create mode 100644 google-cloud-bigtable-kafka-connect-sink/src/main/java/com/google/cloud/kafka/connect/bigtable/config/InsertMode.java create mode 100644 google-cloud-bigtable-kafka-connect-sink/src/main/java/com/google/cloud/kafka/connect/bigtable/config/NullValueMode.java create mode 100644 google-cloud-bigtable-kafka-connect-sink/src/main/java/com/google/cloud/kafka/connect/bigtable/exception/BatchException.java create mode 100644 google-cloud-bigtable-kafka-connect-sink/src/main/java/com/google/cloud/kafka/connect/bigtable/exception/InvalidBigtableSchemaModificationException.java create mode 100644 google-cloud-bigtable-kafka-connect-sink/src/main/java/com/google/cloud/kafka/connect/bigtable/mapping/KeyMapper.java create mode 100644 google-cloud-bigtable-kafka-connect-sink/src/main/java/com/google/cloud/kafka/connect/bigtable/mapping/MutationData.java create mode 100644 google-cloud-bigtable-kafka-connect-sink/src/main/java/com/google/cloud/kafka/connect/bigtable/mapping/MutationDataBuilder.java create mode 100644 google-cloud-bigtable-kafka-connect-sink/src/main/java/com/google/cloud/kafka/connect/bigtable/mapping/ValueMapper.java create mode 100644 google-cloud-bigtable-kafka-connect-sink/src/main/java/com/google/cloud/kafka/connect/bigtable/version/PackageMetadata.java create mode 100644 google-cloud-bigtable-kafka-connect-sink/src/test/java/com/google/cloud/kafka/connect/bigtable/BigtableSinkConnectorTest.java create mode 100644 google-cloud-bigtable-kafka-connect-sink/src/test/java/com/google/cloud/kafka/connect/bigtable/BigtableSinkTaskTest.java create mode 100644 google-cloud-bigtable-kafka-connect-sink/src/test/java/com/google/cloud/kafka/connect/bigtable/autocreate/BigtableSchemaManagerTest.java create mode 100644 google-cloud-bigtable-kafka-connect-sink/src/test/java/com/google/cloud/kafka/connect/bigtable/autocreate/SchemaApiExceptionsTest.java create mode 100644 google-cloud-bigtable-kafka-connect-sink/src/test/java/com/google/cloud/kafka/connect/bigtable/config/BigtableSinkConfigTest.java create mode 100644 google-cloud-bigtable-kafka-connect-sink/src/test/java/com/google/cloud/kafka/connect/bigtable/integration/BaseIT.java create mode 100644 google-cloud-bigtable-kafka-connect-sink/src/test/java/com/google/cloud/kafka/connect/bigtable/integration/BasicIT.java create mode 100644 google-cloud-bigtable-kafka-connect-sink/src/test/java/com/google/cloud/kafka/connect/bigtable/integration/ConfigIT.java create mode 100644 google-cloud-bigtable-kafka-connect-sink/src/test/java/com/google/cloud/kafka/connect/bigtable/integration/VersionIT.java create mode 100644 google-cloud-bigtable-kafka-connect-sink/src/test/java/com/google/cloud/kafka/connect/bigtable/mapping/KeyMapperTest.java create mode 100644 google-cloud-bigtable-kafka-connect-sink/src/test/java/com/google/cloud/kafka/connect/bigtable/mapping/MutationDataBuilderTest.java create mode 100644 google-cloud-bigtable-kafka-connect-sink/src/test/java/com/google/cloud/kafka/connect/bigtable/mapping/ValueMapperTest.java create mode 100644 google-cloud-bigtable-kafka-connect-sink/src/test/java/com/google/cloud/kafka/connect/bigtable/util/ApiExceptionFactory.java create mode 100644 google-cloud-bigtable-kafka-connect-sink/src/test/java/com/google/cloud/kafka/connect/bigtable/util/BasicPropertiesFactory.java create mode 100644 google-cloud-bigtable-kafka-connect-sink/src/test/java/com/google/cloud/kafka/connect/bigtable/util/FutureUtil.java create mode 100644 google-cloud-bigtable-kafka-connect-sink/src/test/java/com/google/cloud/kafka/connect/bigtable/util/JsonConverterFactory.java create mode 100644 google-cloud-bigtable-kafka-connect-sink/src/test/java/com/google/cloud/kafka/connect/bigtable/util/MockUtil.java create mode 100644 google-cloud-bigtable-kafka-connect-sink/src/test/java/com/google/cloud/kafka/connect/bigtable/util/TestId.java create mode 100644 google-cloud-bigtable-kafka-connect-sink/src/test/resources/fake_service_key.json diff --git a/.github/workflows/ci.yaml b/.github/workflows/ci.yaml index b91fa381f5..3684867e24 100644 --- a/.github/workflows/ci.yaml +++ b/.github/workflows/ci.yaml @@ -33,12 +33,12 @@ jobs: distribution: temurin java-version: ${{matrix.java}} - run: java -version - - run: .kokoro/build.sh - env: - JOB_TYPE: test - units-java8: - # Building using Java 17 and run the tests with Java 8 runtime - name: "units (8)" + - run: mvn -B -ntp test + # TODO: remove in the target repo + working-directory: google-cloud-bigtable-kafka-connect-sink + units-java11: + # Building using Java 21 and run the tests with Java 11 runtime + name: "units (11)" runs-on: ubuntu-latest steps: - uses: actions/checkout@v4 @@ -53,71 +53,79 @@ jobs: shell: bash - uses: actions/setup-java@v4 with: - java-version: 17 + java-version: 21 distribution: temurin - - run: .kokoro/build.sh - env: - JOB_TYPE: test - windows: - runs-on: windows-latest + - run: mvn -B -ntp test + # TODO: remove in the target repo + working-directory: google-cloud-bigtable-kafka-connect-sink + integrations: + runs-on: ubuntu-latest steps: - - name: Support longpaths - run: git config --system core.longpaths true - uses: actions/checkout@v4 - uses: actions/setup-java@v4 with: distribution: temurin - java-version: 8 + java-version: 11 - run: java -version - - run: .kokoro/build.bat - env: - JOB_TYPE: test - dependencies: + - name: 'Set up Cloud SDK' + uses: 'google-github-actions/setup-gcloud@v2' + with: + version: latest + install_components: beta,bigtable + - run: | + set -euo pipefail + gcloud beta emulators bigtable start --host-port=0.0.0.0:8086 & + mvn -B -ntp verify -DskipUnitTests + # TODO: remove in the target repo + working-directory: google-cloud-bigtable-kafka-connect-sink + package: runs-on: ubuntu-latest - strategy: - matrix: - java: [17] steps: - uses: actions/checkout@v4 - uses: actions/setup-java@v4 with: distribution: temurin - java-version: ${{matrix.java}} + java-version: 11 - run: java -version - - run: .kokoro/dependencies.sh - javadoc: - runs-on: ubuntu-latest + - run: mvn -B -ntp package -Dmaven.test.skip + # TODO: remove in the target repo + working-directory: google-cloud-bigtable-kafka-connect-sink + windows: + runs-on: windows-latest steps: + - name: Support longpaths + run: git config --system core.longpaths true - uses: actions/checkout@v4 - uses: actions/setup-java@v4 with: distribution: temurin - java-version: 17 + java-version: 11 - run: java -version - - run: .kokoro/build.sh - env: - JOB_TYPE: javadoc - lint: + - run: mvn -B -ntp test + # TODO: remove in the target repo + working-directory: google-cloud-bigtable-kafka-connect-sink + javadoc: runs-on: ubuntu-latest steps: - uses: actions/checkout@v4 - uses: actions/setup-java@v4 with: distribution: temurin - java-version: 11 + java-version: 17 - run: java -version - - run: .kokoro/build.sh - env: - JOB_TYPE: lint - clirr: + # TODO: also run javadoc:test-javadoc? + - run: mvn -B -ntp javadoc:javadoc + # TODO: remove in the target repo + working-directory: google-cloud-bigtable-kafka-connect-sink + lint: runs-on: ubuntu-latest steps: - uses: actions/checkout@v4 - uses: actions/setup-java@v4 with: distribution: temurin - java-version: 8 + java-version: 11 - run: java -version - - run: .kokoro/build.sh - env: - JOB_TYPE: clirr + - run: mvn -B -ntp spotless:check + # TODO: remove in the target repo + working-directory: google-cloud-bigtable-kafka-connect-sink diff --git a/google-cloud-bigtable-kafka-connect-sink/.gitignore b/google-cloud-bigtable-kafka-connect-sink/.gitignore new file mode 100644 index 0000000000..612c5bc968 --- /dev/null +++ b/google-cloud-bigtable-kafka-connect-sink/.gitignore @@ -0,0 +1,3 @@ +target +.idea +*.iml diff --git a/google-cloud-bigtable-kafka-connect-sink/LICENSE b/google-cloud-bigtable-kafka-connect-sink/LICENSE new file mode 100644 index 0000000000..261eeb9e9f --- /dev/null +++ b/google-cloud-bigtable-kafka-connect-sink/LICENSE @@ -0,0 +1,201 @@ + Apache License + Version 2.0, January 2004 + http://www.apache.org/licenses/ + + TERMS AND CONDITIONS FOR USE, REPRODUCTION, AND DISTRIBUTION + + 1. Definitions. + + "License" shall mean the terms and conditions for use, reproduction, + and distribution as defined by Sections 1 through 9 of this document. + + "Licensor" shall mean the copyright owner or entity authorized by + the copyright owner that is granting the License. + + "Legal Entity" shall mean the union of the acting entity and all + other entities that control, are controlled by, or are under common + control with that entity. For the purposes of this definition, + "control" means (i) the power, direct or indirect, to cause the + direction or management of such entity, whether by contract or + otherwise, or (ii) ownership of fifty percent (50%) or more of the + outstanding shares, or (iii) beneficial ownership of such entity. + + "You" (or "Your") shall mean an individual or Legal Entity + exercising permissions granted by this License. + + "Source" form shall mean the preferred form for making modifications, + including but not limited to software source code, documentation + source, and configuration files. + + "Object" form shall mean any form resulting from mechanical + transformation or translation of a Source form, including but + not limited to compiled object code, generated documentation, + and conversions to other media types. + + "Work" shall mean the work of authorship, whether in Source or + Object form, made available under the License, as indicated by a + copyright notice that is included in or attached to the work + (an example is provided in the Appendix below). + + "Derivative Works" shall mean any work, whether in Source or Object + form, that is based on (or derived from) the Work and for which the + editorial revisions, annotations, elaborations, or other modifications + represent, as a whole, an original work of authorship. For the purposes + of this License, Derivative Works shall not include works that remain + separable from, or merely link (or bind by name) to the interfaces of, + the Work and Derivative Works thereof. + + "Contribution" shall mean any work of authorship, including + the original version of the Work and any modifications or additions + to that Work or Derivative Works thereof, that is intentionally + submitted to Licensor for inclusion in the Work by the copyright owner + or by an individual or Legal Entity authorized to submit on behalf of + the copyright owner. For the purposes of this definition, "submitted" + means any form of electronic, verbal, or written communication sent + to the Licensor or its representatives, including but not limited to + communication on electronic mailing lists, source code control systems, + and issue tracking systems that are managed by, or on behalf of, the + Licensor for the purpose of discussing and improving the Work, but + excluding communication that is conspicuously marked or otherwise + designated in writing by the copyright owner as "Not a Contribution." + + "Contributor" shall mean Licensor and any individual or Legal Entity + on behalf of whom a Contribution has been received by Licensor and + subsequently incorporated within the Work. + + 2. Grant of Copyright License. Subject to the terms and conditions of + this License, each Contributor hereby grants to You a perpetual, + worldwide, non-exclusive, no-charge, royalty-free, irrevocable + copyright license to reproduce, prepare Derivative Works of, + publicly display, publicly perform, sublicense, and distribute the + Work and such Derivative Works in Source or Object form. + + 3. Grant of Patent License. Subject to the terms and conditions of + this License, each Contributor hereby grants to You a perpetual, + worldwide, non-exclusive, no-charge, royalty-free, irrevocable + (except as stated in this section) patent license to make, have made, + use, offer to sell, sell, import, and otherwise transfer the Work, + where such license applies only to those patent claims licensable + by such Contributor that are necessarily infringed by their + Contribution(s) alone or by combination of their Contribution(s) + with the Work to which such Contribution(s) was submitted. If You + institute patent litigation against any entity (including a + cross-claim or counterclaim in a lawsuit) alleging that the Work + or a Contribution incorporated within the Work constitutes direct + or contributory patent infringement, then any patent licenses + granted to You under this License for that Work shall terminate + as of the date such litigation is filed. + + 4. Redistribution. You may reproduce and distribute copies of the + Work or Derivative Works thereof in any medium, with or without + modifications, and in Source or Object form, provided that You + meet the following conditions: + + (a) You must give any other recipients of the Work or + Derivative Works a copy of this License; and + + (b) You must cause any modified files to carry prominent notices + stating that You changed the files; and + + (c) You must retain, in the Source form of any Derivative Works + that You distribute, all copyright, patent, trademark, and + attribution notices from the Source form of the Work, + excluding those notices that do not pertain to any part of + the Derivative Works; and + + (d) If the Work includes a "NOTICE" text file as part of its + distribution, then any Derivative Works that You distribute must + include a readable copy of the attribution notices contained + within such NOTICE file, excluding those notices that do not + pertain to any part of the Derivative Works, in at least one + of the following places: within a NOTICE text file distributed + as part of the Derivative Works; within the Source form or + documentation, if provided along with the Derivative Works; or, + within a display generated by the Derivative Works, if and + wherever such third-party notices normally appear. The contents + of the NOTICE file are for informational purposes only and + do not modify the License. You may add Your own attribution + notices within Derivative Works that You distribute, alongside + or as an addendum to the NOTICE text from the Work, provided + that such additional attribution notices cannot be construed + as modifying the License. + + You may add Your own copyright statement to Your modifications and + may provide additional or different license terms and conditions + for use, reproduction, or distribution of Your modifications, or + for any such Derivative Works as a whole, provided Your use, + reproduction, and distribution of the Work otherwise complies with + the conditions stated in this License. + + 5. Submission of Contributions. Unless You explicitly state otherwise, + any Contribution intentionally submitted for inclusion in the Work + by You to the Licensor shall be under the terms and conditions of + this License, without any additional terms or conditions. + Notwithstanding the above, nothing herein shall supersede or modify + the terms of any separate license agreement you may have executed + with Licensor regarding such Contributions. + + 6. Trademarks. This License does not grant permission to use the trade + names, trademarks, service marks, or product names of the Licensor, + except as required for reasonable and customary use in describing the + origin of the Work and reproducing the content of the NOTICE file. + + 7. Disclaimer of Warranty. Unless required by applicable law or + agreed to in writing, Licensor provides the Work (and each + Contributor provides its Contributions) on an "AS IS" BASIS, + WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or + implied, including, without limitation, any warranties or conditions + of TITLE, NON-INFRINGEMENT, MERCHANTABILITY, or FITNESS FOR A + PARTICULAR PURPOSE. You are solely responsible for determining the + appropriateness of using or redistributing the Work and assume any + risks associated with Your exercise of permissions under this License. + + 8. Limitation of Liability. In no event and under no legal theory, + whether in tort (including negligence), contract, or otherwise, + unless required by applicable law (such as deliberate and grossly + negligent acts) or agreed to in writing, shall any Contributor be + liable to You for damages, including any direct, indirect, special, + incidental, or consequential damages of any character arising as a + result of this License or out of the use or inability to use the + Work (including but not limited to damages for loss of goodwill, + work stoppage, computer failure or malfunction, or any and all + other commercial damages or losses), even if such Contributor + has been advised of the possibility of such damages. + + 9. Accepting Warranty or Additional Liability. While redistributing + the Work or Derivative Works thereof, You may choose to offer, + and charge a fee for, acceptance of support, warranty, indemnity, + or other liability obligations and/or rights consistent with this + License. However, in accepting such obligations, You may act only + on Your own behalf and on Your sole responsibility, not on behalf + of any other Contributor, and only if You agree to indemnify, + defend, and hold each Contributor harmless for any liability + incurred by, or claims asserted against, such Contributor by reason + of your accepting any such warranty or additional liability. + + END OF TERMS AND CONDITIONS + + APPENDIX: How to apply the Apache License to your work. + + To apply the Apache License to your work, attach the following + boilerplate notice, with the fields enclosed by brackets "[]" + replaced with your own identifying information. (Don't include + the brackets!) The text should be enclosed in the appropriate + comment syntax for the file format. We also recommend that a + file or class name and description of purpose be included on the + same "printed page" as the copyright notice for easier + identification within third-party archives. + + Copyright [yyyy] [name of copyright owner] + + Licensed under the Apache License, Version 2.0 (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.apache.org/licenses/LICENSE-2.0 + + Unless required by applicable law or agreed to in writing, software + distributed under the License is distributed on an "AS IS" BASIS, + WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + See the License for the specific language governing permissions and + limitations under the License. diff --git a/google-cloud-bigtable-kafka-connect-sink/assembly_package.xml b/google-cloud-bigtable-kafka-connect-sink/assembly_package.xml new file mode 100644 index 0000000000..1a4de4100f --- /dev/null +++ b/google-cloud-bigtable-kafka-connect-sink/assembly_package.xml @@ -0,0 +1,18 @@ + + package + + dir + + false + + + / + true + true + runtime + false + + + \ No newline at end of file diff --git a/google-cloud-bigtable-kafka-connect-sink/license.header b/google-cloud-bigtable-kafka-connect-sink/license.header new file mode 100644 index 0000000000..370494894b --- /dev/null +++ b/google-cloud-bigtable-kafka-connect-sink/license.header @@ -0,0 +1,15 @@ +/* + * Copyright $YEAR Google LLC + * + * Licensed under the Apache License, Version 2.0 (the "License"); + * you may not use this file except in compliance with the License. + * You may obtain a copy of the License at + * + * https://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ diff --git a/google-cloud-bigtable-kafka-connect-sink/pom.xml b/google-cloud-bigtable-kafka-connect-sink/pom.xml new file mode 100644 index 0000000000..30340f29e0 --- /dev/null +++ b/google-cloud-bigtable-kafka-connect-sink/pom.xml @@ -0,0 +1,279 @@ + + + 4.0.0 + com.google.cloud.kafka.connect.bigtable + sink + 1.0.0-SNAPSHOT + jar + kafka-connect-bigtable-sink + + Google Bigtable sink connector for Apache Kafka Connect + + + 26.31.0 + 3.8.1 + 2.12 + 2.6.1 + 1.7.36 + 2.16.2 + + 5.14.2 + 4.13.2 + + 5.11.3 + 2.43.0 + 1.19.2 + 3.11.2 + 0.8.12 + 3.5.2 + 3.5.2 + 3.4.2 + + 11 + 11 + UTF-8 + + + + + com.google.cloud + libraries-bom + ${google.cloud.bom.version} + pom + import + + + + + + org.apache.kafka + connect-api + ${kafka.version} + provided + + + org.apache.kafka + connect-runtime + ${kafka.version} + provided + + + org.apache.kafka + kafka-clients + ${kafka.version} + + + com.google.cloud + google-cloud-bigtable + + + org.apache.hbase + hbase-common + ${hbase.version} + + + org.slf4j + slf4j-api + ${slf4j.version} + + + org.slf4j + slf4j-simple + ${slf4j.version} + + + com.fasterxml.jackson.core + jackson-databind + ${jackson.version} + + + junit + junit + ${junit.version} + test + + + org.apache.kafka + connect-runtime + ${kafka.version} + test + test + + + org.apache.kafka + kafka-clients + ${kafka.version} + test + test-jar + test + + + org.apache.kafka + kafka_${kafka.scala.version} + ${kafka.version} + test + + + org.apache.kafka + kafka_${kafka.scala.version} + ${kafka.version} + test + test-jar + test + + + org.mockito + mockito-core + ${mockito.version} + test + + + org.junit.jupiter + junit-jupiter-api + ${junit.jupiter.version} + test + + + + + + + org.apache.maven.plugins + maven-assembly-plugin + + + assembly_package.xml + + + + + make-assembly + package + + single + + + + + + com.diffplug.spotless + spotless-maven-plugin + ${spotless.version} + + + + + .gitignore + + + + + true + 4 + + + + + + ${google.java.format.version} + + true + true + + + ${project.basedir}/license.header + + + + + + + org.apache.maven.plugins + maven-surefire-plugin + ${surefire.version} + + + ${skipUnitTests} + false + + classes + true + + + target/test-classes/fake_service_key.json + + + + + org.apache.maven.plugins + maven-failsafe-plugin + ${failsafe.version} + + + ${skipIntegrationTests} + false + + classes + true + + + target/test-classes/fake_service_key.json + + localhost:8086 + + + + + + verify + integration-test + + + + + + org.apache.maven.plugins + maven-jar-plugin + ${maven-jar.version} + + + + true + + + + + + org.jacoco + jacoco-maven-plugin + ${jacoco.version} + + + + prepare-agent + + + + + + + + + + org.apache.maven.plugins + maven-javadoc-plugin + ${javadoc.version} + + none + protected + true + + + + + \ No newline at end of file diff --git a/google-cloud-bigtable-kafka-connect-sink/src/main/java/com/google/cloud/kafka/connect/bigtable/BigtableSinkConnector.java b/google-cloud-bigtable-kafka-connect-sink/src/main/java/com/google/cloud/kafka/connect/bigtable/BigtableSinkConnector.java new file mode 100644 index 0000000000..183686c6e3 --- /dev/null +++ b/google-cloud-bigtable-kafka-connect-sink/src/main/java/com/google/cloud/kafka/connect/bigtable/BigtableSinkConnector.java @@ -0,0 +1,83 @@ +/* + * Copyright 2024 Google LLC + * + * Licensed under the Apache License, Version 2.0 (the "License"); + * you may not use this file except in compliance with the License. + * You may obtain a copy of the License at + * + * https://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ +package com.google.cloud.kafka.connect.bigtable; + +import com.google.cloud.kafka.connect.bigtable.config.BigtableSinkConfig; +import com.google.cloud.kafka.connect.bigtable.config.BigtableSinkTaskConfig; +import com.google.cloud.kafka.connect.bigtable.version.PackageMetadata; +import java.util.ArrayList; +import java.util.HashMap; +import java.util.List; +import java.util.Map; +import org.apache.kafka.common.config.Config; +import org.apache.kafka.common.config.ConfigDef; +import org.apache.kafka.connect.connector.Task; +import org.apache.kafka.connect.sink.SinkConnector; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +/** {@link SinkConnector} class for Cloud Bigtable. */ +public class BigtableSinkConnector extends SinkConnector { + private Map configProperties; + private final Logger logger = LoggerFactory.getLogger(BigtableSinkConnector.class); + + @Override + public ConfigDef config() { + logger.trace("config()"); + return BigtableSinkConfig.getDefinition(); + } + + @Override + public Config validate(Map properties) { + logger.trace("validate()"); + return BigtableSinkConfig.validate(properties); + } + + @Override + public void start(Map props) { + logger.trace("start()"); + configProperties = props; + } + + @Override + public void stop() { + logger.trace("stop()"); + } + + @Override + public Class taskClass() { + logger.trace("taskClass()"); + return BigtableSinkTask.class; + } + + @Override + public List> taskConfigs(int maxTasks) { + logger.trace("taskClass({})", maxTasks); + List> configs = new ArrayList<>(maxTasks); + for (int i = 0; i < maxTasks; i++) { + Map config = new HashMap<>(configProperties); + config.put(BigtableSinkTaskConfig.CONFIG_TASK_ID, Integer.toString(i)); + configs.add(config); + } + return configs; + } + + @Override + public String version() { + logger.trace("version()"); + return PackageMetadata.getVersion(); + } +} diff --git a/google-cloud-bigtable-kafka-connect-sink/src/main/java/com/google/cloud/kafka/connect/bigtable/BigtableSinkTask.java b/google-cloud-bigtable-kafka-connect-sink/src/main/java/com/google/cloud/kafka/connect/bigtable/BigtableSinkTask.java new file mode 100644 index 0000000000..e474518e55 --- /dev/null +++ b/google-cloud-bigtable-kafka-connect-sink/src/main/java/com/google/cloud/kafka/connect/bigtable/BigtableSinkTask.java @@ -0,0 +1,471 @@ +/* + * Copyright 2024 Google LLC + * + * Licensed under the Apache License, Version 2.0 (the "License"); + * you may not use this file except in compliance with the License. + * You may obtain a copy of the License at + * + * https://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ +package com.google.cloud.kafka.connect.bigtable; + +import com.google.api.core.ApiFuture; +import com.google.api.core.ApiFutures; +import com.google.api.gax.batching.Batcher; +import com.google.api.gax.rpc.ApiException; +import com.google.cloud.bigtable.admin.v2.BigtableTableAdminClient; +import com.google.cloud.bigtable.data.v2.BigtableDataClient; +import com.google.cloud.bigtable.data.v2.models.ConditionalRowMutation; +import com.google.cloud.bigtable.data.v2.models.Filters; +import com.google.cloud.bigtable.data.v2.models.RowMutationEntry; +import com.google.cloud.kafka.connect.bigtable.autocreate.BigtableSchemaManager; +import com.google.cloud.kafka.connect.bigtable.autocreate.ResourceCreationResult; +import com.google.cloud.kafka.connect.bigtable.config.BigtableErrorMode; +import com.google.cloud.kafka.connect.bigtable.config.BigtableSinkConfig; +import com.google.cloud.kafka.connect.bigtable.config.BigtableSinkTaskConfig; +import com.google.cloud.kafka.connect.bigtable.exception.BatchException; +import com.google.cloud.kafka.connect.bigtable.exception.InvalidBigtableSchemaModificationException; +import com.google.cloud.kafka.connect.bigtable.mapping.KeyMapper; +import com.google.cloud.kafka.connect.bigtable.mapping.MutationData; +import com.google.cloud.kafka.connect.bigtable.mapping.MutationDataBuilder; +import com.google.cloud.kafka.connect.bigtable.mapping.ValueMapper; +import com.google.cloud.kafka.connect.bigtable.version.PackageMetadata; +import com.google.common.annotations.VisibleForTesting; +import com.google.common.collect.Lists; +import com.google.protobuf.ByteString; +import java.util.ArrayList; +import java.util.Collection; +import java.util.HashMap; +import java.util.List; +import java.util.Map; +import java.util.Optional; +import java.util.concurrent.CompletableFuture; +import java.util.concurrent.ExecutionException; +import java.util.concurrent.Future; +import java.util.stream.Collectors; +import org.apache.kafka.connect.errors.ConnectException; +import org.apache.kafka.connect.errors.DataException; +import org.apache.kafka.connect.sink.ErrantRecordReporter; +import org.apache.kafka.connect.sink.SinkRecord; +import org.apache.kafka.connect.sink.SinkTask; +import org.apache.kafka.connect.sink.SinkTaskContext; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +/** + * {@link SinkTask} class used by {@link org.apache.kafka.connect.sink.SinkConnector} to write to + * Cloud Bigtable. + */ +public class BigtableSinkTask extends SinkTask { + private BigtableSinkTaskConfig config; + private BigtableDataClient bigtableData; + private BigtableTableAdminClient bigtableAdmin; + private KeyMapper keyMapper; + private ValueMapper valueMapper; + private BigtableSchemaManager schemaManager; + @VisibleForTesting protected final Map> batchers; + @VisibleForTesting protected Logger logger = LoggerFactory.getLogger(BigtableSinkTask.class); + + /** + * A default empty constructor. Initialization methods such as {@link BigtableSinkTask#start(Map)} + * or {@link SinkTask#initialize(SinkTaskContext)} must be called before {@link + * BigtableSinkTask#put(Collection)} can be called. Kafka Connect handles it well. + */ + public BigtableSinkTask() { + this(null, null, null, null, null, null, null); + } + + // A constructor only used by the tests. + @VisibleForTesting + protected BigtableSinkTask( + BigtableSinkTaskConfig config, + BigtableDataClient bigtableData, + BigtableTableAdminClient bigtableAdmin, + KeyMapper keyMapper, + ValueMapper valueMapper, + BigtableSchemaManager schemaManager, + SinkTaskContext context) { + this.config = config; + this.bigtableData = bigtableData; + this.bigtableAdmin = bigtableAdmin; + this.keyMapper = keyMapper; + this.valueMapper = valueMapper; + this.schemaManager = schemaManager; + this.context = context; + this.batchers = new HashMap<>(); + } + + @Override + public void start(Map props) { + config = new BigtableSinkTaskConfig(props); + logger = + LoggerFactory.getLogger( + BigtableSinkTask.class.getName() + + config.getInt(BigtableSinkTaskConfig.CONFIG_TASK_ID)); + bigtableData = config.getBigtableDataClient(); + bigtableAdmin = config.getBigtableAdminClient(); + keyMapper = + new KeyMapper( + config.getString(BigtableSinkTaskConfig.CONFIG_ROW_KEY_DELIMITER), + config.getList(BigtableSinkTaskConfig.CONFIG_ROW_KEY_DEFINITION)); + valueMapper = + new ValueMapper( + config.getString(BigtableSinkTaskConfig.CONFIG_DEFAULT_COLUMN_FAMILY), + config.getString(BigtableSinkTaskConfig.CONFIG_DEFAULT_COLUMN_QUALIFIER), + config.getNullValueMode()); + schemaManager = new BigtableSchemaManager(bigtableAdmin); + } + + @Override + public void stop() { + logger.trace("stop()"); + try { + Iterable> batcherCloses = + batchers.values().stream().map(Batcher::closeAsync).collect(Collectors.toList()); + ApiFutures.allAsList(batcherCloses).get(); + } catch (ExecutionException | InterruptedException e) { + logger.warn("Error closing Cloud Bigtable batchers.", e); + } finally { + batchers.clear(); + } + if (bigtableAdmin != null) { + try { + bigtableAdmin.close(); + } catch (RuntimeException e) { + logger.warn("Error closing Cloud Bigtable admin client.", e); + } + } + if (bigtableData != null) { + try { + bigtableData.close(); + } catch (RuntimeException e) { + logger.warn("Error closing Cloud Bigtable data client.", e); + } + } + } + + @Override + public String version() { + logger.trace("version()"); + return PackageMetadata.getVersion(); + } + + @Override + public void put(Collection records) { + logger.trace("put(#records={})", records.size()); + if (records.isEmpty()) { + return; + } + + Map mutations = prepareRecords(records); + if (config.getBoolean(BigtableSinkTaskConfig.CONFIG_AUTO_CREATE_TABLES)) { + mutations = autoCreateTablesAndHandleErrors(mutations); + } + if (config.getBoolean(BigtableSinkTaskConfig.CONFIG_AUTO_CREATE_COLUMN_FAMILIES)) { + mutations = autoCreateColumnFamiliesAndHandleErrors(mutations); + } + + Map> perRecordResults = new HashMap<>(); + switch (config.getInsertMode()) { + case INSERT: + insertRows(mutations, perRecordResults); + break; + case UPSERT: + upsertRows(mutations, perRecordResults); + break; + } + handleResults(perRecordResults); + } + + /** + * Generate mutations for input records. + * + * @param records Input records. + * @return {@link Map} containing input records and corresponding mutations that need to be + * applied. + */ + @VisibleForTesting + Map prepareRecords(Collection records) { + Map mutations = new HashMap<>(); + for (SinkRecord record : records) { + try { + Optional maybeRecordMutationData = createRecordMutationData(record); + if (maybeRecordMutationData.isPresent()) { + mutations.put(record, maybeRecordMutationData.get()); + } else { + logger.debug("Skipped a record that maps to an empty value."); + } + } catch (Throwable t) { + reportError(record, t); + } + } + return mutations; + } + + /** + * Generate mutation for a single input record. + * + * @param record Input record. + * @return {@link Optional#empty()} if the input record requires no write to Cloud Bigtable, + * {@link Optional} containing mutation that it needs to be written to Cloud Bigtable + * otherwise. + */ + @VisibleForTesting + Optional createRecordMutationData(SinkRecord record) { + String recordTableId = getTableName(record); + ByteString rowKey = ByteString.copyFrom(keyMapper.getKey(record.key())); + if (rowKey.isEmpty()) { + throw new DataException( + "The record's key converts into an illegal empty Cloud Bigtable row key."); + } + long timestamp = getTimestampMicros(record); + MutationDataBuilder mutationDataBuilder = + valueMapper.getRecordMutationDataBuilder(record.value(), timestamp); + return mutationDataBuilder.maybeBuild(recordTableId, rowKey); + } + + /** + * Get table name the input record's mutation will be written to. + * + * @param record Input record. + * @return Cloud Bigtable table name the input record's mutation will be written to. + */ + @VisibleForTesting + String getTableName(SinkRecord record) { + return config + .getString(BigtableSinkTaskConfig.CONFIG_TABLE_NAME_FORMAT) + .replace("${topic}", record.topic()); + } + + /** + * Get timestamp the input record's mutation's timestamp. + * + * @param record Input record. + * @return UNIX timestamp in microseconds. + */ + @VisibleForTesting + long getTimestampMicros(SinkRecord record) { + // From reading the Java Cloud Bigtable client, it looks that the only usable timestamp + // granularity is the millisecond one. So we assume it. + // There's a test that will break when it starts supporting microsecond granularity with a note + // to modify this function then. + Long timestampMillis = record.timestamp(); + if (timestampMillis == null) { + // The timestamp might be null if the kafka cluster is old (<= v0.9): + // https://github.com/apache/kafka/blob/f9615ed275c3856b73e5b6083049a8def9f59697/clients/src/main/java/org/apache/kafka/clients/consumer/ConsumerRecord.java#L49 + // In such a case, we default to wall clock time as per the design doc. + logger.debug("Used wall clock for a record missing timestamp."); + timestampMillis = System.currentTimeMillis(); + } + return 1000 * timestampMillis; + } + + /** + * Report error as described in {@link BigtableSinkConfig#getDefinition()}. + * + * @param record Input record whose processing caused an error. + * @param throwable The error. + */ + @VisibleForTesting + void reportError(SinkRecord record, Throwable throwable) { + ErrantRecordReporter reporter; + /// We get a reference to `reporter` using a procedure described in javadoc of + /// {@link SinkTaskContext#errantRecordReporter()} that guards against old Kafka versions. + try { + reporter = context.errantRecordReporter(); + } catch (NoSuchMethodError | NoClassDefFoundError ignored) { + reporter = null; + } + if (reporter != null) { + reporter.report(record, throwable); + logger.warn( + "Used DLQ for reporting a problem with a record (throwableClass={}).", + throwable.getClass().getSimpleName()); + } else { + BigtableErrorMode errorMode = config.getBigtableErrorMode(); + switch (errorMode) { + case IGNORE: + break; + case WARN: + logger.warn("Processing of a record with key {} failed", record.key(), throwable); + break; + case FAIL: + throw new BatchException(throwable); + } + } + } + + /** + * Attempts to create Cloud Bigtable tables so that all the mutations can be applied and handles + * errors. + * + * @param mutations Input records and corresponding mutations. + * @return Subset of the input argument containing only those record for which the target Cloud + * Bigtable tables exist. + */ + @VisibleForTesting + Map autoCreateTablesAndHandleErrors( + Map mutations) { + Map okMutations = new HashMap<>(mutations); + ResourceCreationResult resourceCreationResult = schemaManager.ensureTablesExist(okMutations); + String errorMessage = "Table auto-creation failed."; + for (SinkRecord record : resourceCreationResult.getBigtableErrors()) { + reportError(record, new ConnectException(errorMessage)); + okMutations.remove(record); + } + for (SinkRecord record : resourceCreationResult.getDataErrors()) { + reportError(record, new InvalidBigtableSchemaModificationException(errorMessage)); + okMutations.remove(record); + } + return okMutations; + } + + /** + * Attempts to create Cloud Bigtable column families so that all the mutations can be applied and + * handles errors. + * + * @param mutations Input records and corresponding mutations. + * @return Subset of the input argument containing only those record for which the target Cloud + * Bigtable column families exist. + */ + @VisibleForTesting + Map autoCreateColumnFamiliesAndHandleErrors( + Map mutations) { + Map okMutations = new HashMap<>(mutations); + ResourceCreationResult resourceCreationResult = + schemaManager.ensureColumnFamiliesExist(okMutations); + String errorMessage = "Column family auto-creation failed."; + for (SinkRecord record : resourceCreationResult.getBigtableErrors()) { + reportError(record, new ConnectException(errorMessage)); + okMutations.remove(record); + } + for (SinkRecord record : resourceCreationResult.getDataErrors()) { + reportError(record, new InvalidBigtableSchemaModificationException(errorMessage)); + okMutations.remove(record); + } + return okMutations; + } + + /** + * Applies the mutations using upserts. + * + * @param mutations Mutations to be applied. + * @param perRecordResults {@link Map} the per-record results will be written to. + */ + @VisibleForTesting + void upsertRows( + Map mutations, Map> perRecordResults) { + List> mutationsToApply = + new ArrayList<>(mutations.entrySet()); + int maxBatchSize = config.getInt(BigtableSinkTaskConfig.CONFIG_MAX_BATCH_SIZE); + List>> batches = + Lists.partition(mutationsToApply, maxBatchSize); + + try { + for (List> batch : batches) { + performUpsertBatch(batch, perRecordResults); + } + } finally { + for (Batcher b : batchers.values()) { + // We flush the batchers to ensure that no unsent requests remain in the batchers + // after this method returns to make the behavior more predictable. + // We flush asynchronously and await the results instead. + b.sendOutstanding(); + } + } + } + + /** + * Applies a single mutation batch using upserts. + * + * @param batch Batch of mutations to be applied. + * @param perRecordResults A {@link Map} the per-record results will be written to. + */ + @VisibleForTesting + void performUpsertBatch( + List> batch, + Map> perRecordResults) { + logger.trace("upsertBatch(#records={})", batch.size()); + for (Map.Entry recordEntry : batch) { + SinkRecord record = recordEntry.getKey(); + MutationData recordMutationData = recordEntry.getValue(); + String recordTableName = recordMutationData.getTargetTable(); + + Batcher batcher = + batchers.computeIfAbsent(recordTableName, (k) -> bigtableData.newBulkMutationBatcher(k)); + perRecordResults.put(record, batcher.add(recordMutationData.getUpsertMutation())); + } + for (Batcher batcher : batchers.values()) { + // We must flush the batchers to respect CONFIG_MAX_BATCH_SIZE. + // We flush asynchronously and await the results instead. + batcher.sendOutstanding(); + } + } + + /** + * Applies the mutations using inserts. + * + *

Note that no batching is used. + * + * @param mutations Mutations to be applied. + * @param perRecordResults {@link Map} the per-record results will be written to. + */ + @VisibleForTesting + void insertRows( + Map mutations, Map> perRecordResults) { + logger.trace("insertRows(#records={})", mutations.size()); + for (Map.Entry recordEntry : mutations.entrySet()) { + // We keep compatibility with Confluent's sink and disallow batching operations that check if + // the row already exists. + SinkRecord record = recordEntry.getKey(); + MutationData recordMutationData = recordEntry.getValue(); + ConditionalRowMutation insert = + // We want to perform the mutation if and only if the row does not already exist. + ConditionalRowMutation.create( + recordMutationData.getTargetTable(), recordMutationData.getRowKey()) + // We first check if any cell of this row exists... + .condition(Filters.FILTERS.pass()) + // ... and perform the mutation only if no cell exists. + .otherwise(recordMutationData.getInsertMutation()); + boolean insertSuccessful; + Optional exceptionThrown = Optional.empty(); + try { + insertSuccessful = !bigtableData.checkAndMutateRow(insert); + } catch (ApiException e) { + insertSuccessful = false; + exceptionThrown = Optional.of(e); + } + perRecordResults.put( + record, + insertSuccessful + ? CompletableFuture.completedFuture(null) + : CompletableFuture.failedFuture( + exceptionThrown.orElse( + new ConnectException("Insert failed since the row already existed.")))); + } + } + + /** + * Handles results of the whole operation. + * + * @param perRecordResults Results to be handled. + */ + @VisibleForTesting + void handleResults(Map> perRecordResults) { + logger.trace("handleResults(#records={})", perRecordResults.size()); + for (Map.Entry> recordResult : perRecordResults.entrySet()) { + try { + recordResult.getValue().get(); + } catch (ExecutionException | InterruptedException e) { + SinkRecord record = recordResult.getKey(); + reportError(record, e); + } + } + } +} diff --git a/google-cloud-bigtable-kafka-connect-sink/src/main/java/com/google/cloud/kafka/connect/bigtable/autocreate/BigtableSchemaManager.java b/google-cloud-bigtable-kafka-connect-sink/src/main/java/com/google/cloud/kafka/connect/bigtable/autocreate/BigtableSchemaManager.java new file mode 100644 index 0000000000..7db1f31104 --- /dev/null +++ b/google-cloud-bigtable-kafka-connect-sink/src/main/java/com/google/cloud/kafka/connect/bigtable/autocreate/BigtableSchemaManager.java @@ -0,0 +1,470 @@ +/* + * Copyright 2024 Google LLC + * + * Licensed under the Apache License, Version 2.0 (the "License"); + * you may not use this file except in compliance with the License. + * You may obtain a copy of the License at + * + * https://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ +package com.google.cloud.kafka.connect.bigtable.autocreate; + +import com.google.api.core.ApiFuture; +import com.google.api.gax.rpc.ApiException; +import com.google.api.gax.rpc.StatusCode; +import com.google.cloud.bigtable.admin.v2.BigtableTableAdminClient; +import com.google.cloud.bigtable.admin.v2.models.ColumnFamily; +import com.google.cloud.bigtable.admin.v2.models.CreateTableRequest; +import com.google.cloud.bigtable.admin.v2.models.ModifyColumnFamiliesRequest; +import com.google.cloud.bigtable.admin.v2.models.Table; +import com.google.cloud.kafka.connect.bigtable.exception.BatchException; +import com.google.cloud.kafka.connect.bigtable.mapping.MutationData; +import com.google.common.annotations.VisibleForTesting; +import java.util.AbstractMap; +import java.util.ArrayList; +import java.util.Collection; +import java.util.HashMap; +import java.util.HashSet; +import java.util.List; +import java.util.Map; +import java.util.Optional; +import java.util.Set; +import java.util.concurrent.ExecutionException; +import java.util.stream.Collectors; +import org.apache.kafka.connect.sink.SinkRecord; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +/** + * A class responsible for the creation of Cloud Bigtable {@link Table Table(s)} and {@link + * ColumnFamily ColumnFamily(s)} needed by the transformed Kafka Connect records. + * + *

This class contains nontrivial logic since we try to avoid API calls if possible. + * + *

This class does not automatically rediscover deleted resources. If another user of the Cloud + * Bigtable instance deletes a table or a column, the sink using an instance of this class to + * auto-create the resources, might end up sending requests targeting nonexistent {@link Table + * Table(s)} and/or {@link ColumnFamily ColumnFamily(s)}. + */ +public class BigtableSchemaManager { + @VisibleForTesting protected Logger logger = LoggerFactory.getLogger(BigtableSchemaManager.class); + + private final BigtableTableAdminClient bigtable; + + /** + * A {@link Map} storing the names of existing Cloud Bigtable tables as keys and existing column + * families within these tables as the values. + * + *

We have a single data structure for table and column family caches to ensure that they are + * consistent.
+ * An {@link Optional#empty()} value means that a table exists, but we don't know what column + * families it contains. + */ + @VisibleForTesting protected Map>> tableNameToColumnFamilies; + + /** + * The default constructor. + * + * @param bigtable The Cloud Bigtable admin client used to auto-create {@link Table Table(s)} and + * {@link ColumnFamily ColumnFamily(s)}. + */ + public BigtableSchemaManager(BigtableTableAdminClient bigtable) { + this.bigtable = bigtable; + tableNameToColumnFamilies = new HashMap<>(); + } + + /** + * Ensures that all the {@link Table Table(s)} needed by the input records exist by attempting to + * create the missing ones. + * + * @param recordsAndOutputs A {@link Map} containing {@link SinkRecord SinkRecord(s)} and their + * matching {@link MutationData} specifying which {@link Table Table(s)} need to exist. + * @return A {@link ResourceCreationResult} containing {@link SinkRecord SinkRecord(s)} for whose + * {@link MutationData} auto-creation of {@link Table Table(s)} failed. + */ + public ResourceCreationResult ensureTablesExist(Map recordsAndOutputs) { + Map> recordsByTableNames = getTableNamesToRecords(recordsAndOutputs); + + Map> recordsByMissingTableNames = + missingTablesToRecords(recordsByTableNames); + if (recordsByMissingTableNames.isEmpty()) { + return ResourceCreationResult.empty(); + } + logger.debug("Missing {} tables", recordsByMissingTableNames.size()); + Map, ResourceAndRecords> recordsByCreateTableFutures = + sendCreateTableRequests(recordsByMissingTableNames); + // No cache update here since we create tables with no column families, so every (non-delete) + // write to the table will need to create needed column families first, so saving the data from + // the response gives us no benefit. + // We ignore errors to handle races between multiple tasks of a single connector and refresh + // the cache in a further step. + Set dataErrors = + awaitResourceCreationAndHandleInvalidInputErrors( + recordsByCreateTableFutures, "Error creating a Cloud Bigtable table: %s"); + refreshTableNamesCache(); + Set bigtableErrors = + missingTablesToRecords(recordsByMissingTableNames).values().stream() + .flatMap(Collection::stream) + .collect(Collectors.toSet()); + bigtableErrors.removeAll(dataErrors); + return new ResourceCreationResult(bigtableErrors, dataErrors); + } + + /** + * Ensures that all the {@link ColumnFamily ColumnFamily(s)} needed by the input records exist by + * attempting to create the missing ones. + * + *

This method will not try to create missing {@link Table Table(s)} tables if some of the + * needed ones do not exist, but it will handle that case gracefully. + * + * @param recordsAndOutputs A {@link Map} containing {@link SinkRecord SinkRecord(s)} and their + * matching {@link MutationData} specifying which {@link ColumnFamily ColumnFamily(s)} need to + * exist. + * @return A {@link ResourceCreationResult} containing {@link SinkRecord SinkRecord(s)} for whose + * {@link MutationData} needed {@link Table Table(s)} are missing or auto-creation of {@link + * ColumnFamily ColumnFamily(s)} failed. + */ + public ResourceCreationResult ensureColumnFamiliesExist( + Map recordsAndOutputs) { + Map, List> recordsByColumnFamilies = + getTableColumnFamiliesToRecords(recordsAndOutputs); + + Map, List> recordsByMissingColumnFamilies = + missingTableColumnFamiliesToRecords(recordsByColumnFamilies); + if (recordsByMissingColumnFamilies.isEmpty()) { + return ResourceCreationResult.empty(); + } + logger.debug("Missing {} column families", recordsByMissingColumnFamilies.size()); + Map, ResourceAndRecords>> + recordsByCreateColumnFamilyFutures = + sendCreateColumnFamilyRequests(recordsByMissingColumnFamilies); + + // No cache update here since the requests are handled by Cloud Bigtable in a random order. + // We ignore errors to handle races between multiple tasks of a single connector + // and refresh the cache in a further step. + Set dataErrors = + awaitResourceCreationAndHandleInvalidInputErrors( + recordsByCreateColumnFamilyFutures, "Error creating a Cloud Bigtable column family %s"); + + Set tablesRequiringRefresh = + recordsByMissingColumnFamilies.keySet().stream() + .map(Map.Entry::getKey) + .collect(Collectors.toSet()); + refreshTableColumnFamiliesCache(tablesRequiringRefresh); + + Map, List> missing = + missingTableColumnFamiliesToRecords(recordsByMissingColumnFamilies); + Set bigtableErrors = + missing.values().stream().flatMap(Collection::stream).collect(Collectors.toSet()); + bigtableErrors.removeAll(dataErrors); + return new ResourceCreationResult(bigtableErrors, dataErrors); + } + + /** + * @param recordsAndOutputs A {@link Map} containing {@link SinkRecord SinkRecords} and + * corresponding Cloud Bigtable mutations. + * @return A {@link Map} containing Cloud Bigtable table names and {@link SinkRecord SinkRecords} + * that need these tables to exist. + */ + private static Map> getTableNamesToRecords( + Map recordsAndOutputs) { + Map> tableNamesToRecords = new HashMap<>(); + for (Map.Entry rowEntry : recordsAndOutputs.entrySet()) { + SinkRecord record = rowEntry.getKey(); + String tableName = rowEntry.getValue().getTargetTable(); + List records = + tableNamesToRecords.computeIfAbsent(tableName, k -> new ArrayList<>()); + records.add(record); + } + return tableNamesToRecords; + } + + /** + * @param recordsAndOutputs A {@link Map} containing {@link SinkRecord SinkRecords} and + * corresponding Cloud Bigtable mutations. + * @return A {@link Map} containing {@link Map.Entry Map.Entry(s)} consisting of Bigtable table + * names and column families and {@link SinkRecord SinkRecords} that need to use these tables + * and column families to exist. + */ + private static Map, List> getTableColumnFamiliesToRecords( + Map recordsAndOutputs) { + Map, List> tableColumnFamiliesToRecords = new HashMap<>(); + for (Map.Entry e : recordsAndOutputs.entrySet()) { + SinkRecord record = e.getKey(); + MutationData recordMutationData = e.getValue(); + String tableName = recordMutationData.getTargetTable(); + for (String columnFamily : recordMutationData.getRequiredColumnFamilies()) { + Map.Entry key = + new AbstractMap.SimpleImmutableEntry<>(tableName, columnFamily); + List records = + tableColumnFamiliesToRecords.computeIfAbsent(key, k -> new ArrayList<>()); + records.add(record); + } + } + return tableColumnFamiliesToRecords; + } + + /** + * Refreshes the existing table names in the cache. + * + *

Note that it deletes the entries from the cache if the tables disappear. + */ + @VisibleForTesting + void refreshTableNamesCache() { + Set tables; + try { + tables = new HashSet<>(bigtable.listTables()); + } catch (ApiException e) { + logger.error("listTables() exception", e); + // We don't allow listTables() to fail. It means something is seriously wrong, so we fail the + // whole batch. + throw new BatchException(e); + } + for (String key : new HashSet<>(tableNameToColumnFamilies.keySet())) { + if (!tables.contains(key)) { + tableNameToColumnFamilies.remove(key); + } + } + for (String table : tables) { + tableNameToColumnFamilies.putIfAbsent(table, Optional.empty()); + } + } + + /** + * Refreshes existing table names and a subset of existing column families in the cache. + * + *

Note that it deletes the entries from the cache if the tables disappeared and that it + * doesn't modify column family caches of tables that aren't provided as an argument. + * + * @param tablesRequiringRefresh A {@link Set} of table names whose column family caches will be + * refreshed. + */ + @VisibleForTesting + void refreshTableColumnFamiliesCache(Set tablesRequiringRefresh) { + refreshTableNamesCache(); + List>> tableFutures = + tableNameToColumnFamilies.keySet().stream() + .filter(tablesRequiringRefresh::contains) + .map(t -> new AbstractMap.SimpleImmutableEntry<>(t, bigtable.getTableAsync(t))) + .collect(Collectors.toList()); + Map>> newCache = new HashMap<>(tableNameToColumnFamilies); + for (Map.Entry> entry : tableFutures) { + String tableName = entry.getKey(); + try { + Table tableDetails = entry.getValue().get(); + Set tableColumnFamilies = + tableDetails.getColumnFamilies().stream() + .map(ColumnFamily::getId) + .collect(Collectors.toSet()); + newCache.put(tableName, Optional.of(tableColumnFamilies)); + } catch (ExecutionException | InterruptedException e) { + // We don't allow getTable() to fail. If it does, the entry is removed from the cache. This + // way its SinkRecord will be failed by ensureColumnFamiliesExist(). The alternative is to + // throw an exception and fail the whole batch that way. + logger.warn("getTable({}) exception", tableName, e); + newCache.remove(tableName); + } + } + // Note that we update the cache atomically to avoid partial errors. If an unexpected exception + // is thrown, the whole batch is failed. It's not ideal, but in line with the behavior of other + // connectors. + tableNameToColumnFamilies = newCache; + } + + /** + * @param tableNamesToRecords A {@link Map} containing Cloud Bigtable table names and {@link + * SinkRecord SinkRecords} that need these tables to exist. + * @return A subset of the input argument with the entries corresponding to existing tables + * removed. + */ + private Map> missingTablesToRecords( + Map> tableNamesToRecords) { + Map> recordsByMissingTableNames = new HashMap<>(tableNamesToRecords); + recordsByMissingTableNames.keySet().removeAll(tableNameToColumnFamilies.keySet()); + return recordsByMissingTableNames; + } + + /** + * @param tableColumnFamiliesToRecords A {@link Map} containing {@link Map.Entry} consisting of + * Bigtable table names and column families and {@link SinkRecord SinkRecords} that need to + * use these tables and column families to exist. + * @return A subset of the input argument with the entries corresponding to existing column + * families removed. + */ + private Map, List> missingTableColumnFamiliesToRecords( + Map, List> tableColumnFamiliesToRecords) { + Map, List> recordsByMissingColumnFamilies = + new HashMap<>(tableColumnFamiliesToRecords); + for (Map.Entry>> existingEntry : + tableNameToColumnFamilies.entrySet()) { + String tableName = existingEntry.getKey(); + for (String columnFamily : existingEntry.getValue().orElse(new HashSet<>())) { + recordsByMissingColumnFamilies.remove( + new AbstractMap.SimpleImmutableEntry<>(tableName, columnFamily)); + } + } + return recordsByMissingColumnFamilies; + } + + private Map, ResourceAndRecords> sendCreateTableRequests( + Map> recordsByMissingTables) { + Map, ResourceAndRecords> result = new HashMap<>(); + for (Map.Entry> e : recordsByMissingTables.entrySet()) { + ResourceAndRecords resourceAndRecords = + new ResourceAndRecords<>(e.getKey(), e.getValue()); + result.put(createTable(e.getKey()), resourceAndRecords); + } + return result; + } + + private Map, ResourceAndRecords>> + sendCreateColumnFamilyRequests( + Map, List> recordsByMissingColumnFamilies) { + Map, ResourceAndRecords>> result = new HashMap<>(); + for (Map.Entry, List> e : + recordsByMissingColumnFamilies.entrySet()) { + ResourceAndRecords> resourceAndRecords = + new ResourceAndRecords<>(e.getKey(), e.getValue()); + result.put(createColumnFamily(e.getKey()), resourceAndRecords); + } + return result; + } + + private ApiFuture createTable(String tableName) { + logger.info("Creating table '{}'", tableName); + CreateTableRequest createTableRequest = CreateTableRequest.of(tableName); + return bigtable.createTableAsync(createTableRequest); + } + + // We only issue one request at a time because each multi-column-family operation on a single + // Table is atomic and fails if any of the Column Families to be created already exists. + // Thus by sending multiple requests, we simplify error handling when races between multiple + // tasks of a single connector happen. + private ApiFuture
createColumnFamily(Map.Entry tableNameAndColumnFamily) { + String tableName = tableNameAndColumnFamily.getKey(); + String columnFamily = tableNameAndColumnFamily.getValue(); + logger.info("Creating column family '{}' in table '{}'", columnFamily, tableName); + ModifyColumnFamiliesRequest request = + ModifyColumnFamiliesRequest.of(tableName).addFamily(columnFamily); + return bigtable.modifyFamiliesAsync(request); + } + + /** + * Awaits resource auto-creation result futures and handles the errors. + * + *

The errors might be handled in two ways: + * + *

    + *
  • If a resource's creation failed with an exception signifying that the request was + * invalid, it is assumed that input {@link SinkRecord SinkRecord(s)} map to invalid values, + * so all the {@link SinkRecord SinkRecord(s)} needing the resource whose creation failed + * are returned. + *
  • Other resource creation errors are logged. + *
+ * + * @param createdColumnFamilyFuturesAndRecords {@link Map} of {@link ApiFuture ApiFuture(s)} and + * information what resource is created and for which {@link SinkRecord SinkRecord(s)}. + * @param errorMessageTemplate The Java format string template of error message with which Cloud + * Bigtable exceptions for valid input data are logged. + * @return A {@link Set} of {@link SinkRecord SinkRecord(s)} for which auto resource creation + * failed due to their invalid data. + * @param {@link ApiFuture} containing result of the resource creation operation. + * @param The resources' type identifier. + */ + @VisibleForTesting + , Id> Set awaitResourceCreationAndHandleInvalidInputErrors( + Map> createdColumnFamilyFuturesAndRecords, + String errorMessageTemplate) { + Set dataErrors = new HashSet<>(); + createdColumnFamilyFuturesAndRecords.forEach( + (fut, resourceAndRecords) -> { + Object resource = resourceAndRecords.getResource(); + List sinkRecords = resourceAndRecords.getRecords(); + try { + fut.get(); + } catch (ExecutionException | InterruptedException e) { + String errorMessage = String.format(errorMessageTemplate, resource.toString()); + if (SchemaApiExceptions.isCausedByInputError(e)) { + dataErrors.addAll(sinkRecords); + } else { + logger.info(errorMessage, e); + } + } + }); + return dataErrors; + } + + /** + * A record class connecting an auto-created resource and {@link SinkRecord SinkRecord(s)} + * requiring it to exist. + * + * @param The resources' type identifier. + */ + @VisibleForTesting + static class ResourceAndRecords { + private final Id resource; + private final List records; + + public ResourceAndRecords(Id resource, List records) { + this.resource = resource; + this.records = records; + } + + public Id getResource() { + return resource; + } + + public List getRecords() { + return records; + } + } + + /** + * A helper class containing logic for grouping {@link ApiException ApiException(s)} encountered + * when modifying Cloud Bigtable schema. + */ + @VisibleForTesting + static class SchemaApiExceptions { + /** + * @param t Exception thrown by some function using Cloud Bigtable API. + * @return true if input exception was caused by invalid Cloud Bigtable request, false + * otherwise. + */ + @VisibleForTesting + static boolean isCausedByInputError(Throwable t) { + return maybeExtractBigtableStatusCode(t) + .map(sc -> isStatusCodeCausedByInputError(sc.getCode())) + .orElse(false); + } + + @VisibleForTesting + static Optional maybeExtractBigtableStatusCode(Throwable t) { + while (t != null) { + if (t instanceof ApiException) { + ApiException apiException = (ApiException) t; + return Optional.of(apiException.getStatusCode()); + } + t = t.getCause(); + } + return Optional.empty(); + } + + @VisibleForTesting + static boolean isStatusCodeCausedByInputError(StatusCode.Code code) { + switch (code) { + case INVALID_ARGUMENT: + case FAILED_PRECONDITION: + case OUT_OF_RANGE: + return true; + default: + return false; + } + } + } +} diff --git a/google-cloud-bigtable-kafka-connect-sink/src/main/java/com/google/cloud/kafka/connect/bigtable/autocreate/ResourceCreationResult.java b/google-cloud-bigtable-kafka-connect-sink/src/main/java/com/google/cloud/kafka/connect/bigtable/autocreate/ResourceCreationResult.java new file mode 100644 index 0000000000..f4de7efcaa --- /dev/null +++ b/google-cloud-bigtable-kafka-connect-sink/src/main/java/com/google/cloud/kafka/connect/bigtable/autocreate/ResourceCreationResult.java @@ -0,0 +1,51 @@ +/* + * Copyright 2025 Google LLC + * + * Licensed under the Apache License, Version 2.0 (the "License"); + * you may not use this file except in compliance with the License. + * You may obtain a copy of the License at + * + * https://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ +package com.google.cloud.kafka.connect.bigtable.autocreate; + +import java.util.HashSet; +import java.util.Set; +import org.apache.kafka.connect.sink.SinkRecord; + +/** A record class storing the output of {@link BigtableSchemaManager} operations. */ +public class ResourceCreationResult { + private final Set bigtableErrors; + private final Set dataErrors; + + public static ResourceCreationResult empty() { + return new ResourceCreationResult(new HashSet<>(), new HashSet<>()); + } + + public ResourceCreationResult(Set bigtableErrors, Set dataErrors) { + this.bigtableErrors = bigtableErrors; + this.dataErrors = dataErrors; + } + + /** + * @return A {@link Set} of {@link SinkRecord SinkRecord(s)} for which resource auto-creation + * failed due to some problems on Cloud Bigtable part. + */ + public Set getBigtableErrors() { + return bigtableErrors; + } + + /** + * @return A {@link Set} of {@link SinkRecord SinkRecord(s)} for which resource auto-creation + * failed due to invalid input data. These records should not ever be retried. + */ + public Set getDataErrors() { + return dataErrors; + } +} diff --git a/google-cloud-bigtable-kafka-connect-sink/src/main/java/com/google/cloud/kafka/connect/bigtable/config/BigtableErrorMode.java b/google-cloud-bigtable-kafka-connect-sink/src/main/java/com/google/cloud/kafka/connect/bigtable/config/BigtableErrorMode.java new file mode 100644 index 0000000000..f8447c854a --- /dev/null +++ b/google-cloud-bigtable-kafka-connect-sink/src/main/java/com/google/cloud/kafka/connect/bigtable/config/BigtableErrorMode.java @@ -0,0 +1,22 @@ +/* + * Copyright 2024 Google LLC + * + * Licensed under the Apache License, Version 2.0 (the "License"); + * you may not use this file except in compliance with the License. + * You may obtain a copy of the License at + * + * https://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ +package com.google.cloud.kafka.connect.bigtable.config; + +public enum BigtableErrorMode { + FAIL, + WARN, + IGNORE, +} diff --git a/google-cloud-bigtable-kafka-connect-sink/src/main/java/com/google/cloud/kafka/connect/bigtable/config/BigtableSinkConfig.java b/google-cloud-bigtable-kafka-connect-sink/src/main/java/com/google/cloud/kafka/connect/bigtable/config/BigtableSinkConfig.java new file mode 100644 index 0000000000..0fe27aff4a --- /dev/null +++ b/google-cloud-bigtable-kafka-connect-sink/src/main/java/com/google/cloud/kafka/connect/bigtable/config/BigtableSinkConfig.java @@ -0,0 +1,548 @@ +/* + * Copyright 2024 Google LLC + * + * Licensed under the Apache License, Version 2.0 (the "License"); + * you may not use this file except in compliance with the License. + * You may obtain a copy of the License at + * + * https://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ +package com.google.cloud.kafka.connect.bigtable.config; + +import com.google.api.gax.core.CredentialsProvider; +import com.google.api.gax.core.FixedCredentialsProvider; +import com.google.api.gax.retrying.RetrySettings; +import com.google.auth.oauth2.GoogleCredentials; +import com.google.cloud.bigtable.admin.v2.BigtableTableAdminClient; +import com.google.cloud.bigtable.admin.v2.BigtableTableAdminSettings; +import com.google.cloud.bigtable.admin.v2.stub.BigtableTableAdminStubSettings; +import com.google.cloud.bigtable.data.v2.BigtableDataClient; +import com.google.cloud.bigtable.data.v2.BigtableDataSettings; +import com.google.cloud.bigtable.data.v2.stub.EnhancedBigtableStubSettings; +import com.google.common.annotations.VisibleForTesting; +import java.io.ByteArrayInputStream; +import java.io.FileInputStream; +import java.io.IOException; +import java.nio.charset.StandardCharsets; +import java.util.ArrayList; +import java.util.Arrays; +import java.util.Collections; +import java.util.List; +import java.util.Map; +import java.util.Optional; +import java.util.function.Function; +import org.apache.kafka.common.config.AbstractConfig; +import org.apache.kafka.common.config.Config; +import org.apache.kafka.common.config.ConfigDef; +import org.apache.kafka.common.config.ConfigException; +import org.apache.kafka.common.config.ConfigValue; +import org.apache.kafka.common.utils.Utils; +import org.apache.kafka.connect.errors.RetriableException; +import org.threeten.bp.Duration; +import org.threeten.bp.temporal.ChronoUnit; + +/** + * A class defining the configuration of {@link + * com.google.cloud.kafka.connect.bigtable.BigtableSinkConnector}. + * + *

It's responsible for the validation and parsing of the user-provided values. + */ +public class BigtableSinkConfig extends AbstractConfig { + public static final String CONFIG_GCP_PROJECT_ID = "gcp.bigtable.project.id"; + public static final String CONFIG_GCP_CREDENTIALS_PATH = "gcp.bigtable.credentials.path"; + public static final String CONFIG_GCP_CREDENTIALS_JSON = "gcp.bigtable.credentials.json"; + public static final String CONFIG_BIGTABLE_INSTANCE_ID = "gcp.bigtable.instance.id"; + public static final String CONFIG_BIGTABLE_APP_PROFILE_ID = "gcp.bigtable.app.profile.id"; + public static final String CONFIG_INSERT_MODE = "insert.mode"; + public static final String CONFIG_MAX_BATCH_SIZE = "max.batch.size"; + public static final String CONFIG_VALUE_NULL_MODE = "value.null.mode"; + public static final String CONFIG_ERROR_MODE = "error.mode"; + public static final String CONFIG_TABLE_NAME_FORMAT = "table.name.format"; + public static final String CONFIG_ROW_KEY_DEFINITION = "row.key.definition"; + public static final String CONFIG_ROW_KEY_DELIMITER = "row.key.delimiter"; + public static final String CONFIG_AUTO_CREATE_TABLES = "auto.create.tables"; + public static final String CONFIG_AUTO_CREATE_COLUMN_FAMILIES = "auto.create.column.families"; + public static final String CONFIG_DEFAULT_COLUMN_FAMILY = "default.column.family"; + public static final String CONFIG_DEFAULT_COLUMN_QUALIFIER = "default.column.qualifier"; + public static final String CONFIG_RETRY_TIMEOUT_MILLIS = "retry.timeout.ms"; + private static final InsertMode DEFAULT_INSERT_MODE = InsertMode.INSERT; + private static final NullValueMode DEFAULT_NULL_VALUE_MODE = NullValueMode.WRITE; + private static final BigtableErrorMode DEFAULT_ERROR_MODE = BigtableErrorMode.FAIL; + private static final Integer DEFAULT_MAX_BATCH_SIZE = 1; + private static final List BIGTABLE_CONFIGURATION_PROPERTIES = + List.of( + CONFIG_GCP_CREDENTIALS_JSON, + CONFIG_GCP_CREDENTIALS_PATH, + CONFIG_GCP_PROJECT_ID, + CONFIG_BIGTABLE_INSTANCE_ID, + CONFIG_BIGTABLE_APP_PROFILE_ID); + private static final int BIGTABLE_CREDENTIALS_CHECK_TIMEOUT_SECONDS = 2; + + protected BigtableSinkConfig(ConfigDef definition, Map properties) { + super(definition, properties); + } + + /** + * The main constructor. + * + * @param properties The properties provided by the user. + */ + public BigtableSinkConfig(Map properties) { + this(getDefinition(), properties); + } + + /** + * Validates that a valid {@link BigtableSinkConfig} can be created using the input properties. + * + * @param props The properties provided by the user. + * @return {@link Config} containing validation results. + */ + public static Config validate(Map props) { + return validate(props, true); + } + + /** + * Validates that a valid {@link BigtableSinkConfig} can be created using the input properties. + * + * @param props The properties provided by the user. + * @param accessBigtableToValidateConfiguration If set to true, validation includes checking + * whether the Cloud Bigtable configuration is valid by connecting to Cloud Bigtable and + * attempting to execute a simple read-only operation. + * @return {@link Config} containing validation results. + */ + @VisibleForTesting + static Config validate(Map props, boolean accessBigtableToValidateConfiguration) { + // Note that we only need to verify the properties we define, the generic Sink configuration is + // handled in SinkConnectorConfig::validate(). + String credentialsPath = props.get(CONFIG_GCP_CREDENTIALS_PATH); + String credentialsJson = props.get(CONFIG_GCP_CREDENTIALS_JSON); + String insertMode = props.get(CONFIG_INSERT_MODE); + String maxBatchSize = props.get(CONFIG_MAX_BATCH_SIZE); + String effectiveInsertMode = + Optional.ofNullable(insertMode).orElse(DEFAULT_INSERT_MODE.name()).toUpperCase(); + String effectiveMaxBatchSize = + Optional.ofNullable(maxBatchSize).orElse(DEFAULT_MAX_BATCH_SIZE.toString()).trim(); + + Map validationResult = getDefinition().validateAll(props); + if (!Utils.isBlank(credentialsPath) && !Utils.isBlank(credentialsJson)) { + String errorMessage = + CONFIG_GCP_CREDENTIALS_JSON + + " and " + + CONFIG_GCP_CREDENTIALS_PATH + + " are mutually exclusive options, but both are set."; + addErrorMessage(validationResult, CONFIG_GCP_CREDENTIALS_JSON, credentialsJson, errorMessage); + addErrorMessage(validationResult, CONFIG_GCP_CREDENTIALS_PATH, credentialsPath, errorMessage); + } + if (effectiveInsertMode.equals(InsertMode.INSERT.name()) + && !effectiveMaxBatchSize.equals("1")) { + String errorMessage = + "When using `" + + CONFIG_INSERT_MODE + + "` of `insert`, " + + CONFIG_MAX_BATCH_SIZE + + " must be set to `1`."; + addErrorMessage(validationResult, CONFIG_INSERT_MODE, insertMode, errorMessage); + addErrorMessage(validationResult, CONFIG_MAX_BATCH_SIZE, maxBatchSize, errorMessage); + } + + if (accessBigtableToValidateConfiguration + && validationResult.values().stream().allMatch(v -> v.errorMessages().isEmpty())) { + // We validate the user's credentials in order to warn them early rather than fill DLQ + // with records whose processing would fail due to invalid credentials. + // We only call it after validating that all other parameters are fine since creating + // a Cloud Bigtable client uses many of these parameters, and we don't want to warn + // the user unnecessarily. + BigtableSinkConfig config = new BigtableSinkConfig(props); + if (!config.isBigtableConfigurationValid()) { + String errorMessage = "Cloud Bigtable configuration is invalid."; + for (String bigtableProp : BIGTABLE_CONFIGURATION_PROPERTIES) { + addErrorMessage(validationResult, bigtableProp, props.get(bigtableProp), errorMessage); + } + } + } + return new Config(new ArrayList<>(validationResult.values())); + } + + /** + * @return {@link ConfigDef} used by Kafka Connect to advertise configuration options to the user + * and by us to perform basic validation of the user-provided values. + */ + public static ConfigDef getDefinition() { + return new ConfigDef() + .define( + CONFIG_GCP_PROJECT_ID, + ConfigDef.Type.STRING, + ConfigDef.NO_DEFAULT_VALUE, + ConfigDef.CompositeValidator.of( + new ConfigDef.NonNullValidator(), new ConfigDef.NonEmptyString()), + ConfigDef.Importance.HIGH, + "The ID of the GCP project.") + .define( + CONFIG_BIGTABLE_INSTANCE_ID, + ConfigDef.Type.STRING, + ConfigDef.NO_DEFAULT_VALUE, + ConfigDef.CompositeValidator.of( + new ConfigDef.NonNullValidator(), new ConfigDef.NonEmptyString()), + ConfigDef.Importance.HIGH, + "The ID of the Cloud Bigtable instance.") + .define( + CONFIG_BIGTABLE_APP_PROFILE_ID, + ConfigDef.Type.STRING, + null, + ConfigDef.Importance.MEDIUM, + "The application profile that the connector should use. If none is supplied," + + " the default app profile will be used.") + .define( + CONFIG_GCP_CREDENTIALS_PATH, + ConfigDef.Type.STRING, + null, + ConfigDef.Importance.HIGH, + "The path to the JSON service key file. Configure at most one of `" + + CONFIG_GCP_CREDENTIALS_PATH + + "` and `" + + CONFIG_GCP_CREDENTIALS_JSON + + "`. If neither is provided, Application Default Credentials will be used.") + .define( + CONFIG_GCP_CREDENTIALS_JSON, + ConfigDef.Type.STRING, + null, + ConfigDef.Importance.HIGH, + "The path to the JSON service key file. Configure at most one of `" + + CONFIG_GCP_CREDENTIALS_PATH + + "` and `" + + CONFIG_GCP_CREDENTIALS_JSON + + "`. If neither is provided, Application Default Credentials will be used.") + .define( + CONFIG_INSERT_MODE, + ConfigDef.Type.STRING, + DEFAULT_INSERT_MODE.name(), + enumValidator(InsertMode.values()), + ConfigDef.Importance.HIGH, + "Defines the insertion mode to use. Supported modes are:" + + "\n- insert - Insert new record only." + + " If the row to be written already exists in the table, an error is thrown." + + "\n- upsert - If the row to be written already exists," + + " then its column values are overwritten with the ones provided.") + .define( + CONFIG_MAX_BATCH_SIZE, + ConfigDef.Type.INT, + DEFAULT_MAX_BATCH_SIZE, + ConfigDef.Range.atLeast(1), + ConfigDef.Importance.MEDIUM, + "The maximum number of records that can be batched into a batch of upserts." + + " Note that since only a batch size of 1 for inserts is supported, `" + + CONFIG_MAX_BATCH_SIZE + + "` must be exactly `1` when `" + + CONFIG_INSERT_MODE + + "` is set to `INSERT`.") + .define( + CONFIG_VALUE_NULL_MODE, + ConfigDef.Type.STRING, + DEFAULT_NULL_VALUE_MODE.name(), + enumValidator(NullValueMode.values()), + ConfigDef.Importance.MEDIUM, + "Defines what to do with `null` Kafka values. Supported modes are:" + + "\n- write - Serialize `null`s to empty byte arrays." + + "\n- ignore - Ignore `null`s." + + "\n- delete - Use them to issue DELETE commands. Root-level `null` deletes a" + + " row. `null` nested one level deletes a column family named after the" + + " `null`-valued field. `null` nested two levels deletes a column named after the" + + " `null`-valued field in column family named after the `null-valued` field parent" + + " field. `null` values nested more than two levels are serialized like other" + + " values and don't result in any DELETE commands.") + .define( + CONFIG_ERROR_MODE, + ConfigDef.Type.STRING, + DEFAULT_ERROR_MODE.name(), + enumValidator(BigtableErrorMode.values()), + ConfigDef.Importance.MEDIUM, + "Specifies how to handle errors that result from writes, after retries. It is ignored" + + " if DLQ is configured. Supported modes are:" + + "\n- fail - The connector fails and must be manually restarted." + + "\n- warn - The connector logs a warning and continues operating normally." + + "\n- ignore - The connector does not log a warning but continues operating" + + " normally.") + .define( + CONFIG_TABLE_NAME_FORMAT, + ConfigDef.Type.STRING, + "${topic}", + ConfigDef.CompositeValidator.of( + new ConfigDef.NonNullValidator(), new ConfigDef.NonEmptyString()), + ConfigDef.Importance.MEDIUM, + "Name of the destination table. Use `${topic}` within the table name to specify" + + " the originating topic name.\nFor example, `user_${topic}` for the topic `stats`" + + " will map to the table name `user_stats`.") + .define( + CONFIG_ROW_KEY_DEFINITION, + ConfigDef.Type.LIST, + "", + ConfigDef.Importance.MEDIUM, + "A comma separated list of Kafka Record key field names that specifies the order of" + + " Kafka key fields to be concatenated to form the row key." + + "\nFor example the list: `username, post_id, time_stamp` when applied to a Kafka" + + " key: `{'username': 'bob','post_id': '213', 'time_stamp': '123123'}` and with" + + " delimiter `#` gives the row key `bob#213#123123`. You can also access terms" + + " nested in the key by using `.` as a delimiter. If this configuration is empty" + + " or unspecified and the Kafka Message Key is a" + + "\n- struct, all the fields in the struct are used to construct the row key." + + "\n- byte array, the row key is set to the byte array as is." + + "\n- primitive, the row key is set to the primitive stringified." + + "If prefixes, more complicated delimiters, and string constants are required in" + + " your Row Key, consider configuring an SMT to add relevant fields to the Kafka" + + " Record key.") + .define( + CONFIG_ROW_KEY_DELIMITER, + ConfigDef.Type.STRING, + "", + ConfigDef.Importance.LOW, + "The delimiter used in concatenating Kafka key fields in the row key. If this" + + " configuration is empty or unspecified, the key fields will be concatenated" + + " together directly.") + .define( + CONFIG_AUTO_CREATE_TABLES, + ConfigDef.Type.BOOLEAN, + false, + new ConfigDef.NonNullValidator(), + ConfigDef.Importance.MEDIUM, + "Whether to automatically create the destination table if it is found to be missing." + + "\nWhen enabled, the records for which the auto-creation fails, are failed." + + "\nRecreation of tables deleted by other Cloud Bigtable users is not supported.") + .define( + CONFIG_AUTO_CREATE_COLUMN_FAMILIES, + ConfigDef.Type.BOOLEAN, + false, + new ConfigDef.NonNullValidator(), + ConfigDef.Importance.MEDIUM, + "Whether to automatically create missing columns families in the table relative to the" + + " record schema." + + "\nDoes not imply auto-creation of tables." + + "\nWhen enabled, the records for which the auto-creation fails, are failed." + + "\nRecreation of column families deleted by other Cloud Bigtable users is not" + + " supported.") + .define( + CONFIG_DEFAULT_COLUMN_FAMILY, + ConfigDef.Type.STRING, + "default", + ConfigDef.Importance.MEDIUM, + "Any root-level fields on the SinkRecord that aren't objects will be added to this" + + " column family. If empty, the fields will be ignored.") + .define( + CONFIG_DEFAULT_COLUMN_QUALIFIER, + ConfigDef.Type.STRING, + "KAFKA_VALUE", + ConfigDef.Importance.MEDIUM, + "Any root-level values on the SinkRecord that aren't objects will be added to this" + + " column within default column family. If empty, the value will be ignored.") + .define( + CONFIG_RETRY_TIMEOUT_MILLIS, + ConfigDef.Type.LONG, + 90000, + ConfigDef.Range.atLeast(0), + ConfigDef.Importance.MEDIUM, + "Maximum time in milliseconds allocated for retrying database operations before trying" + + " other error handling mechanisms."); + } + + /** + * Adds a validation error in the format expected by {@link BigtableSinkConfig#validate(Map)}. + * + * @param validatedConfig Input/output parameter containing current validation result. + * @param name Configuration parameter name. + * @param value Configuration parameter value. + * @param errorMessage Error message to be added. + */ + private static void addErrorMessage( + Map validatedConfig, String name, String value, String errorMessage) { + validatedConfig + .computeIfAbsent( + name, p -> new ConfigValue(name, value, Collections.emptyList(), new ArrayList<>())) + .addErrorMessage(errorMessage); + } + + public NullValueMode getNullValueMode() { + return getEnum(CONFIG_VALUE_NULL_MODE, NullValueMode::valueOf); + } + + public BigtableErrorMode getBigtableErrorMode() { + return getEnum(CONFIG_ERROR_MODE, BigtableErrorMode::valueOf); + } + + public InsertMode getInsertMode() { + return getEnum(CONFIG_INSERT_MODE, InsertMode::valueOf); + } + + /** + * @return {@link BigtableTableAdminClient} connected to a Cloud Bigtable instance configured as + * described in {@link BigtableSinkConfig#getDefinition()}. + */ + public BigtableTableAdminClient getBigtableAdminClient() { + RetrySettings retrySettings = getRetrySettings(); + return getBigtableAdminClient(retrySettings); + } + + @VisibleForTesting + BigtableTableAdminClient getBigtableAdminClient(RetrySettings retrySettings) { + Optional credentialsProvider = + getUserConfiguredBigtableCredentialsProvider(); + + BigtableTableAdminSettings.Builder adminSettingsBuilder = + BigtableTableAdminSettings.newBuilder() + .setProjectId(getString(BigtableSinkTaskConfig.CONFIG_GCP_PROJECT_ID)) + .setInstanceId(getString(BigtableSinkTaskConfig.CONFIG_BIGTABLE_INSTANCE_ID)); + if (credentialsProvider.isPresent()) { + adminSettingsBuilder.setCredentialsProvider(credentialsProvider.get()); + } else { + // Use the default credential provider that utilizes Application Default Credentials. + } + + BigtableTableAdminStubSettings.Builder adminStubSettings = adminSettingsBuilder.stubSettings(); + adminStubSettings.createTableSettings().setRetrySettings(retrySettings); + adminStubSettings.modifyColumnFamiliesSettings().setRetrySettings(retrySettings); + adminStubSettings.listTablesSettings().setRetrySettings(retrySettings); + adminStubSettings.getTableSettings().setRetrySettings(retrySettings); + try { + return BigtableTableAdminClient.create(adminSettingsBuilder.build()); + } catch (IOException e) { + throw new RetriableException(e); + } + } + + /** + * @return {@link BigtableDataClient} connected to Cloud Bigtable instance configured as described + * in {@link BigtableSinkConfig#getDefinition()}. + */ + public BigtableDataClient getBigtableDataClient() { + RetrySettings retrySettings = getRetrySettings(); + Optional credentialsProvider = + getUserConfiguredBigtableCredentialsProvider(); + + BigtableDataSettings.Builder dataSettingsBuilder = + BigtableDataSettings.newBuilder() + .setProjectId(getString(BigtableSinkTaskConfig.CONFIG_GCP_PROJECT_ID)) + .setInstanceId(getString(BigtableSinkTaskConfig.CONFIG_BIGTABLE_INSTANCE_ID)); + if (credentialsProvider.isPresent()) { + dataSettingsBuilder.setCredentialsProvider(credentialsProvider.get()); + } else { + // Use the default credential provider that utilizes Application Default Credentials. + } + String appProfileId = getString(BigtableSinkTaskConfig.CONFIG_BIGTABLE_APP_PROFILE_ID); + if (appProfileId == null) { + dataSettingsBuilder.setDefaultAppProfileId(); + } else { + dataSettingsBuilder.setAppProfileId(appProfileId); + } + + EnhancedBigtableStubSettings.Builder dataStubSettings = dataSettingsBuilder.stubSettings(); + dataStubSettings.mutateRowSettings().setRetrySettings(retrySettings); + dataStubSettings.bulkMutateRowsSettings().setRetrySettings(retrySettings); + dataStubSettings.readRowSettings().setRetrySettings(retrySettings); + dataStubSettings.readRowsSettings().setRetrySettings(retrySettings); + + try { + return BigtableDataClient.create(dataSettingsBuilder.build()); + } catch (IOException e) { + throw new RetriableException(e); + } + } + + /** + * Checks whether Cloud Bigtable configuration is valid by connecting to Cloud Bigtable and + * attempting to execute a simple read-only operation. + * + * @return true if Cloud Bigtable configuration is valid, false otherwise. + */ + @VisibleForTesting + boolean isBigtableConfigurationValid() { + BigtableTableAdminClient bigtable = null; + try { + RetrySettings retrySettings = + RetrySettings.newBuilder() + .setMaxAttempts(0) + .setTotalTimeout( + Duration.of(BIGTABLE_CREDENTIALS_CHECK_TIMEOUT_SECONDS, ChronoUnit.SECONDS)) + .build(); + bigtable = getBigtableAdminClient(retrySettings); + bigtable.listTables(); + return true; + } catch (Throwable t) { + return false; + } finally { + if (bigtable != null) { + bigtable.close(); + } + } + } + + /** + * @return {@link RetrySettings} of Cloud Bigtable clients configured as described in {@link + * BigtableSinkConfig#getDefinition()}. + */ + protected RetrySettings getRetrySettings() { + return RetrySettings.newBuilder() + .setTotalTimeout( + Duration.of( + getLong(BigtableSinkTaskConfig.CONFIG_RETRY_TIMEOUT_MILLIS), ChronoUnit.MILLIS)) + .build(); + } + + /** + * Extracts typed enum value from this object. + * + * @param configName Enum parameter name in {@link BigtableSinkConfig}. + * @param converter Function that parses parameter value into an enum value. It's assumed to throw + * only {@link NullPointerException} and {@link IllegalArgumentException}. + * @return Parsed enum value. + * @param Enum type. + */ + private T getEnum(String configName, Function converter) { + String s = this.getString(configName); + try { + return converter.apply(s.toUpperCase()); + } catch (NullPointerException | IllegalArgumentException e) { + throw new ConfigException(configName, s); + } + } + + private static ConfigDef.Validator enumValidator(Enum[] enumValues) { + return ConfigDef.CaseInsensitiveValidString.in( + Arrays.stream(enumValues).map(Enum::name).toArray(String[]::new)); + } + + /** + * @return {@link Optional#empty()} if the user didn't configure the Cloud Bigtable credentials, + * {@link Optional} containing {@link CredentialsProvider} configured as described in {@link + * BigtableSinkConfig#getDefinition()} otherwise. + */ + protected Optional getUserConfiguredBigtableCredentialsProvider() { + String credentialsJson = getString(BigtableSinkTaskConfig.CONFIG_GCP_CREDENTIALS_JSON); + String credentialsPath = getString(BigtableSinkTaskConfig.CONFIG_GCP_CREDENTIALS_PATH); + byte[] credentials; + if (!Utils.isBlank(credentialsJson)) { + credentials = credentialsJson.getBytes(StandardCharsets.UTF_8); + } else if (!Utils.isBlank(credentialsPath)) { + try (FileInputStream is = new FileInputStream(credentialsPath)) { + credentials = is.readAllBytes(); + } catch (IOException e) { + throw new ConfigException( + String.format("Error getting credentials from file: %s.", credentialsPath)); + } + } else { + // We will use the default CredentialsProvider, which doesn't need any application-level + // configuration. + return Optional.empty(); + } + try { + return Optional.of( + FixedCredentialsProvider.create( + GoogleCredentials.fromStream(new ByteArrayInputStream(credentials)))); + } catch (IOException e) { + throw new ConfigException("Cloud Bigtable credentials creation failed."); + } + } +} diff --git a/google-cloud-bigtable-kafka-connect-sink/src/main/java/com/google/cloud/kafka/connect/bigtable/config/BigtableSinkTaskConfig.java b/google-cloud-bigtable-kafka-connect-sink/src/main/java/com/google/cloud/kafka/connect/bigtable/config/BigtableSinkTaskConfig.java new file mode 100644 index 0000000000..48129c32f2 --- /dev/null +++ b/google-cloud-bigtable-kafka-connect-sink/src/main/java/com/google/cloud/kafka/connect/bigtable/config/BigtableSinkTaskConfig.java @@ -0,0 +1,49 @@ +/* + * Copyright 2024 Google LLC + * + * Licensed under the Apache License, Version 2.0 (the "License"); + * you may not use this file except in compliance with the License. + * You may obtain a copy of the License at + * + * https://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ +package com.google.cloud.kafka.connect.bigtable.config; + +import java.util.Map; +import org.apache.kafka.common.config.ConfigDef; + +/** + * A class defining configuration of {@link + * com.google.cloud.kafka.connect.bigtable.BigtableSinkTask}. + */ +public class BigtableSinkTaskConfig extends BigtableSinkConfig { + public static String CONFIG_TASK_ID = "taskId"; + + /** + * The main constructor. + * + * @param properties The properties provided by the caller. + */ + public BigtableSinkTaskConfig(Map properties) { + super(getDefinition(), properties); + } + + /** + * @return {@link ConfigDef} used by Kafka Connect to advertise configuration options to the user + * and by us to perform basic validation of the user-provided values. + */ + public static ConfigDef getDefinition() { + return BigtableSinkConfig.getDefinition() + .defineInternal( + CONFIG_TASK_ID, + ConfigDef.Type.INT, + ConfigDef.NO_DEFAULT_VALUE, + ConfigDef.Importance.LOW); + } +} diff --git a/google-cloud-bigtable-kafka-connect-sink/src/main/java/com/google/cloud/kafka/connect/bigtable/config/InsertMode.java b/google-cloud-bigtable-kafka-connect-sink/src/main/java/com/google/cloud/kafka/connect/bigtable/config/InsertMode.java new file mode 100644 index 0000000000..a34481aa31 --- /dev/null +++ b/google-cloud-bigtable-kafka-connect-sink/src/main/java/com/google/cloud/kafka/connect/bigtable/config/InsertMode.java @@ -0,0 +1,21 @@ +/* + * Copyright 2024 Google LLC + * + * Licensed under the Apache License, Version 2.0 (the "License"); + * you may not use this file except in compliance with the License. + * You may obtain a copy of the License at + * + * https://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ +package com.google.cloud.kafka.connect.bigtable.config; + +public enum InsertMode { + INSERT, + UPSERT, +} diff --git a/google-cloud-bigtable-kafka-connect-sink/src/main/java/com/google/cloud/kafka/connect/bigtable/config/NullValueMode.java b/google-cloud-bigtable-kafka-connect-sink/src/main/java/com/google/cloud/kafka/connect/bigtable/config/NullValueMode.java new file mode 100644 index 0000000000..ad7a208f73 --- /dev/null +++ b/google-cloud-bigtable-kafka-connect-sink/src/main/java/com/google/cloud/kafka/connect/bigtable/config/NullValueMode.java @@ -0,0 +1,22 @@ +/* + * Copyright 2024 Google LLC + * + * Licensed under the Apache License, Version 2.0 (the "License"); + * you may not use this file except in compliance with the License. + * You may obtain a copy of the License at + * + * https://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ +package com.google.cloud.kafka.connect.bigtable.config; + +public enum NullValueMode { + WRITE, + IGNORE, + DELETE, +} diff --git a/google-cloud-bigtable-kafka-connect-sink/src/main/java/com/google/cloud/kafka/connect/bigtable/exception/BatchException.java b/google-cloud-bigtable-kafka-connect-sink/src/main/java/com/google/cloud/kafka/connect/bigtable/exception/BatchException.java new file mode 100644 index 0000000000..67575ac264 --- /dev/null +++ b/google-cloud-bigtable-kafka-connect-sink/src/main/java/com/google/cloud/kafka/connect/bigtable/exception/BatchException.java @@ -0,0 +1,28 @@ +/* + * Copyright 2024 Google LLC + * + * Licensed under the Apache License, Version 2.0 (the "License"); + * you may not use this file except in compliance with the License. + * You may obtain a copy of the License at + * + * https://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ +package com.google.cloud.kafka.connect.bigtable.exception; + +import org.apache.kafka.connect.errors.ConnectException; + +/** + * A wrapper exception class that may be thrown to explicitly mark a throw as supposed to fail an + * entire batch of input records. + */ +public class BatchException extends ConnectException { + public BatchException(Throwable t) { + super(t); + } +} diff --git a/google-cloud-bigtable-kafka-connect-sink/src/main/java/com/google/cloud/kafka/connect/bigtable/exception/InvalidBigtableSchemaModificationException.java b/google-cloud-bigtable-kafka-connect-sink/src/main/java/com/google/cloud/kafka/connect/bigtable/exception/InvalidBigtableSchemaModificationException.java new file mode 100644 index 0000000000..8c2fcfc5bd --- /dev/null +++ b/google-cloud-bigtable-kafka-connect-sink/src/main/java/com/google/cloud/kafka/connect/bigtable/exception/InvalidBigtableSchemaModificationException.java @@ -0,0 +1,29 @@ +/* + * Copyright 2025 Google LLC + * + * Licensed under the Apache License, Version 2.0 (the "License"); + * you may not use this file except in compliance with the License. + * You may obtain a copy of the License at + * + * https://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ +package com.google.cloud.kafka.connect.bigtable.exception; + +import org.apache.kafka.connect.errors.DataException; + +/** + * An {@link Exception} that signifies that input {@link org.apache.kafka.connect.sink.SinkRecord + * SinkRecord(s)} cause attempt of invalid Cloud Bigtable schema modification and thus is invalid + * and should not be retried. + */ +public class InvalidBigtableSchemaModificationException extends DataException { + public InvalidBigtableSchemaModificationException(String message) { + super(message); + } +} diff --git a/google-cloud-bigtable-kafka-connect-sink/src/main/java/com/google/cloud/kafka/connect/bigtable/mapping/KeyMapper.java b/google-cloud-bigtable-kafka-connect-sink/src/main/java/com/google/cloud/kafka/connect/bigtable/mapping/KeyMapper.java new file mode 100644 index 0000000000..81436dc906 --- /dev/null +++ b/google-cloud-bigtable-kafka-connect-sink/src/main/java/com/google/cloud/kafka/connect/bigtable/mapping/KeyMapper.java @@ -0,0 +1,262 @@ +/* + * Copyright 2024 Google LLC + * + * Licensed under the Apache License, Version 2.0 (the "License"); + * you may not use this file except in compliance with the License. + * You may obtain a copy of the License at + * + * https://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ +package com.google.cloud.kafka.connect.bigtable.mapping; + +import com.google.cloud.kafka.connect.bigtable.config.BigtableSinkConfig; +import java.io.ByteArrayOutputStream; +import java.io.IOException; +import java.nio.ByteBuffer; +import java.nio.charset.StandardCharsets; +import java.util.AbstractMap; +import java.util.Arrays; +import java.util.Collections; +import java.util.Iterator; +import java.util.List; +import java.util.Map; +import java.util.Optional; +import java.util.stream.Collectors; +import java.util.stream.Stream; +import org.apache.kafka.connect.data.Field; +import org.apache.kafka.connect.data.Struct; +import org.apache.kafka.connect.errors.DataException; + +/** + * A class responsible for converting Kafka {@link org.apache.kafka.connect.sink.SinkRecord + * SinkRecord(s)} into Cloud Bigtable row keys. + */ +public class KeyMapper { + final List> definition; + final byte[] delimiter; + + /** + * The main constructor. + * + * @param delimiter Delimiter in the mapping as per {@link + * com.google.cloud.kafka.connect.bigtable.config.BigtableSinkConfig#CONFIG_ROW_KEY_DELIMITER} + * @param definition Definition of the mapping as per {@link + * com.google.cloud.kafka.connect.bigtable.config.BigtableSinkConfig#CONFIG_ROW_KEY_DEFINITION}. + */ + public KeyMapper(String delimiter, List definition) { + this.delimiter = delimiter.getBytes(StandardCharsets.UTF_8); + this.definition = + definition.stream() + .map(s -> s.split("\\.")) + .map(Arrays::asList) + .collect(Collectors.toList()); + } + + /** + * Converts input data into Cloud Bigtable row key bytes as described in {@link + * BigtableSinkConfig#getDefinition()}. + * + * @param kafkaKey An {@link Object} to be converted into Cloud Bigtable row key. + * @return {@link Optional#empty()} if the input doesn't convert into a valid Cloud Bigtable row + * key, {@link Optional} containing row Cloud Bigtable row key bytes the input converts into + * otherwise. + */ + public byte[] getKey(Object kafkaKey) { + ensureKeyElementIsNotNull(kafkaKey); + Stream keyParts = + this.getDefinition(kafkaKey).stream() + .map((d) -> serializeTopLevelKeyElement(extractField(kafkaKey, d.iterator()))); + return concatenateByteArrays(new byte[0], keyParts, delimiter, new byte[0]); + } + + /** + * Returns key definition as configured during object creation or extracted from the object being + * mapped if it's been configured to an empty {@link List}. + * + * @param kafkaKey {@link org.apache.kafka.connect.sink.SinkRecord SinkRecord's} key. + * @return {@link List} containing {@link List Lists} of key fields that need to be retrieved and + * concatenated to construct the Cloud Bigtable row key. + *

See {@link KeyMapper#extractField(Object, Iterator)} for details on semantics of the + * inner list. + */ + private List> getDefinition(Object kafkaKey) { + if (this.definition.isEmpty()) { + Optional> maybeRootFields = getFieldsOfRootValue(kafkaKey); + if (maybeRootFields.isEmpty()) { + List rootElementDefinition = List.of(); + return List.of(rootElementDefinition); + } else { + return maybeRootFields.get().stream() + .map(Collections::singletonList) + .collect(Collectors.toList()); + } + } + return this.definition; + } + + /** + * Extracts names of child fields of the value. + * + * @param kafkaKey {@link org.apache.kafka.connect.sink.SinkRecord SinkRecord's} key. + * @return {@link Optional#empty()} if the input value has no children, {@link Optional} + * containing names of its child fields otherwise. + */ + private static Optional> getFieldsOfRootValue(Object kafkaKey) { + if (kafkaKey instanceof Struct) { + return Optional.of( + ((Struct) kafkaKey) + .schema().fields().stream().map(Field::name).collect(Collectors.toList())); + } else if (kafkaKey instanceof Map) { + return Optional.of( + ((Map) kafkaKey) + .keySet().stream().map(Object::toString).collect(Collectors.toList())); + } else { + return Optional.empty(); + } + } + + /** + * Extract possibly nested fields from the input value. + * + * @param value {@link org.apache.kafka.connect.sink.SinkRecord SinkRecord's} key or some its + * child. + * @param fields Fields that need to be accessed before the target value is reached. + * @return Extracted nested field. + */ + private Object extractField(Object value, Iterator fields) { + ensureKeyElementIsNotNull(value); + if (!fields.hasNext()) { + return value; + } + String field = fields.next(); + if (value instanceof Struct) { + Struct struct = (Struct) value; + // Note that getWithoutDefault() throws if such a field does not exist. + return extractField(struct.getWithoutDefault(field), fields); + } else if (value instanceof Map) { + Map map = (Map) value; + if (!map.containsKey(field)) { + throw new DataException("Map contains no value for key `" + field + "`."); + } + return extractField(map.get(field), fields); + } else { + throw new DataException( + "Unexpected class `" + + value.getClass() + + "` doesn't " + + "support extracting field `" + + field + + "` using a dot."); + } + } + + private static byte[] serializeTopLevelKeyElement(Object keyElement) { + ensureKeyElementIsNotNull(keyElement); + return serializeKeyElement(keyElement); + } + + /** + * Serializes Kafka Connect entry key. + * + *

We implement custom serialization since {@link Object#toString()} mangles arrays. + * + * @param keyElement {@link org.apache.kafka.connect.sink.SinkRecord SinkRecord's} key to be + * serialized. + * @return Serialization of the input value. + */ + private static byte[] serializeKeyElement(Object keyElement) { + if (keyElement == null) { + // Note that it's needed for serializing null-containing Maps and Lists. + return "null".getBytes(StandardCharsets.UTF_8); + } else if (keyElement instanceof byte[]) { + // Note that it breaks compatibility with Confluent's sink. + return (byte[]) keyElement; + } else if (keyElement instanceof ByteBuffer) { + return ((ByteBuffer) keyElement).array(); + } else if (keyElement instanceof List) { + List list = (List) keyElement; + return concatenateByteArrays( + "[", list.stream().map(o -> o.toString().getBytes(StandardCharsets.UTF_8)), ", ", "]"); + } else if (keyElement instanceof Map) { + Map map = (Map) keyElement; + return concatenateByteArrays( + "{", + map.entrySet().stream() + .map( + e -> + concatenateByteArrays( + new byte[0], + Stream.of( + serializeKeyElement(e.getKey()), serializeKeyElement(e.getValue())), + "=".getBytes(StandardCharsets.UTF_8), + new byte[0])), + // Note that Map and Struct have different delimiters for compatibility's sake. + ", ", + "}"); + } else if (keyElement instanceof Struct) { + Struct struct = (Struct) keyElement; + return concatenateByteArrays( + "Struct{", + struct.schema().fields().stream() + .flatMap( + f -> + Optional.ofNullable(struct.get(f)) + .map(v -> new AbstractMap.SimpleImmutableEntry<>(f.name(), v)) + .stream()) + .map( + e -> + concatenateByteArrays( + new byte[0], + Stream.of( + serializeKeyElement(e.getKey()), serializeKeyElement(e.getValue())), + "=".getBytes(StandardCharsets.UTF_8), + new byte[0])), + // Note that Map and Struct have different delimiters for compatibility's sake. + ",", + "}"); + } else { + // TODO: handle logical data types. + return keyElement.toString().getBytes(StandardCharsets.UTF_8); + } + } + + private static void ensureKeyElementIsNotNull(Object value) { + if (value == null) { + // Matching Confluent's sink behavior. + throw new DataException("Error with row key definition: row key fields cannot be null."); + } + } + + private static byte[] concatenateByteArrays( + byte[] start, Stream byteArrays, byte[] delimiter, byte[] end) { + try (ByteArrayOutputStream bos = new ByteArrayOutputStream()) { + bos.write(start); + for (Iterator it = byteArrays.iterator(); it.hasNext(); ) { + byte[] keyPart = it.next(); + bos.write(keyPart); + if (it.hasNext()) { + bos.write(delimiter); + } + } + bos.write(end); + return bos.toByteArray(); + } catch (IOException e) { + throw new DataException("Concatenation of Cloud Bigtable key failed.", e); + } + } + + private static byte[] concatenateByteArrays( + String start, Stream byteArrays, String delimiter, String end) { + return concatenateByteArrays( + start.getBytes(StandardCharsets.UTF_8), + byteArrays, + delimiter.getBytes(StandardCharsets.UTF_8), + end.getBytes(StandardCharsets.UTF_8)); + } +} diff --git a/google-cloud-bigtable-kafka-connect-sink/src/main/java/com/google/cloud/kafka/connect/bigtable/mapping/MutationData.java b/google-cloud-bigtable-kafka-connect-sink/src/main/java/com/google/cloud/kafka/connect/bigtable/mapping/MutationData.java new file mode 100644 index 0000000000..65d4fc1c47 --- /dev/null +++ b/google-cloud-bigtable-kafka-connect-sink/src/main/java/com/google/cloud/kafka/connect/bigtable/mapping/MutationData.java @@ -0,0 +1,63 @@ +/* + * Copyright 2024 Google LLC + * + * Licensed under the Apache License, Version 2.0 (the "License"); + * you may not use this file except in compliance with the License. + * You may obtain a copy of the License at + * + * https://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ +package com.google.cloud.kafka.connect.bigtable.mapping; + +import com.google.cloud.bigtable.data.v2.models.Mutation; +import com.google.cloud.bigtable.data.v2.models.RowMutationEntry; +import com.google.protobuf.ByteString; +import java.util.Set; + +/** + * A class representing single Kafka {@link org.apache.kafka.connect.sink.SinkRecord SinkRecord's} + * output to be written into Cloud Bigtable. + */ +public class MutationData { + private final String targetTable; + private final ByteString rowKey; + private final Mutation mutation; + private final Set requiredColumnFamilies; + + public MutationData( + String targetTable, + ByteString rowKey, + Mutation mutation, + Set requiredColumnFamilies) { + this.targetTable = targetTable; + this.rowKey = rowKey; + this.mutation = mutation; + this.requiredColumnFamilies = requiredColumnFamilies; + } + + public String getTargetTable() { + return targetTable; + } + + public ByteString getRowKey() { + return rowKey; + } + + public RowMutationEntry getUpsertMutation() { + return RowMutationEntry.createFromMutationUnsafe(this.rowKey, this.mutation); + } + + public Mutation getInsertMutation() { + return mutation; + } + + public Set getRequiredColumnFamilies() { + return requiredColumnFamilies; + } +} diff --git a/google-cloud-bigtable-kafka-connect-sink/src/main/java/com/google/cloud/kafka/connect/bigtable/mapping/MutationDataBuilder.java b/google-cloud-bigtable-kafka-connect-sink/src/main/java/com/google/cloud/kafka/connect/bigtable/mapping/MutationDataBuilder.java new file mode 100644 index 0000000000..229853dae4 --- /dev/null +++ b/google-cloud-bigtable-kafka-connect-sink/src/main/java/com/google/cloud/kafka/connect/bigtable/mapping/MutationDataBuilder.java @@ -0,0 +1,81 @@ +/* + * Copyright 2024 Google LLC + * + * Licensed under the Apache License, Version 2.0 (the "License"); + * you may not use this file except in compliance with the License. + * You may obtain a copy of the License at + * + * https://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ +package com.google.cloud.kafka.connect.bigtable.mapping; + +import com.google.cloud.bigtable.data.v2.models.Mutation; +import com.google.cloud.bigtable.data.v2.models.Range; +import com.google.common.annotations.VisibleForTesting; +import com.google.protobuf.ByteString; +import java.util.HashSet; +import java.util.Optional; +import java.util.Set; + +/** A builder class for {@link MutationData}. */ +public class MutationDataBuilder { + private final Mutation mutation; + private boolean mutationIsEmpty; + private final Set requiredColumnFamilies; + + @VisibleForTesting + MutationDataBuilder(Mutation mutation) { + this.mutation = mutation; + mutationIsEmpty = true; + requiredColumnFamilies = new HashSet<>(); + } + + public MutationDataBuilder() { + this(Mutation.create()); + } + + /** + * Tries to convert this object into {@link MutationData}. + * + * @param targetTable - Cloud Bigtable {@link com.google.cloud.bigtable.admin.v2.models.Table} + * this mutation is to be written to. + * @param rowKey - Cloud Bigtable row key this mutation is to be written to. + * @return {@link Optional#empty()} if this mutation is empty, an {@link Optional} containing this + * mutation ready to be written to Cloud Bigtable otherwise. + */ + public Optional maybeBuild(String targetTable, ByteString rowKey) { + return this.mutationIsEmpty + ? Optional.empty() + : Optional.of( + new MutationData(targetTable, rowKey, this.mutation, this.requiredColumnFamilies)); + } + + public void deleteRow() { + mutationIsEmpty = false; + mutation.deleteRow(); + } + + public void deleteFamily(String columnFamily) { + mutationIsEmpty = false; + mutation.deleteFamily(columnFamily); + } + + public void deleteCells( + String columnFamily, ByteString columnQualifier, Range.TimestampRange timestampRange) { + mutationIsEmpty = false; + mutation.deleteCells(columnFamily, columnQualifier, timestampRange); + } + + public void setCell( + String columnFamily, ByteString columnQualifier, long timestampMicros, ByteString value) { + mutationIsEmpty = false; + requiredColumnFamilies.add(columnFamily); + mutation.setCell(columnFamily, columnQualifier, timestampMicros, value); + } +} diff --git a/google-cloud-bigtable-kafka-connect-sink/src/main/java/com/google/cloud/kafka/connect/bigtable/mapping/ValueMapper.java b/google-cloud-bigtable-kafka-connect-sink/src/main/java/com/google/cloud/kafka/connect/bigtable/mapping/ValueMapper.java new file mode 100644 index 0000000000..f97fc78f53 --- /dev/null +++ b/google-cloud-bigtable-kafka-connect-sink/src/main/java/com/google/cloud/kafka/connect/bigtable/mapping/ValueMapper.java @@ -0,0 +1,245 @@ +/* + * Copyright 2024 Google LLC + * + * Licensed under the Apache License, Version 2.0 (the "License"); + * you may not use this file except in compliance with the License. + * You may obtain a copy of the License at + * + * https://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ +package com.google.cloud.kafka.connect.bigtable.mapping; + +import com.fasterxml.jackson.core.JsonGenerator; +import com.fasterxml.jackson.core.JsonProcessingException; +import com.fasterxml.jackson.databind.ObjectMapper; +import com.fasterxml.jackson.databind.SerializerProvider; +import com.fasterxml.jackson.databind.module.SimpleModule; +import com.fasterxml.jackson.databind.ser.std.StdSerializer; +import com.google.cloud.ByteArray; +import com.google.cloud.bigtable.data.v2.models.Range; +import com.google.cloud.kafka.connect.bigtable.config.NullValueMode; +import com.google.common.annotations.VisibleForTesting; +import com.google.protobuf.ByteString; +import java.io.IOException; +import java.math.BigDecimal; +import java.nio.charset.StandardCharsets; +import java.util.AbstractMap; +import java.util.ArrayList; +import java.util.Date; +import java.util.List; +import java.util.Map; +import java.util.stream.Collectors; +import javax.annotation.Nonnull; +import org.apache.hadoop.hbase.util.Bytes; +import org.apache.kafka.common.utils.Utils; +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.errors.DataException; + +/** + * A class responsible for converting Kafka {@link org.apache.kafka.connect.sink.SinkRecord + * SinkRecord(s)} into Cloud Bigtable {@link com.google.cloud.bigtable.data.v2.models.Mutation + * Mutation(s)}. + */ +public class ValueMapper { + public final String defaultColumnFamily; + public final ByteString defaultColumnQualifier; + private final NullValueMode nullMode; + private static final ObjectMapper jsonMapper = getJsonMapper(); + + /** + * The main constructor. + * + * @param defaultColumnFamily Default column family as per {@link + * com.google.cloud.kafka.connect.bigtable.config.BigtableSinkConfig#CONFIG_DEFAULT_COLUMN_FAMILY}. + * @param defaultColumnQualifier Default column as per {@link + * com.google.cloud.kafka.connect.bigtable.config.BigtableSinkConfig#CONFIG_ROW_KEY_DELIMITER}. + */ + public ValueMapper( + String defaultColumnFamily, String defaultColumnQualifier, @Nonnull NullValueMode nullMode) { + this.defaultColumnFamily = Utils.isBlank(defaultColumnFamily) ? null : defaultColumnFamily; + this.defaultColumnQualifier = + Utils.isBlank(defaultColumnQualifier) + ? null + : ByteString.copyFrom(defaultColumnQualifier.getBytes(StandardCharsets.UTF_8)); + this.nullMode = nullMode; + } + + /** + * Creates a {@link MutationDataBuilder} that can be used to create a {@link MutationData} + * representing the input Kafka Connect value as Cloud Bigtable mutations that need to be applied. + * + * @param rootKafkaValue The value to be converted into Cloud Bigtable {@link + * com.google.cloud.bigtable.data.v2.models.Mutation Mutation(s)}. + * @param timestampMicros The timestamp the mutations will be created at in microseconds. + */ + public MutationDataBuilder getRecordMutationDataBuilder( + Object rootKafkaValue, long timestampMicros) { + MutationDataBuilder mutationDataBuilder = createMutationDataBuilder(); + if (rootKafkaValue == null && nullMode == NullValueMode.IGNORE) { + // Do nothing + } else if (rootKafkaValue == null && nullMode == NullValueMode.DELETE) { + mutationDataBuilder.deleteRow(); + } else if (rootKafkaValue instanceof Map || rootKafkaValue instanceof Struct) { + for (Map.Entry field : getChildren(rootKafkaValue)) { + String kafkaFieldName = field.getKey().toString(); + Object kafkaFieldValue = field.getValue(); + if (kafkaFieldValue == null && nullMode == NullValueMode.IGNORE) { + continue; + } else if (kafkaFieldValue == null && nullMode == NullValueMode.DELETE) { + mutationDataBuilder.deleteFamily(kafkaFieldName); + } else if (kafkaFieldValue instanceof Map || kafkaFieldValue instanceof Struct) { + for (Map.Entry subfield : getChildren(kafkaFieldValue)) { + ByteString kafkaSubfieldName = + ByteString.copyFrom(subfield.getKey().toString().getBytes(StandardCharsets.UTF_8)); + Object kafkaSubfieldValue = subfield.getValue(); + if (kafkaSubfieldValue == null && nullMode == NullValueMode.IGNORE) { + continue; + } else if (kafkaSubfieldValue == null && nullMode == NullValueMode.DELETE) { + mutationDataBuilder.deleteCells( + kafkaFieldName, + kafkaSubfieldName, + Range.TimestampRange.create(0, timestampMicros)); + } else { + mutationDataBuilder.setCell( + kafkaFieldName, + kafkaSubfieldName, + timestampMicros, + ByteString.copyFrom(serialize(kafkaSubfieldValue))); + } + } + } else { + if (defaultColumnFamily != null) { + mutationDataBuilder.setCell( + defaultColumnFamily, + ByteString.copyFrom(kafkaFieldName.getBytes(StandardCharsets.UTF_8)), + timestampMicros, + ByteString.copyFrom(serialize(kafkaFieldValue))); + } + } + } + } else { + if (defaultColumnFamily != null && defaultColumnQualifier != null) { + mutationDataBuilder.setCell( + defaultColumnFamily, + defaultColumnQualifier, + timestampMicros, + ByteString.copyFrom(serialize(rootKafkaValue))); + } + } + return mutationDataBuilder; + } + + @VisibleForTesting + // Method only needed for use in tests. It could be inlined otherwise. + protected MutationDataBuilder createMutationDataBuilder() { + return new MutationDataBuilder(); + } + + /** + * @param mapOrStruct {@link Map} or {@link Struct} whose children we want to list + * @return {@link List} of names or keys of input value's child entries. + */ + private static List> getChildren(Object mapOrStruct) { + if (mapOrStruct instanceof Map) { + @SuppressWarnings("unchecked") + Map kafkaMapValue = (Map) mapOrStruct; + return new ArrayList<>(kafkaMapValue.entrySet()); + } else if (mapOrStruct instanceof Struct) { + Struct kafkaStructValue = (Struct) mapOrStruct; + return kafkaStructValue.schema().fields().stream() + .map( + f -> + new AbstractMap.SimpleImmutableEntry<>( + (Object) f.name(), kafkaStructValue.get(f))) + .collect(Collectors.toList()); + } else { + throw new IllegalStateException(); + } + } + + /** + * @param value Input value. + * @return Input value's serialization's bytes that will be written to Cloud Bigtable as a cell's + * value. + */ + private static byte[] serialize(Object value) { + if (value == null) { + return new byte[0]; + } + if (value instanceof byte[]) { + return (byte[]) value; + } else if (value instanceof ByteArray) { + return serialize(((ByteArray) value).toByteArray()); + } else if (value instanceof Integer) { + return Bytes.toBytes((Integer) value); + } else if (value instanceof Long) { + return Bytes.toBytes((Long) value); + } else if (value instanceof Short) { + return Bytes.toBytes((Short) value); + } else if (value instanceof Byte) { + return Bytes.toBytes((Byte) value); + } else if (value instanceof Float) { + return Bytes.toBytes((Float) value); + } else if (value instanceof Double) { + return Bytes.toBytes((Double) value); + } else if (value instanceof Boolean) { + return Bytes.toBytes((Boolean) value); + } else if (value instanceof String) { + return Bytes.toBytes((String) value); + } else if (value instanceof Character) { + return serialize(Character.toString((Character) value)); + } else if (value instanceof Date) { + // TODO: implement. + throw new DataException("TODO"); + } else if (value instanceof BigDecimal) { + // TODO: implement. + throw new DataException("TODO"); + } else if (value instanceof Map || value instanceof Struct || value instanceof List) { + try { + return jsonMapper.writeValueAsBytes(value); + } catch (JsonProcessingException e) { + throw new DataException("Failed to deserialize a(n) " + value.getClass(), e); + } + } else { + throw new DataException( + "Unsupported serialization of an unexpected class `" + value.getClass() + "`."); + } + } + + /** + * @return {@link ObjectMapper} that can serialize all the Kafka Connect types. + */ + private static ObjectMapper getJsonMapper() { + ObjectMapper mapper = new ObjectMapper(); + SimpleModule mapperModule = new SimpleModule("KafkaConnectSerializer"); + mapperModule.addSerializer(Struct.class, new StructJsonSerializer(Struct.class)); + mapper.registerModule(mapperModule); + return mapper; + } + + private static class StructJsonSerializer extends StdSerializer { + protected StructJsonSerializer(Class t) { + super(t); + } + + @Override + public void serialize(Struct value, JsonGenerator gen, SerializerProvider provider) + throws IOException { + Schema schema = value.schema(); + gen.writeStartObject(); + for (Field field : schema.fields()) { + String fieldName = field.name(); + gen.writeObjectField(fieldName, value.getWithoutDefault(fieldName)); + } + gen.writeEndObject(); + } + } +} diff --git a/google-cloud-bigtable-kafka-connect-sink/src/main/java/com/google/cloud/kafka/connect/bigtable/version/PackageMetadata.java b/google-cloud-bigtable-kafka-connect-sink/src/main/java/com/google/cloud/kafka/connect/bigtable/version/PackageMetadata.java new file mode 100644 index 0000000000..0954d8d01a --- /dev/null +++ b/google-cloud-bigtable-kafka-connect-sink/src/main/java/com/google/cloud/kafka/connect/bigtable/version/PackageMetadata.java @@ -0,0 +1,39 @@ +/* + * Copyright 2024 Google LLC + * + * Licensed under the Apache License, Version 2.0 (the "License"); + * you may not use this file except in compliance with the License. + * You may obtain a copy of the License at + * + * https://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ +package com.google.cloud.kafka.connect.bigtable.version; + +import java.util.Optional; + +/** A class responsible for extracting maven-generated package metadata. */ +public class PackageMetadata { + public static String UNKNOWN_VERSION = "unknown"; + + /** + * Extracts version information from the package metadata. + * + * @return String representation of the version of the package. Is equal to {@link + * PackageMetadata#UNKNOWN_VERSION} when the information is missing from package metadata. + */ + public static String getVersion() { + Optional discoveredVersion = Optional.empty(); + try { + discoveredVersion = + Optional.ofNullable(PackageMetadata.class.getPackage().getImplementationVersion()); + } catch (NullPointerException ignored) { + } + return discoveredVersion.orElse(UNKNOWN_VERSION); + } +} diff --git a/google-cloud-bigtable-kafka-connect-sink/src/test/java/com/google/cloud/kafka/connect/bigtable/BigtableSinkConnectorTest.java b/google-cloud-bigtable-kafka-connect-sink/src/test/java/com/google/cloud/kafka/connect/bigtable/BigtableSinkConnectorTest.java new file mode 100644 index 0000000000..6b3141d80e --- /dev/null +++ b/google-cloud-bigtable-kafka-connect-sink/src/test/java/com/google/cloud/kafka/connect/bigtable/BigtableSinkConnectorTest.java @@ -0,0 +1,84 @@ +/* + * Copyright 2024 Google LLC + * + * Licensed under the Apache License, Version 2.0 (the "License"); + * you may not use this file except in compliance with the License. + * You may obtain a copy of the License at + * + * https://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ +package com.google.cloud.kafka.connect.bigtable; + +import static com.google.cloud.kafka.connect.bigtable.config.BigtableSinkTaskConfig.CONFIG_TASK_ID; +import static org.junit.Assert.assertEquals; +import static org.junit.Assert.assertNotNull; + +import com.google.cloud.kafka.connect.bigtable.util.BasicPropertiesFactory; +import java.util.HashMap; +import java.util.List; +import java.util.Map; +import org.junit.Before; +import org.junit.Test; +import org.junit.runner.RunWith; +import org.junit.runners.JUnit4; + +@RunWith(JUnit4.class) +public class BigtableSinkConnectorTest { + BigtableSinkConnector connector; + + @Before + public void setUp() { + connector = new BigtableSinkConnector(); + } + + @Test + public void testConfig() { + assertNotNull(connector.config()); + } + + @Test + public void testValidate() { + connector.validate(BasicPropertiesFactory.getSinkProps()); + } + + @Test + public void testStart() { + connector.start(BasicPropertiesFactory.getSinkProps()); + } + + @Test + public void testStop() { + connector.stop(); + } + + @Test + public void testTaskClass() { + assertEquals(BigtableSinkTask.class, connector.taskClass()); + } + + @Test + public void testTaskConfigs() { + Map connectorConfig = BasicPropertiesFactory.getSinkProps(); + connector.start(new HashMap<>(connectorConfig)); + int maxTasks = 1000; + List> taskConfigs = connector.taskConfigs(maxTasks); + assertEquals(maxTasks, taskConfigs.size()); + for (Integer i = 0; i < maxTasks; i++) { + Map taskConfig = taskConfigs.get(i); + assertEquals(i.toString(), taskConfig.get(CONFIG_TASK_ID)); + taskConfig.remove(CONFIG_TASK_ID); + assertEquals(connectorConfig, taskConfig); + } + } + + @Test + public void testVersion() { + assertNotNull(connector.version()); + } +} diff --git a/google-cloud-bigtable-kafka-connect-sink/src/test/java/com/google/cloud/kafka/connect/bigtable/BigtableSinkTaskTest.java b/google-cloud-bigtable-kafka-connect-sink/src/test/java/com/google/cloud/kafka/connect/bigtable/BigtableSinkTaskTest.java new file mode 100644 index 0000000000..3348d4bf60 --- /dev/null +++ b/google-cloud-bigtable-kafka-connect-sink/src/test/java/com/google/cloud/kafka/connect/bigtable/BigtableSinkTaskTest.java @@ -0,0 +1,612 @@ +/* + * Copyright 2024 Google LLC + * + * Licensed under the Apache License, Version 2.0 (the "License"); + * you may not use this file except in compliance with the License. + * You may obtain a copy of the License at + * + * https://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ +package com.google.cloud.kafka.connect.bigtable; + +import static com.google.cloud.kafka.connect.bigtable.config.BigtableSinkConfig.CONFIG_AUTO_CREATE_COLUMN_FAMILIES; +import static com.google.cloud.kafka.connect.bigtable.config.BigtableSinkConfig.CONFIG_AUTO_CREATE_TABLES; +import static com.google.cloud.kafka.connect.bigtable.config.BigtableSinkConfig.CONFIG_ERROR_MODE; +import static com.google.cloud.kafka.connect.bigtable.config.BigtableSinkConfig.CONFIG_INSERT_MODE; +import static com.google.cloud.kafka.connect.bigtable.config.BigtableSinkConfig.CONFIG_TABLE_NAME_FORMAT; +import static com.google.cloud.kafka.connect.bigtable.util.FutureUtil.completedApiFuture; +import static com.google.cloud.kafka.connect.bigtable.util.MockUtil.assertTotalNumberOfInvocations; +import static org.junit.Assert.assertEquals; +import static org.junit.Assert.assertFalse; +import static org.junit.Assert.assertNotNull; +import static org.junit.Assert.assertThrows; +import static org.junit.Assert.assertTrue; +import static org.mockito.Mockito.any; +import static org.mockito.Mockito.anyLong; +import static org.mockito.Mockito.anyString; +import static org.mockito.Mockito.argThat; +import static org.mockito.Mockito.doAnswer; +import static org.mockito.Mockito.doReturn; +import static org.mockito.Mockito.doThrow; +import static org.mockito.Mockito.eq; +import static org.mockito.Mockito.mock; +import static org.mockito.Mockito.reset; +import static org.mockito.Mockito.spy; +import static org.mockito.Mockito.times; +import static org.mockito.Mockito.verify; +import static org.mockito.Mockito.verifyNoMoreInteractions; +import static org.mockito.MockitoAnnotations.openMocks; + +import com.google.api.gax.batching.Batcher; +import com.google.api.gax.rpc.ApiException; +import com.google.bigtable.admin.v2.Table; +import com.google.cloud.bigtable.admin.v2.BigtableTableAdminClient; +import com.google.cloud.bigtable.data.v2.BigtableDataClient; +import com.google.cloud.bigtable.data.v2.models.Mutation; +import com.google.cloud.bigtable.data.v2.models.RowMutationEntry; +import com.google.cloud.kafka.connect.bigtable.autocreate.BigtableSchemaManager; +import com.google.cloud.kafka.connect.bigtable.autocreate.ResourceCreationResult; +import com.google.cloud.kafka.connect.bigtable.config.BigtableErrorMode; +import com.google.cloud.kafka.connect.bigtable.config.BigtableSinkTaskConfig; +import com.google.cloud.kafka.connect.bigtable.config.InsertMode; +import com.google.cloud.kafka.connect.bigtable.exception.InvalidBigtableSchemaModificationException; +import com.google.cloud.kafka.connect.bigtable.mapping.KeyMapper; +import com.google.cloud.kafka.connect.bigtable.mapping.MutationData; +import com.google.cloud.kafka.connect.bigtable.mapping.MutationDataBuilder; +import com.google.cloud.kafka.connect.bigtable.mapping.ValueMapper; +import com.google.cloud.kafka.connect.bigtable.util.ApiExceptionFactory; +import com.google.cloud.kafka.connect.bigtable.util.BasicPropertiesFactory; +import com.google.cloud.kafka.connect.bigtable.util.FutureUtil; +import com.google.protobuf.ByteString; +import java.nio.charset.StandardCharsets; +import java.util.AbstractMap; +import java.util.ArrayList; +import java.util.Arrays; +import java.util.Collections; +import java.util.HashMap; +import java.util.LinkedHashMap; +import java.util.List; +import java.util.Map; +import java.util.Optional; +import java.util.Set; +import java.util.concurrent.CompletableFuture; +import java.util.concurrent.ExecutionException; +import java.util.concurrent.Future; +import java.util.stream.Collectors; +import java.util.stream.IntStream; +import org.apache.kafka.common.record.TimestampType; +import org.apache.kafka.connect.errors.ConnectException; +import org.apache.kafka.connect.sink.ErrantRecordReporter; +import org.apache.kafka.connect.sink.SinkRecord; +import org.apache.kafka.connect.sink.SinkTaskContext; +import org.junit.Before; +import org.junit.Test; +import org.junit.runner.RunWith; +import org.junit.runners.JUnit4; +import org.mockito.Mock; +import org.slf4j.Logger; + +@RunWith(JUnit4.class) +public class BigtableSinkTaskTest { + TestBigtableSinkTask task; + BigtableSinkTaskConfig config; + @Mock BigtableDataClient bigtableData; + @Mock BigtableTableAdminClient bigtableAdmin; + @Mock KeyMapper keyMapper; + @Mock ValueMapper valueMapper; + @Mock BigtableSchemaManager schemaManager; + @Mock SinkTaskContext context; + @Mock ErrantRecordReporter errorReporter; + + @Before + public void setUp() { + openMocks(this); + config = new BigtableSinkTaskConfig(BasicPropertiesFactory.getTaskProps()); + } + + @Test + public void testStart() { + task = spy(new TestBigtableSinkTask(null, null, null, null, null, null, null)); + task.start(BasicPropertiesFactory.getTaskProps()); + } + + @Test + public void testStop() throws InterruptedException { + for (List test : + List.of( + List.of(false, false), + List.of(false, true), + List.of(true, false), + List.of(true, true))) { + assertEquals(2, test.size()); + boolean adminIsNotNull = test.get(0); + boolean dataIsNotNull = test.get(1); + int expectedAdminCloseCallCount = adminIsNotNull ? 1 : 0; + int expectedDataCloseCallCount = dataIsNotNull ? 1 : 0; + + BigtableTableAdminClient maybeAdmin = adminIsNotNull ? bigtableAdmin : null; + BigtableDataClient maybeData = dataIsNotNull ? bigtableData : null; + task = new TestBigtableSinkTask(null, maybeData, maybeAdmin, null, null, null, null); + Batcher batcher = mock(Batcher.class); + doReturn(completedApiFuture(null)).when(batcher).closeAsync(); + task.getBatchers().put("batcherTable", batcher); + + doThrow(new RuntimeException()).when(bigtableAdmin).close(); + doThrow(new RuntimeException()).when(bigtableData).close(); + + assertFalse(task.getBatchers().isEmpty()); + task.stop(); + assertTrue(task.getBatchers().isEmpty()); + verify(bigtableAdmin, times(expectedAdminCloseCallCount)).close(); + verify(bigtableData, times(expectedDataCloseCallCount)).close(); + verify(batcher, times(1)).closeAsync(); + + reset(bigtableAdmin); + reset(bigtableData); + } + } + + @Test + public void testVersion() { + task = spy(new TestBigtableSinkTask(null, null, null, null, null, null, null)); + assertNotNull(task.version()); + } + + @Test + public void testGetTableName() { + SinkRecord record = new SinkRecord("topic${test}", 1, null, null, null, null, 1); + for (Map.Entry test : + List.of( + new AbstractMap.SimpleImmutableEntry<>( + "prefix_${topic}_suffix", "prefix_topic${test}_suffix"), + new AbstractMap.SimpleImmutableEntry<>( + "prefix_${topic_suffix", "prefix_${topic_suffix"), + new AbstractMap.SimpleImmutableEntry<>("prefix_$topic_suffix", "prefix_$topic_suffix"), + new AbstractMap.SimpleImmutableEntry<>("prefix_${bad}_suffix", "prefix_${bad}_suffix"), + new AbstractMap.SimpleImmutableEntry<>("noSubstitution", "noSubstitution"))) { + Map props = BasicPropertiesFactory.getTaskProps(); + props.put(CONFIG_TABLE_NAME_FORMAT, test.getKey()); + task = + new TestBigtableSinkTask( + new BigtableSinkTaskConfig(props), null, null, null, null, null, null); + assertEquals(test.getValue(), task.getTableName(record)); + } + } + + @Test + public void testCreateRecordMutationDataEmptyKey() { + task = new TestBigtableSinkTask(config, null, null, keyMapper, null, null, null); + doReturn(new byte[0]).when(keyMapper).getKey(any()); + SinkRecord record = new SinkRecord("topic", 1, null, new Object(), null, null, 1); + assertThrows(ConnectException.class, () -> task.createRecordMutationData(record)); + } + + @Test + public void testCreateRecordMutationDataNonemptyKey() { + SinkRecord record = new SinkRecord("topic", 1, null, new Object(), null, null, 1); + task = new TestBigtableSinkTask(config, null, null, keyMapper, valueMapper, null, null); + + byte[] rowKey = "rowKey".getBytes(StandardCharsets.UTF_8); + doReturn(rowKey).when(keyMapper).getKey(any()); + doAnswer( + i -> { + MutationDataBuilder builder = new MutationDataBuilder(); + return builder; + }) + .when(valueMapper) + .getRecordMutationDataBuilder(any(), anyLong()); + assertTrue(task.createRecordMutationData(record).isEmpty()); + + doAnswer( + i -> { + MutationDataBuilder builder = new MutationDataBuilder(); + builder.deleteRow(); + return builder; + }) + .when(valueMapper) + .getRecordMutationDataBuilder(any(), anyLong()); + assertTrue(task.createRecordMutationData(record).isPresent()); + } + + @Test + public void testErrorReporterWithDLQ() { + doReturn(errorReporter).when(context).errantRecordReporter(); + task = new TestBigtableSinkTask(null, null, null, null, null, null, context); + SinkRecord record = new SinkRecord(null, 1, null, null, null, null, 1); + Throwable t = new Exception("testErrorReporterWithDLQ"); + verifyNoMoreInteractions(task.getLogger()); + task.reportError(record, t); + verify(errorReporter, times(1)).report(record, t); + } + + @Test + public void testErrorReporterNoDLQIgnoreMode() { + Map props = BasicPropertiesFactory.getTaskProps(); + props.put(CONFIG_ERROR_MODE, BigtableErrorMode.IGNORE.name()); + BigtableSinkTaskConfig config = new BigtableSinkTaskConfig(props); + + doThrow(new NoSuchMethodError()).when(context).errantRecordReporter(); + task = new TestBigtableSinkTask(config, null, null, null, null, null, context); + SinkRecord record = new SinkRecord(null, 1, null, null, null, null, 1); + verifyNoMoreInteractions(task.getLogger()); + verifyNoMoreInteractions(errorReporter); + task.reportError(record, new Exception("testErrorReporterWithDLQ")); + } + + @Test + public void testErrorReporterNoDLQWarnMode() { + Map props = BasicPropertiesFactory.getTaskProps(); + props.put(CONFIG_ERROR_MODE, BigtableErrorMode.WARN.name()); + BigtableSinkTaskConfig config = new BigtableSinkTaskConfig(props); + + doReturn(null).when(context).errantRecordReporter(); + task = new TestBigtableSinkTask(config, null, null, null, null, null, context); + SinkRecord record = new SinkRecord(null, 1, null, "key", null, null, 1); + Throwable t = new Exception("testErrorReporterNoDLQWarnMode"); + verifyNoMoreInteractions(errorReporter); + task.reportError(record, t); + verify(task.getLogger(), times(1)).warn(anyString(), eq(record.key()), eq(t)); + } + + @Test + public void testErrorReporterNoDLQFailMode() { + Map props = BasicPropertiesFactory.getTaskProps(); + props.put(CONFIG_ERROR_MODE, BigtableErrorMode.FAIL.name()); + BigtableSinkTaskConfig config = new BigtableSinkTaskConfig(props); + + doReturn(null).when(context).errantRecordReporter(); + task = new TestBigtableSinkTask(config, null, null, null, null, null, context); + SinkRecord record = new SinkRecord(null, 1, null, "key", null, null, 1); + Throwable t = new Exception("testErrorReporterNoDLQFailMode"); + verifyNoMoreInteractions(errorReporter); + verifyNoMoreInteractions(task.getLogger()); + assertThrows(ConnectException.class, () -> task.reportError(record, t)); + } + + @Test + public void testGetTimestamp() { + task = new TestBigtableSinkTask(null, null, null, null, null, null, null); + long timestampMillis = 123L; + SinkRecord recordWithTimestamp = + new SinkRecord( + null, 1, null, null, null, null, 1, timestampMillis, TimestampType.CREATE_TIME); + SinkRecord recordWithNullTimestamp = new SinkRecord(null, 1, null, null, null, null, 2); + + assertEquals( + (Long) (1000L * timestampMillis), (Long) task.getTimestampMicros(recordWithTimestamp)); + assertNotNull(task.getTimestampMicros(recordWithNullTimestamp)); + + // Assertion that the Java Bigtable client doesn't support microsecond timestamp granularity. + // When it starts supporting it, getTimestamp() will need to get modified. + assertEquals( + Arrays.stream(Table.TimestampGranularity.values()).collect(Collectors.toSet()), + Set.of( + Table.TimestampGranularity.TIMESTAMP_GRANULARITY_UNSPECIFIED, + Table.TimestampGranularity.MILLIS, + Table.TimestampGranularity.UNRECOGNIZED)); + } + + @Test + public void testHandleResults() { + SinkRecord errorSinkRecord = new SinkRecord("", 1, null, null, null, null, 1); + SinkRecord successSinkRecord = new SinkRecord("", 1, null, null, null, null, 2); + Map> perRecordResults = + Map.of( + errorSinkRecord, CompletableFuture.failedFuture(new Exception("testHandleResults")), + successSinkRecord, CompletableFuture.completedFuture(null)); + doReturn(errorReporter).when(context).errantRecordReporter(); + task = new TestBigtableSinkTask(null, null, null, null, null, null, context); + task.handleResults(perRecordResults); + verify(errorReporter, times(1)).report(eq(errorSinkRecord), any()); + assertTotalNumberOfInvocations(errorReporter, 1); + } + + @Test + public void testPrepareRecords() { + task = spy(new TestBigtableSinkTask(null, null, null, null, null, null, context)); + doReturn(errorReporter).when(context).errantRecordReporter(); + + MutationData okMutationData = mock(MutationData.class); + Exception exception = new RuntimeException(); + doThrow(exception) + .doReturn(Optional.empty()) + .doReturn(Optional.of(okMutationData)) + .when(task) + .createRecordMutationData(any()); + + SinkRecord exceptionRecord = new SinkRecord("", 1, null, null, null, null, 1); + SinkRecord emptyRecord = new SinkRecord("", 1, null, null, null, null, 3); + SinkRecord okRecord = new SinkRecord("", 1, null, null, null, null, 2); + + Map result = + task.prepareRecords(List.of(exceptionRecord, emptyRecord, okRecord)); + assertEquals(Map.of(okRecord, okMutationData), result); + verify(errorReporter, times(1)).report(exceptionRecord, exception); + assertTotalNumberOfInvocations(errorReporter, 1); + } + + @Test + public void testAutoCreateTablesAndHandleErrors() { + task = spy(new TestBigtableSinkTask(null, null, null, null, null, schemaManager, context)); + doReturn(errorReporter).when(context).errantRecordReporter(); + + doReturn(errorReporter).when(context).errantRecordReporter(); + SinkRecord okRecord = new SinkRecord("", 1, null, null, null, null, 1); + SinkRecord bigtableErrorRecord = new SinkRecord("", 1, null, null, null, null, 2); + SinkRecord dataErrorRecord = new SinkRecord("", 1, null, null, null, null, 3); + MutationData okMutationData = mock(MutationData.class); + MutationData bigtableErrorMutationData = mock(MutationData.class); + MutationData dataErrorMutationData = mock(MutationData.class); + + Map mutations = new HashMap<>(); + mutations.put(okRecord, okMutationData); + mutations.put(bigtableErrorRecord, bigtableErrorMutationData); + mutations.put(dataErrorRecord, dataErrorMutationData); + + ResourceCreationResult resourceCreationResult = + new ResourceCreationResult(Set.of(bigtableErrorRecord), Set.of(dataErrorRecord)); + doReturn(resourceCreationResult).when(schemaManager).ensureTablesExist(any()); + Map mutationsToApply = + task.autoCreateTablesAndHandleErrors(mutations); + + assertEquals(Map.of(okRecord, okMutationData), mutationsToApply); + verify(errorReporter, times(1)) + .report(eq(bigtableErrorRecord), argThat(e -> e instanceof ConnectException)); + verify(errorReporter, times(1)) + .report( + eq(dataErrorRecord), + argThat(e -> e instanceof InvalidBigtableSchemaModificationException)); + assertTotalNumberOfInvocations(errorReporter, 2); + } + + @Test + public void testAutoCreateColumnFamiliesAndHandleErrors() { + task = spy(new TestBigtableSinkTask(null, null, null, null, null, schemaManager, context)); + doReturn(errorReporter).when(context).errantRecordReporter(); + + doReturn(errorReporter).when(context).errantRecordReporter(); + SinkRecord okRecord = new SinkRecord("", 1, null, null, null, null, 1); + SinkRecord bigtableErrorRecord = new SinkRecord("", 1, null, null, null, null, 2); + SinkRecord dataErrorRecord = new SinkRecord("", 1, null, null, null, null, 3); + MutationData okMutationData = mock(MutationData.class); + MutationData bigtableErrorMutationData = mock(MutationData.class); + MutationData dataErrorMutationData = mock(MutationData.class); + + Map mutations = new HashMap<>(); + mutations.put(okRecord, okMutationData); + mutations.put(bigtableErrorRecord, bigtableErrorMutationData); + mutations.put(dataErrorRecord, dataErrorMutationData); + + ResourceCreationResult resourceCreationResult = + new ResourceCreationResult(Set.of(bigtableErrorRecord), Set.of(dataErrorRecord)); + doReturn(resourceCreationResult).when(schemaManager).ensureColumnFamiliesExist(any()); + Map mutationsToApply = + task.autoCreateColumnFamiliesAndHandleErrors(mutations); + + assertEquals(Map.of(okRecord, okMutationData), mutationsToApply); + verify(errorReporter, times(1)) + .report(eq(bigtableErrorRecord), argThat(e -> e instanceof ConnectException)); + verify(errorReporter, times(1)) + .report( + eq(dataErrorRecord), + argThat(e -> e instanceof InvalidBigtableSchemaModificationException)); + assertTotalNumberOfInvocations(errorReporter, 2); + } + + @Test + public void testInsertRows() throws ExecutionException, InterruptedException { + task = new TestBigtableSinkTask(null, bigtableData, null, null, null, null, null); + ApiException exception = ApiExceptionFactory.create(); + doReturn(false).doReturn(true).doThrow(exception).when(bigtableData).checkAndMutateRow(any()); + + SinkRecord successRecord = new SinkRecord("", 1, null, null, null, null, 1); + SinkRecord errorRecord = new SinkRecord("", 1, null, null, null, null, 2); + SinkRecord exceptionRecord = new SinkRecord("", 1, null, null, null, null, 3); + MutationData commonMutationData = mock(MutationData.class); + doReturn("ignored").when(commonMutationData).getTargetTable(); + doReturn(ByteString.copyFrom("ignored".getBytes(StandardCharsets.UTF_8))) + .when(commonMutationData) + .getRowKey(); + doReturn(mock(Mutation.class)).when(commonMutationData).getInsertMutation(); + + // LinkedHashMap, because we mock consecutive return values of Bigtable client mock and thus + // rely on the order. + Map input = new LinkedHashMap<>(); + input.put(successRecord, commonMutationData); + input.put(errorRecord, commonMutationData); + input.put(exceptionRecord, commonMutationData); + Map> output = new HashMap<>(); + task.insertRows(input, output); + + assertEquals(input.keySet(), output.keySet()); + verify(bigtableData, times(input.size())).checkAndMutateRow(any()); + assertTotalNumberOfInvocations(bigtableData, input.size()); + + output.get(successRecord).get(); + assertThrows(ExecutionException.class, () -> output.get(errorRecord).get()); + assertThrows(ExecutionException.class, () -> output.get(exceptionRecord).get()); + } + + @Test + public void testUpsertRows() { + Map props = BasicPropertiesFactory.getTaskProps(); + int maxBatchSize = 3; + int totalRecords = 1000; + props.put(BigtableSinkTaskConfig.CONFIG_MAX_BATCH_SIZE, Integer.toString(maxBatchSize)); + BigtableSinkTaskConfig config = new BigtableSinkTaskConfig(props); + + task = spy(new TestBigtableSinkTask(config, null, null, null, null, null, null)); + String batcherTable = "batcherTable"; + Batcher batcher = mock(Batcher.class); + doAnswer( + invocation -> { + TestBigtableSinkTask task = (TestBigtableSinkTask) invocation.getMock(); + task.getBatchers().computeIfAbsent(batcherTable, ignored -> batcher); + return null; + }) + .when(task) + .performUpsertBatch(any(), any()); + + MutationData commonMutationData = mock(MutationData.class); + + Map input = + IntStream.range(0, totalRecords) + .mapToObj(i -> new SinkRecord("", 1, null, null, null, null, i)) + .collect(Collectors.toMap(i -> i, ignored -> commonMutationData)); + + Map> fakeMutationData = mock(Map.class); + assertTrue(task.getBatchers().isEmpty()); + task.upsertRows(input, fakeMutationData); + assertEquals(Set.of(batcher), task.getBatchers().values().stream().collect(Collectors.toSet())); + + int expectedFullBatches = totalRecords / maxBatchSize; + int expectedPartialBatches = totalRecords % maxBatchSize == 0 ? 0 : 1; + + verify(task, times(expectedFullBatches)) + .performUpsertBatch(argThat(v -> v.size() == maxBatchSize), any()); + verify(task, times(expectedPartialBatches)) + .performUpsertBatch(argThat(v -> v.size() != maxBatchSize), any()); + } + + @Test + public void testPerformUpsertBatch() throws ExecutionException, InterruptedException { + String okTable = "okTable"; + String errorTable = "errorTable"; + + Batcher okBatcher = mock(Batcher.class); + doReturn(completedApiFuture(null)).when(okBatcher).add(any()); + Batcher errorBatcher = mock(Batcher.class); + doReturn(FutureUtil.failedApiFuture(new Exception())).when(errorBatcher).add(any()); + + doReturn(okBatcher).when(bigtableData).newBulkMutationBatcher(okTable); + doReturn(errorBatcher).when(bigtableData).newBulkMutationBatcher(errorTable); + task = new TestBigtableSinkTask(null, bigtableData, null, null, null, null, null); + + SinkRecord okRecord = new SinkRecord(okTable, 1, null, null, null, null, 1); + SinkRecord errorRecord = new SinkRecord(errorTable, 1, null, null, null, null, 2); + + MutationData okMutationData = mock(MutationData.class); + doReturn(okTable).when(okMutationData).getTargetTable(); + doReturn(mock(RowMutationEntry.class)).when(okMutationData).getUpsertMutation(); + MutationData errorMutationData = mock(MutationData.class); + doReturn(errorTable).when(errorMutationData).getTargetTable(); + doReturn(mock(RowMutationEntry.class)).when(errorMutationData).getUpsertMutation(); + + Map input = + Map.of( + okRecord, okMutationData, + errorRecord, errorMutationData); + Map> output = new HashMap<>(); + + assertTrue(task.getBatchers().isEmpty()); + task.performUpsertBatch(new ArrayList<>(input.entrySet()), output); + assertEquals( + Set.of(okBatcher, errorBatcher), + task.getBatchers().values().stream().collect(Collectors.toSet())); + + assertEquals(input.keySet(), output.keySet()); + verify(okBatcher, times(1)).add(any()); + verify(okBatcher, times(1)).sendOutstanding(); + assertTotalNumberOfInvocations(okBatcher, 2); + verify(errorBatcher, times(1)).add(any()); + verify(errorBatcher, times(1)).sendOutstanding(); + assertTotalNumberOfInvocations(errorBatcher, 2); + + output.get(okRecord).get(); + assertThrows(ExecutionException.class, () -> output.get(errorRecord).get()); + } + + @Test + public void testPutBranches() { + SinkRecord record1 = new SinkRecord("table1", 1, null, null, null, null, 1); + SinkRecord record2 = new SinkRecord("table2", 1, null, null, null, null, 2); + + for (List test : + List.of( + List.of(false, false, false), + List.of(false, false, true), + List.of(false, true, false), + List.of(false, true, true), + List.of(true, false, false), + List.of(true, false, true), + List.of(true, true, false), + List.of(true, true, true))) { + boolean autoCreateTables = test.get(0); + boolean autoCreateColumnFamilies = test.get(1); + boolean useInsertMode = test.get(2); + + Map props = BasicPropertiesFactory.getTaskProps(); + props.put(CONFIG_AUTO_CREATE_TABLES, Boolean.toString(autoCreateTables)); + props.put(CONFIG_AUTO_CREATE_COLUMN_FAMILIES, Boolean.toString(autoCreateColumnFamilies)); + props.put(CONFIG_INSERT_MODE, (useInsertMode ? InsertMode.INSERT : InsertMode.UPSERT).name()); + config = new BigtableSinkTaskConfig(props); + + byte[] rowKey = "rowKey".getBytes(StandardCharsets.UTF_8); + doReturn(rowKey).when(keyMapper).getKey(any()); + doAnswer( + i -> { + MutationDataBuilder builder = new MutationDataBuilder(); + builder.deleteRow(); + return builder; + }) + .when(valueMapper) + .getRecordMutationDataBuilder(any(), anyLong()); + + Batcher batcher = mock(Batcher.class); + doReturn(completedApiFuture(null)).when(batcher).add(any()); + doReturn(batcher).when(bigtableData).newBulkMutationBatcher(anyString()); + doReturn(new ResourceCreationResult(Collections.emptySet(), Collections.emptySet())) + .when(schemaManager) + .ensureTablesExist(any()); + doReturn(new ResourceCreationResult(Collections.emptySet(), Collections.emptySet())) + .when(schemaManager) + .ensureColumnFamiliesExist(any()); + + task = + spy( + new TestBigtableSinkTask( + config, bigtableData, null, keyMapper, valueMapper, schemaManager, null)); + + task.put(List.of(record1, record2)); + + verify(task, times(1)).prepareRecords(any()); + verify(schemaManager, times(autoCreateTables ? 1 : 0)).ensureTablesExist(any()); + verify(schemaManager, times(autoCreateColumnFamilies ? 1 : 0)) + .ensureColumnFamiliesExist(any()); + verify(task, times(useInsertMode ? 1 : 0)).insertRows(any(), any()); + verify(task, times(useInsertMode ? 0 : 1)).upsertRows(any(), any()); + verify(task, times(1)).handleResults(any()); + + reset(task); + reset(schemaManager); + } + } + + private static class TestBigtableSinkTask extends BigtableSinkTask { + public TestBigtableSinkTask( + BigtableSinkTaskConfig config, + BigtableDataClient bigtableData, + BigtableTableAdminClient bigtableAdmin, + KeyMapper keyMapper, + ValueMapper valueMapper, + BigtableSchemaManager schemaManager, + SinkTaskContext context) { + super(config, bigtableData, bigtableAdmin, keyMapper, valueMapper, schemaManager, context); + this.logger = mock(Logger.class); + } + + public Logger getLogger() { + return logger; + } + + public Map> getBatchers() { + return batchers; + } + } +} diff --git a/google-cloud-bigtable-kafka-connect-sink/src/test/java/com/google/cloud/kafka/connect/bigtable/autocreate/BigtableSchemaManagerTest.java b/google-cloud-bigtable-kafka-connect-sink/src/test/java/com/google/cloud/kafka/connect/bigtable/autocreate/BigtableSchemaManagerTest.java new file mode 100644 index 0000000000..c8b6eaf45d --- /dev/null +++ b/google-cloud-bigtable-kafka-connect-sink/src/test/java/com/google/cloud/kafka/connect/bigtable/autocreate/BigtableSchemaManagerTest.java @@ -0,0 +1,749 @@ +/* + * Copyright 2024 Google LLC + * + * Licensed under the Apache License, Version 2.0 (the "License"); + * you may not use this file except in compliance with the License. + * You may obtain a copy of the License at + * + * https://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ +package com.google.cloud.kafka.connect.bigtable.autocreate; + +import static com.google.cloud.kafka.connect.bigtable.util.FutureUtil.completedApiFuture; +import static com.google.cloud.kafka.connect.bigtable.util.FutureUtil.failedApiFuture; +import static com.google.cloud.kafka.connect.bigtable.util.MockUtil.assertTotalNumberOfInvocations; +import static org.junit.Assert.assertEquals; +import static org.junit.Assert.assertFalse; +import static org.junit.Assert.assertThrows; +import static org.junit.Assert.assertTrue; +import static org.mockito.ArgumentMatchers.any; +import static org.mockito.ArgumentMatchers.eq; +import static org.mockito.Mockito.argThat; +import static org.mockito.Mockito.doAnswer; +import static org.mockito.Mockito.doReturn; +import static org.mockito.Mockito.doThrow; +import static org.mockito.Mockito.mock; +import static org.mockito.Mockito.reset; +import static org.mockito.Mockito.spy; +import static org.mockito.Mockito.times; +import static org.mockito.Mockito.verify; +import static org.mockito.Mockito.verifyNoInteractions; + +import com.google.api.core.ApiFuture; +import com.google.cloud.bigtable.admin.v2.BigtableTableAdminClient; +import com.google.cloud.bigtable.admin.v2.models.ColumnFamily; +import com.google.cloud.bigtable.admin.v2.models.CreateTableRequest; +import com.google.cloud.bigtable.admin.v2.models.ModifyColumnFamiliesRequest; +import com.google.cloud.bigtable.admin.v2.models.Table; +import com.google.cloud.kafka.connect.bigtable.autocreate.BigtableSchemaManager.ResourceAndRecords; +import com.google.cloud.kafka.connect.bigtable.mapping.MutationData; +import com.google.cloud.kafka.connect.bigtable.util.ApiExceptionFactory; +import io.grpc.Status; +import java.util.AbstractMap; +import java.util.ArrayList; +import java.util.Collections; +import java.util.HashMap; +import java.util.HashSet; +import java.util.List; +import java.util.Map; +import java.util.Optional; +import java.util.Set; +import java.util.stream.Collectors; +import org.apache.kafka.connect.errors.ConnectException; +import org.apache.kafka.connect.sink.SinkRecord; +import org.junit.Before; +import org.junit.Test; +import org.junit.runner.RunWith; +import org.junit.runners.JUnit4; + +@RunWith(JUnit4.class) +public class BigtableSchemaManagerTest { + BigtableTableAdminClient bigtable; + TestBigtableSchemaManager bigtableSchemaManager; + + @Before + public void setUp() { + bigtable = mock(BigtableTableAdminClient.class); + bigtableSchemaManager = spy(new TestBigtableSchemaManager(bigtable)); + } + + @Test + public void testTableCachePopulationSuccess() { + List tables = List.of("table1", "table2"); + doReturn(tables).when(bigtable).listTables(); + bigtableSchemaManager.refreshTableNamesCache(); + assertEquals(new HashSet<>(tables), bigtableSchemaManager.getCache().keySet()); + assertTotalNumberOfInvocations(bigtable, 1); + + reset(bigtable); + verifyNoInteractions(bigtable); + Map input = + generateInput( + tables.stream() + .map(l -> new AbstractMap.SimpleImmutableEntry<>(l, Set.of("cf"))) + .collect(Collectors.toList())); + ResourceCreationResult result = bigtableSchemaManager.ensureTablesExist(input); + assertTrue(result.getBigtableErrors().isEmpty()); + assertTrue(result.getDataErrors().isEmpty()); + } + + @Test + public void testTableCachePopulationMayRemoveElements() { + List tables1 = List.of("table1", "table2"); + List tables2 = List.of(tables1.get(0)); + + doReturn(tables1).when(bigtable).listTables(); + bigtableSchemaManager.refreshTableNamesCache(); + assertEquals(new HashSet<>(tables1), bigtableSchemaManager.getCache().keySet()); + reset(bigtable); + + doReturn(tables2).when(bigtable).listTables(); + bigtableSchemaManager.refreshTableNamesCache(); + assertEquals(new HashSet<>(tables2), bigtableSchemaManager.getCache().keySet()); + verify(bigtable, times(1)).listTables(); + assertTotalNumberOfInvocations(bigtable, 1); + } + + @Test + public void testTableCachePopulationError() { + doThrow(ApiExceptionFactory.create()).when(bigtable).listTables(); + assertThrows(ConnectException.class, () -> bigtableSchemaManager.refreshTableNamesCache()); + } + + @Test + public void testTableColumnFamiliesCachePopulationSuccess() { + Map> tablesAndColumnFamilies = + Map.of( + "table1", Set.of("cf1", "cf2"), + "table2", Set.of("cf3", "cf4")); + doReturn(new ArrayList<>(tablesAndColumnFamilies.keySet())).when(bigtable).listTables(); + for (Map.Entry> entry : tablesAndColumnFamilies.entrySet()) { + mockGetTableSuccess(bigtable, entry.getKey(), entry.getValue()); + } + + Set refreshedTables = tablesAndColumnFamilies.keySet(); + bigtableSchemaManager.refreshTableColumnFamiliesCache(refreshedTables); + verify(bigtableSchemaManager, times(1)).refreshTableColumnFamiliesCache(refreshedTables); + verify(bigtableSchemaManager, times(1)).refreshTableNamesCache(); + assertTotalNumberOfInvocations(bigtableSchemaManager, 2); + assertEquals( + bigtableSchemaManager.getCache(), + tablesAndColumnFamilies.entrySet().stream() + .map(e -> new AbstractMap.SimpleImmutableEntry<>(e.getKey(), Optional.of(e.getValue()))) + .collect(Collectors.toMap(Map.Entry::getKey, Map.Entry::getValue))); + verify(bigtable, times(1)).listTables(); + for (String tableName : tablesAndColumnFamilies.keySet()) { + verify(bigtable, times(1)).getTableAsync(tableName); + } + // One for listTables() and one for each table lookup. + int expectedBigtableCalls = tablesAndColumnFamilies.size() + 1; + assertTotalNumberOfInvocations(bigtable, expectedBigtableCalls); + + reset(bigtable); + verifyNoInteractions(bigtable); + Map input = + generateInput(new ArrayList<>(tablesAndColumnFamilies.entrySet())); + ResourceCreationResult result = bigtableSchemaManager.ensureTablesExist(input); + assertTrue(result.getBigtableErrors().isEmpty()); + assertTrue(result.getDataErrors().isEmpty()); + } + + @Test + public void testTableColumnFamiliesCachePopulationErrors() { + doThrow(ApiExceptionFactory.create()).when(bigtable).listTables(); + assertThrows( + ConnectException.class, + () -> bigtableSchemaManager.refreshTableColumnFamiliesCache(Collections.emptySet())); + verify(bigtable, times(1)).listTables(); + reset(bigtable); + + String successTable = "table1"; + String errorTable = "table2"; + List allTables = List.of(successTable, errorTable); + + doReturn(allTables).when(bigtable).listTables(); + mockGetTableSuccess(bigtable, successTable, Collections.emptySet()); + // We simulate an error due to e.g., deletion of the table by another user. + doReturn(failedApiFuture(ApiExceptionFactory.create())) + .when(bigtable) + .getTableAsync(errorTable); + bigtableSchemaManager.refreshTableColumnFamiliesCache(new HashSet<>(allTables)); + assertEquals(Set.of(successTable), bigtableSchemaManager.getCache().keySet()); + verify(bigtable, times(1)).listTables(); + verify(bigtable, times(1)).getTableAsync(successTable); + verify(bigtable, times(1)).getTableAsync(errorTable); + assertTotalNumberOfInvocations(bigtable, 3); + } + + @Test + public void testEnsureTablesExistAllExisted() { + // Prepopulate the cache. + List tables = List.of("table1", "table2"); + doReturn(tables).when(bigtable).listTables(); + bigtableSchemaManager.refreshTableNamesCache(); + reset(bigtable); + + Map ensureTablesExistInput = + generateInput( + List.of( + new AbstractMap.SimpleImmutableEntry<>(tables.get(0), Set.of("missingCF")), + new AbstractMap.SimpleImmutableEntry<>(tables.get(0), Set.of("missingCF")), + new AbstractMap.SimpleImmutableEntry<>(tables.get(1), Set.of("missingCF")), + new AbstractMap.SimpleImmutableEntry<>(tables.get(1), Set.of("missingCF")))); + ResourceCreationResult result = bigtableSchemaManager.ensureTablesExist(ensureTablesExistInput); + assertTrue(result.getBigtableErrors().isEmpty()); + assertTrue(result.getDataErrors().isEmpty()); + assertTotalNumberOfInvocations(bigtable, 0); + } + + @Test + public void testEnsureTablesExistAllCreatedSuccessfully() { + List tables = List.of("table1", "table2"); + // We call listTables() only once, after sending all the create requests. In this case all the + // requests were successful. + doReturn(tables).when(bigtable).listTables(); + for (String table : tables) { + mockCreateTableSuccess(bigtable, table, Collections.emptySet()); + } + + assertTrue(bigtableSchemaManager.getCache().isEmpty()); + Map ensureTablesExistInput = + generateInput( + List.of( + new AbstractMap.SimpleImmutableEntry<>(tables.get(0), Set.of("missingCF")), + new AbstractMap.SimpleImmutableEntry<>(tables.get(0), Set.of("missingCF")), + new AbstractMap.SimpleImmutableEntry<>(tables.get(1), Set.of("missingCF")), + new AbstractMap.SimpleImmutableEntry<>(tables.get(1), Set.of("missingCF")))); + ResourceCreationResult result = bigtableSchemaManager.ensureTablesExist(ensureTablesExistInput); + assertTrue(result.getBigtableErrors().isEmpty()); + assertTrue(result.getDataErrors().isEmpty()); + for (String table : tables) { + assertTrue(bigtableSchemaManager.getCache().containsKey(table)); + verify(bigtable, times(1)) + .createTableAsync(argThat(ctr -> createTableMockRefersTable(table, ctr))); + } + + // One for each table creation and one for result check. + int expectedBigtableCalls = tables.size() + 1; + assertTotalNumberOfInvocations(bigtable, expectedBigtableCalls); + } + + @Test + public void testEnsureTablesExistSomeCreatedSuccessfullySomeErrorsDueToRaces() { + List tables = List.of("table1", "table2"); + // We call listTables() only once, after sending all the create requests. In this case some + // requests failed since another thread concurrently created one of these tables. + doReturn(tables).when(bigtable).listTables(); + String tableWhoseCreationFailed = tables.get(1); + for (String table : tables) { + if (!table.equals(tableWhoseCreationFailed)) { + mockCreateTableSuccess(bigtable, table, Collections.emptySet()); + } + } + doReturn(failedApiFuture(ApiExceptionFactory.create())) + .when(bigtable) + .createTableAsync( + argThat(ctr -> createTableMockRefersTable(tableWhoseCreationFailed, ctr))); + + assertTrue(bigtableSchemaManager.getCache().isEmpty()); + Map ensureTablesExistInput = + generateInput( + List.of( + new AbstractMap.SimpleImmutableEntry<>(tables.get(0), Set.of("missingCF")), + new AbstractMap.SimpleImmutableEntry<>(tables.get(0), Set.of("missingCF")), + new AbstractMap.SimpleImmutableEntry<>(tables.get(1), Set.of("missingCF")), + new AbstractMap.SimpleImmutableEntry<>(tables.get(1), Set.of("missingCF")))); + ResourceCreationResult result = bigtableSchemaManager.ensureTablesExist(ensureTablesExistInput); + assertTrue(result.getBigtableErrors().isEmpty()); + assertTrue(result.getDataErrors().isEmpty()); + for (String table : tables) { + assertTrue(bigtableSchemaManager.getCache().containsKey(table)); + verify(bigtable, times(1)) + .createTableAsync(argThat(ctr -> createTableMockRefersTable(table, ctr))); + } + + // One for each table creation and one for result check. + int expectedBigtableCalls = tables.size() + 1; + assertTotalNumberOfInvocations(bigtable, expectedBigtableCalls); + } + + @Test + public void testEnsureTablesExistSomeCreatedSuccessfullySomeErrors() { + String successfulTable = "table1"; + String bigtableErrorTable = "table2"; + String dataErrorTable = "table3"; + Set columnFamilies = Set.of("cf1"); + + doReturn(List.of(successfulTable)).when(bigtable).listTables(); + mockCreateTableSuccess(bigtable, successfulTable, columnFamilies); + doReturn(failedApiFuture(ApiExceptionFactory.create(Status.Code.RESOURCE_EXHAUSTED))) + .when(bigtable) + .createTableAsync(argThat(ctr -> createTableMockRefersTable(bigtableErrorTable, ctr))); + doReturn(failedApiFuture(ApiExceptionFactory.create(Status.Code.INVALID_ARGUMENT))) + .when(bigtable) + .createTableAsync(argThat(ctr -> createTableMockRefersTable(dataErrorTable, ctr))); + + assertTrue(bigtableSchemaManager.getCache().isEmpty()); + Map ensureTablesExistInput = + generateInput( + List.of( + new AbstractMap.SimpleImmutableEntry<>(successfulTable, columnFamilies), + new AbstractMap.SimpleImmutableEntry<>(successfulTable, columnFamilies), + new AbstractMap.SimpleImmutableEntry<>(bigtableErrorTable, columnFamilies), + new AbstractMap.SimpleImmutableEntry<>(bigtableErrorTable, columnFamilies), + new AbstractMap.SimpleImmutableEntry<>(dataErrorTable, columnFamilies), + new AbstractMap.SimpleImmutableEntry<>(dataErrorTable, columnFamilies))); + ResourceCreationResult result = bigtableSchemaManager.ensureTablesExist(ensureTablesExistInput); + Set bigtableErrors = result.getBigtableErrors(); + Set dataErrors = result.getDataErrors(); + assertEquals( + ensureTablesExistInput.entrySet().stream() + .filter(e -> e.getValue().getTargetTable().equals(bigtableErrorTable)) + .map(Map.Entry::getKey) + .collect(Collectors.toSet()), + bigtableErrors); + assertEquals( + ensureTablesExistInput.entrySet().stream() + .filter(e -> e.getValue().getTargetTable().equals(dataErrorTable)) + .map(Map.Entry::getKey) + .collect(Collectors.toSet()), + dataErrors); + Map>> cache = bigtableSchemaManager.getCache(); + assertTrue(cache.containsKey(successfulTable)); + verify(bigtable, times(1)) + .createTableAsync(argThat(ctr -> createTableMockRefersTable(successfulTable, ctr))); + assertFalse(cache.containsKey(bigtableErrorTable)); + verify(bigtable, times(1)) + .createTableAsync(argThat(ctr -> createTableMockRefersTable(bigtableErrorTable, ctr))); + assertFalse(cache.containsKey(dataErrorTable)); + verify(bigtable, times(1)) + .createTableAsync(argThat(ctr -> createTableMockRefersTable(dataErrorTable, ctr))); + + // One for each table creation and one for result check. + int expectedBigtableCalls = 4; + assertTotalNumberOfInvocations(bigtable, expectedBigtableCalls); + } + + @Test + public void testEnsureTablesExistConcurrentDeletion() { + String createdTable = "table1"; + String createdAndThenConcurrentlyDeletedTable = "table2"; + Set columnFamilies = Set.of("cf1"); + + // Note that only a single table is returned - we simulate concurrent deletion of the other + // table. + doAnswer(ignored -> List.of(createdTable)).when(bigtable).listTables(); + mockCreateTableSuccess(bigtable, createdTable, columnFamilies); + mockCreateTableSuccess(bigtable, createdAndThenConcurrentlyDeletedTable, columnFamilies); + + assertTrue(bigtableSchemaManager.getCache().isEmpty()); + Map ensureTablesExistInput = + generateInput( + List.of( + new AbstractMap.SimpleImmutableEntry<>(createdTable, columnFamilies), + new AbstractMap.SimpleImmutableEntry<>(createdTable, columnFamilies), + new AbstractMap.SimpleImmutableEntry<>( + createdAndThenConcurrentlyDeletedTable, columnFamilies), + new AbstractMap.SimpleImmutableEntry<>( + createdAndThenConcurrentlyDeletedTable, columnFamilies))); + ResourceCreationResult result = bigtableSchemaManager.ensureTablesExist(ensureTablesExistInput); + assertTrue(result.getDataErrors().isEmpty()); + Set missingTables = result.getBigtableErrors(); + assertEquals( + ensureTablesExistInput.entrySet().stream() + .filter( + e -> e.getValue().getTargetTable().equals(createdAndThenConcurrentlyDeletedTable)) + .map(Map.Entry::getKey) + .collect(Collectors.toSet()), + missingTables); + Map>> cache = bigtableSchemaManager.getCache(); + assertTrue(cache.containsKey(createdTable)); + verify(bigtable, times(1)) + .createTableAsync(argThat(ctr -> createTableMockRefersTable(createdTable, ctr))); + assertFalse(cache.containsKey(createdAndThenConcurrentlyDeletedTable)); + verify(bigtable, times(1)) + .createTableAsync( + argThat( + ctr -> createTableMockRefersTable(createdAndThenConcurrentlyDeletedTable, ctr))); + + // One for each table creation and one for result check. + int expectedBigtableCalls = 3; + assertTotalNumberOfInvocations(bigtable, expectedBigtableCalls); + } + + @Test + public void testEnsureColumnFamiliesExistAllExisted() { + Map> tablesAndColumnFamilies = + Map.of( + "table1", Set.of("cf1", "cf2"), + "table2", Set.of("cf3", "cf4")); + doReturn(new ArrayList<>(tablesAndColumnFamilies.keySet())).when(bigtable).listTables(); + for (Map.Entry> entry : tablesAndColumnFamilies.entrySet()) { + mockGetTableSuccess(bigtable, entry.getKey(), entry.getValue()); + } + Set refreshedTables = tablesAndColumnFamilies.keySet(); + bigtableSchemaManager.refreshTableColumnFamiliesCache(refreshedTables); + reset(bigtable); + verifyNoInteractions(bigtable); + + Map ensureColumnFamiliesExistInput = + generateInput(new ArrayList<>(tablesAndColumnFamilies.entrySet())); + ResourceCreationResult result = + bigtableSchemaManager.ensureColumnFamiliesExist(ensureColumnFamiliesExistInput); + assertTrue(result.getDataErrors().isEmpty()); + assertTrue(result.getBigtableErrors().isEmpty()); + } + + @Test + public void testEnsureColumnFamiliesExistAllCreatedSuccessfully() { + Map> tablesAndColumnFamilies = + Map.of( + "table1", Set.of("cf1", "cf2"), + "table2", Set.of("cf3", "cf4")); + doReturn(new ArrayList<>(tablesAndColumnFamilies.keySet())).when(bigtable).listTables(); + for (Map.Entry> entry : tablesAndColumnFamilies.entrySet()) { + mockCreateColumnFamilySuccess(bigtable, entry.getKey(), entry.getValue()); + mockGetTableSuccess(bigtable, entry.getKey(), entry.getValue()); + } + Map ensureColumnFamiliesExistInput = + generateInput(new ArrayList<>(tablesAndColumnFamilies.entrySet())); + + ResourceCreationResult result = + bigtableSchemaManager.ensureColumnFamiliesExist(ensureColumnFamiliesExistInput); + assertTrue(result.getDataErrors().isEmpty()); + assertTrue(result.getBigtableErrors().isEmpty()); + for (Map.Entry> entry : tablesAndColumnFamilies.entrySet()) { + String tableName = entry.getKey(); + for (String columnFamily : entry.getValue()) { + verify(bigtable, times(1)) + .modifyFamiliesAsync( + argThat( + mcfr -> + createColumnFamilyMockRefersTableAndColumnFamily( + tableName, columnFamily, mcfr))); + } + verify(bigtable, times(1)).getTableAsync(tableName); + } + int expectedBigtableInteractions = + 1 // listTables() + + tablesAndColumnFamilies.values().stream() + .mapToInt(Set::size) + .sum() // modifyColumnFamily() + + tablesAndColumnFamilies.keySet().size(); // getTable() + assertTotalNumberOfInvocations(bigtable, expectedBigtableInteractions); + } + + @Test + public void + testEnsureColumnFamiliesExistSomeCreatedSuccessfullySomeErrorsDueToRacesOrInvalidRequests() { + String successTable = "table1"; + String bigtableErrorTable = "table2"; + String dataErrorTable = "table3"; + String invalidArgumentColumnFamilyName = "INVALID_ARGUMENT_COLUMN_FAMILY_NAME"; + Map> tablesAndColumnFamilies = + Map.of( + successTable, Set.of("cf1", "cf2"), + bigtableErrorTable, Set.of("cf3", "cf4"), + dataErrorTable, Set.of("cf5", invalidArgumentColumnFamilyName)); + doReturn(new ArrayList<>(tablesAndColumnFamilies.keySet())).when(bigtable).listTables(); + for (Map.Entry> entry : tablesAndColumnFamilies.entrySet()) { + String table = entry.getKey(); + for (String columnFamily : entry.getValue()) { + if (table.equals(bigtableErrorTable)) { + doReturn(failedApiFuture(ApiExceptionFactory.create(Status.Code.RESOURCE_EXHAUSTED))) + .when(bigtable) + .modifyFamiliesAsync( + argThat( + mcfr -> + createColumnFamilyMockRefersTableAndColumnFamily( + table, columnFamily, mcfr))); + } else if (table.equals(dataErrorTable)) { + doReturn(failedApiFuture(ApiExceptionFactory.create(Status.Code.INVALID_ARGUMENT))) + .when(bigtable) + .modifyFamiliesAsync( + argThat( + mcfr -> + createColumnFamilyMockRefersTableAndColumnFamily( + table, columnFamily, mcfr))); + } else { + mockCreateColumnFamilySuccess(bigtable, entry.getKey(), entry.getValue()); + } + } + Set columnFamilies = new HashSet<>(entry.getValue()); + columnFamilies.remove(invalidArgumentColumnFamilyName); + mockGetTableSuccess(bigtable, table, columnFamilies); + } + Map ensureColumnFamiliesExistInput = + generateInput(new ArrayList<>(tablesAndColumnFamilies.entrySet())); + ResourceCreationResult result = + bigtableSchemaManager.ensureColumnFamiliesExist(ensureColumnFamiliesExistInput); + assertTrue(result.getBigtableErrors().isEmpty()); + Set missingColumnFamilies = result.getDataErrors(); + assertEquals( + ensureColumnFamiliesExistInput.entrySet().stream() + .filter(e -> e.getValue().getTargetTable().equals(dataErrorTable)) + .map(Map.Entry::getKey) + .collect(Collectors.toSet()), + missingColumnFamilies); + + for (Map.Entry> entry : tablesAndColumnFamilies.entrySet()) { + String tableName = entry.getKey(); + for (String columnFamily : entry.getValue()) { + verify(bigtable, times(1)) + .modifyFamiliesAsync( + argThat( + mcfr -> + createColumnFamilyMockRefersTableAndColumnFamily( + tableName, columnFamily, mcfr))); + } + verify(bigtable, times(1)).getTableAsync(tableName); + } + int expectedBigtableInteractions = + 1 // listTables() + + tablesAndColumnFamilies.values().stream() + .mapToInt(Set::size) + .sum() // modifyColumnFamily() + + tablesAndColumnFamilies.keySet().size(); // getTable() + assertTotalNumberOfInvocations(bigtable, expectedBigtableInteractions); + } + + @Test + public void testEnsureColumnFamiliesExistSomeSomeErrorsDueToConcurrentColumnFamilyDeletion() { + String successTable = "table1"; + String errorTable = "table2"; + Map> tablesAndColumnFamilies = + Map.of( + successTable, Set.of("cf1", "cf2"), + errorTable, Set.of("cf3", "cf4")); + doReturn(new ArrayList<>(tablesAndColumnFamilies.keySet())).when(bigtable).listTables(); + for (Map.Entry> entry : tablesAndColumnFamilies.entrySet()) { + String table = entry.getKey(); + mockCreateColumnFamilySuccess(bigtable, table, entry.getValue()); + if (table.equals(errorTable)) { + doReturn(failedApiFuture(ApiExceptionFactory.create())).when(bigtable).getTableAsync(table); + } else { + mockGetTableSuccess(bigtable, table, entry.getValue()); + } + } + Map ensureColumnFamiliesExistInput = + generateInput(new ArrayList<>(tablesAndColumnFamilies.entrySet())); + ResourceCreationResult result = + bigtableSchemaManager.ensureColumnFamiliesExist(ensureColumnFamiliesExistInput); + assertTrue(result.getDataErrors().isEmpty()); + Set missingColumnFamilies = result.getBigtableErrors(); + assertEquals(1, missingColumnFamilies.size()); + assertEquals( + ensureColumnFamiliesExistInput.entrySet().stream() + .filter(e -> e.getValue().getTargetTable().equals(errorTable)) + .findFirst() + .get() + .getKey(), + missingColumnFamilies.stream().findFirst().get()); + + for (Map.Entry> entry : tablesAndColumnFamilies.entrySet()) { + String tableName = entry.getKey(); + for (String columnFamily : entry.getValue()) { + verify(bigtable, times(1)) + .modifyFamiliesAsync( + argThat( + mcfr -> + createColumnFamilyMockRefersTableAndColumnFamily( + tableName, columnFamily, mcfr))); + } + verify(bigtable, times(1)).getTableAsync(tableName); + } + int expectedBigtableInteractions = + 1 // listTables() + + tablesAndColumnFamilies.values().stream() + .mapToInt(Set::size) + .sum() // modifyColumnFamily() + + tablesAndColumnFamilies.keySet().size(); // getTable() + assertTotalNumberOfInvocations(bigtable, expectedBigtableInteractions); + } + + @Test + public void testEnsureColumnFamiliesExistMissingTable() { + String successTable = "table1"; + String errorTable = "table2"; + Map> tablesAndColumnFamilies = + Map.of( + successTable, Set.of("cf1", "cf2"), + errorTable, Set.of("cf3", "cf4")); + doReturn(List.of(successTable)).when(bigtable).listTables(); + for (Map.Entry> entry : tablesAndColumnFamilies.entrySet()) { + String table = entry.getKey(); + mockGetTableSuccess(bigtable, table, entry.getValue()); + if (table.equals(errorTable)) { + doReturn(failedApiFuture(ApiExceptionFactory.create())) + .when(bigtable) + .modifyFamiliesAsync(argThat(mcfr -> createColumnFamilyMockRefersTable(table, mcfr))); + } else { + mockCreateColumnFamilySuccess(bigtable, table, entry.getValue()); + } + } + Map ensureColumnFamiliesExistInput = + generateInput(new ArrayList<>(tablesAndColumnFamilies.entrySet())); + ResourceCreationResult result = + bigtableSchemaManager.ensureColumnFamiliesExist(ensureColumnFamiliesExistInput); + assertTrue(result.getDataErrors().isEmpty()); + Set missingColumnFamilies = result.getBigtableErrors(); + assertEquals(1, missingColumnFamilies.size()); + assertEquals( + ensureColumnFamiliesExistInput.entrySet().stream() + .filter(e -> e.getValue().getTargetTable().equals(errorTable)) + .findFirst() + .get() + .getKey(), + missingColumnFamilies.stream().findFirst().get()); + + for (Map.Entry> entry : tablesAndColumnFamilies.entrySet()) { + String table = entry.getKey(); + for (String columnFamily : entry.getValue()) { + verify(bigtable, times(1)) + .modifyFamiliesAsync( + argThat( + mcfr -> + createColumnFamilyMockRefersTableAndColumnFamily( + table, columnFamily, mcfr))); + } + if (!table.equals(errorTable)) { + verify(bigtable, times(1)).getTableAsync(table); + } + } + int expectedBigtableInteractions = + 1 // listTables() + + tablesAndColumnFamilies.values().stream() + .mapToInt(Set::size) + .sum() // modifyColumnFamily() + + 1; // getTable() + assertTotalNumberOfInvocations(bigtable, expectedBigtableInteractions); + } + + @Test + public void testErrorsCreatingColumnFamilies() {} + + @Test + public void testAwaitResourceCreationAndHandleInvalidInputErrors() { + int uniqueKafkaOffset = 0; + SinkRecord ok1 = spoofSinkRecord("topic1", uniqueKafkaOffset++); + SinkRecord ok2 = spoofSinkRecord("topic2", uniqueKafkaOffset++); + SinkRecord dataError1 = spoofSinkRecord("topic3", uniqueKafkaOffset++); + SinkRecord dataError2 = spoofSinkRecord("topic4", uniqueKafkaOffset++); + SinkRecord bigtableError1 = spoofSinkRecord("topic5", uniqueKafkaOffset++); + SinkRecord bigtableError2 = spoofSinkRecord("topic6", uniqueKafkaOffset++); + + ResourceAndRecords ok = new ResourceAndRecords("ok", List.of(ok1, ok2)); + ResourceAndRecords dataError = + new ResourceAndRecords("data", List.of(dataError1, dataError2)); + ResourceAndRecords bigtableError = + new ResourceAndRecords("bigtable", List.of(bigtableError1, bigtableError2)); + + Map, ResourceAndRecords> input = + Map.of( + completedApiFuture(null), ok, + failedApiFuture(ApiExceptionFactory.create(Status.Code.INVALID_ARGUMENT)), dataError, + failedApiFuture(ApiExceptionFactory.create(Status.Code.RESOURCE_EXHAUSTED)), + bigtableError); + + Set dataErrors = + bigtableSchemaManager.awaitResourceCreationAndHandleInvalidInputErrors(input, "%s"); + assertEquals(new HashSet<>(dataError.getRecords()), dataErrors); + verify(bigtableSchemaManager.logger, times(1)) + .info(eq(bigtableError.getResource()), any(Throwable.class)); + } + + private static Map generateInput( + List>> records) { + int uniqueKafkaOffset = 1; + Map result = new HashMap<>(); + for (Map.Entry> record : records) { + SinkRecord sinkRecord = spoofSinkRecord("topic" + record.getKey(), uniqueKafkaOffset++); + MutationData mutationData = spoofSinkRecordOutput(record.getKey(), record.getValue()); + result.put(sinkRecord, mutationData); + } + return result; + } + + private static SinkRecord spoofSinkRecord(String topic, int uniqueKafkaOffset) { + return new SinkRecord(topic, 1, null, new Object(), null, new Object(), uniqueKafkaOffset); + } + + private static MutationData spoofSinkRecordOutput( + String targetTable, Set columnFamilies) { + return new MutationData(targetTable, null, null, columnFamilies); + } + + private boolean createTableMockRefersTable(String tableName, CreateTableRequest ctr) { + return tableName.equals(ctr.toProto("unused", "unused").getTableId()); + } + + private boolean createColumnFamilyMockRefersTable( + String tableName, ModifyColumnFamiliesRequest mcfr) { + // getName() returns whole table id comprising project ID, instance ID, table name, ... + return mcfr.toProto("unused", "unused").getName().endsWith("/" + tableName); + } + + private boolean createColumnFamilyMockRefersTableAndColumnFamily( + String tableName, String columnFamily, ModifyColumnFamiliesRequest mcfr) { + boolean refersTable = createColumnFamilyMockRefersTable(tableName, mcfr); + List modifications = + mcfr.toProto("unused", "unused").getModificationsList(); + return refersTable + && modifications.stream() + .filter( + com.google.bigtable.admin.v2.ModifyColumnFamiliesRequest.Modification::hasCreate) + .anyMatch(m -> columnFamily.equals(m.getId())); + } + + private void mockCreateTableSuccess( + BigtableTableAdminClient bigtable, String tableName, Set tableColumnFamilies) { + Table table = mockTable(tableName, tableColumnFamilies); + doAnswer(ignored -> completedApiFuture(table)) + .when(bigtable) + .createTableAsync(argThat(ctr -> createTableMockRefersTable(tableName, ctr))); + } + + private void mockCreateColumnFamilySuccess( + BigtableTableAdminClient bigtable, String tableName, Set tableColumnFamilies) { + Table table = mockTable(tableName, tableColumnFamilies); + doAnswer(ignored -> completedApiFuture(table)) + .when(bigtable) + .modifyFamiliesAsync(argThat(mcfr -> createColumnFamilyMockRefersTable(tableName, mcfr))); + } + + private void mockGetTableSuccess( + BigtableTableAdminClient bigtable, String tableName, Set tableColumnFamilies) { + Table table = mockTable(tableName, tableColumnFamilies); + doAnswer(ignored -> completedApiFuture(table)).when(bigtable).getTableAsync(tableName); + } + + private Table mockTable(String tableName, Set tableColumnFamilies) { + List columnFamilies = new ArrayList<>(); + for (String tableColumnFamily : tableColumnFamilies) { + ColumnFamily columnFamily = mock(ColumnFamily.class); + doReturn(tableColumnFamily).when(columnFamily).getId(); + columnFamilies.add(columnFamily); + } + Table table = mock(Table.class); + doReturn(tableName).when(table).getId(); + doReturn(columnFamilies).when(table).getColumnFamilies(); + return table; + } + + private static class TestBigtableSchemaManager extends BigtableSchemaManager { + public TestBigtableSchemaManager(BigtableTableAdminClient bigtable) { + super(bigtable); + this.logger = spy(this.logger); + } + + public Map>> getCache() { + return tableNameToColumnFamilies; + } + } +} diff --git a/google-cloud-bigtable-kafka-connect-sink/src/test/java/com/google/cloud/kafka/connect/bigtable/autocreate/SchemaApiExceptionsTest.java b/google-cloud-bigtable-kafka-connect-sink/src/test/java/com/google/cloud/kafka/connect/bigtable/autocreate/SchemaApiExceptionsTest.java new file mode 100644 index 0000000000..b01595f185 --- /dev/null +++ b/google-cloud-bigtable-kafka-connect-sink/src/test/java/com/google/cloud/kafka/connect/bigtable/autocreate/SchemaApiExceptionsTest.java @@ -0,0 +1,143 @@ +/* + * Copyright 2025 Google LLC + * + * Licensed under the Apache License, Version 2.0 (the "License"); + * you may not use this file except in compliance with the License. + * You may obtain a copy of the License at + * + * https://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ +package com.google.cloud.kafka.connect.bigtable.autocreate; + +import static org.junit.Assert.assertEquals; +import static org.junit.Assert.assertFalse; +import static org.junit.Assert.assertTrue; + +import com.google.api.gax.grpc.GrpcStatusCode; +import com.google.api.gax.rpc.ApiException; +import com.google.api.gax.rpc.StatusCode; +import com.google.api.gax.rpc.StatusCode.Code; +import com.google.cloud.kafka.connect.bigtable.util.ApiExceptionFactory; +import io.grpc.Status; +import java.util.List; +import java.util.Map; +import java.util.concurrent.ExecutionException; +import org.junit.Test; +import org.junit.runner.RunWith; +import org.junit.runners.JUnit4; + +@RunWith(JUnit4.class) +public class SchemaApiExceptionsTest { + @Test + public void testIsStatusCodeCausedByInputError() { + for (Code causedByInputError : + List.of(Code.INVALID_ARGUMENT, Code.OUT_OF_RANGE, Code.FAILED_PRECONDITION)) { + assertTrue( + BigtableSchemaManager.SchemaApiExceptions.isStatusCodeCausedByInputError( + causedByInputError)); + } + + for (Code notCausedByInputError : + List.of(Code.NOT_FOUND, Code.RESOURCE_EXHAUSTED, Code.CANCELLED, Code.UNKNOWN)) { + assertFalse( + BigtableSchemaManager.SchemaApiExceptions.isStatusCodeCausedByInputError( + notCausedByInputError)); + } + } + + @Test + public void testExtractStatusCodeNonempty() { + StatusCode code = GrpcStatusCode.of(Status.Code.RESOURCE_EXHAUSTED); + ApiException apiException = ApiExceptionFactory.create(new Throwable(), code, true); + Throwable one = new Throwable(apiException); + Throwable two = new Throwable(one); + Throwable three = new Throwable(two); + for (Throwable t : List.of(apiException, one, two, three)) { + assertTrue( + BigtableSchemaManager.SchemaApiExceptions.maybeExtractBigtableStatusCode(t).isPresent()); + } + } + + @Test + public void testExtractGetsTheFirstStatusCode() { + StatusCode causeCode = GrpcStatusCode.of(Status.Code.RESOURCE_EXHAUSTED); + ApiException cause = ApiExceptionFactory.create(new Throwable(), causeCode, true); + StatusCode exceptionCode = GrpcStatusCode.of(Status.Code.INVALID_ARGUMENT); + ApiException exception = ApiExceptionFactory.create(cause, exceptionCode, false); + + assertEquals( + causeCode, + BigtableSchemaManager.SchemaApiExceptions.maybeExtractBigtableStatusCode(cause).get()); + assertEquals( + exceptionCode, + BigtableSchemaManager.SchemaApiExceptions.maybeExtractBigtableStatusCode(exception).get()); + } + + @Test + public void testExtractStatusCodeEmpty() { + Throwable one = new Throwable(); + Throwable two = new Throwable(one); + Throwable three = new Throwable(two); + + for (Throwable t : List.of(one, two, three)) { + assertTrue( + BigtableSchemaManager.SchemaApiExceptions.maybeExtractBigtableStatusCode(t).isEmpty()); + } + assertTrue( + BigtableSchemaManager.SchemaApiExceptions.maybeExtractBigtableStatusCode(null).isEmpty()); + } + + @Test + public void testIsCausedByInputError() { + assertFalse(BigtableSchemaManager.SchemaApiExceptions.isCausedByInputError(null)); + + ApiException inputErrorException = + ApiExceptionFactory.create( + new Throwable(), GrpcStatusCode.of(Status.Code.INVALID_ARGUMENT), false); + ApiException bigtableErrorException = + ApiExceptionFactory.create(new Throwable(), GrpcStatusCode.of(Status.Code.UNKNOWN), false); + + assertTrue(BigtableSchemaManager.SchemaApiExceptions.isCausedByInputError(inputErrorException)); + assertTrue( + BigtableSchemaManager.SchemaApiExceptions.isCausedByInputError( + new ExecutionException(inputErrorException))); + assertTrue( + BigtableSchemaManager.SchemaApiExceptions.isCausedByInputError( + new ExecutionException(new Throwable(inputErrorException)))); + + assertFalse( + BigtableSchemaManager.SchemaApiExceptions.isCausedByInputError(bigtableErrorException)); + assertFalse( + BigtableSchemaManager.SchemaApiExceptions.isCausedByInputError( + new ExecutionException(bigtableErrorException))); + assertFalse( + BigtableSchemaManager.SchemaApiExceptions.isCausedByInputError( + new ExecutionException(new Throwable(bigtableErrorException)))); + } + + @Test + public void testIsCausedByInputErrorIgnoresRetriableField() { + for (Map.Entry testCase : + Map.of( + Status.Code.INVALID_ARGUMENT, true, + Status.Code.RESOURCE_EXHAUSTED, false) + .entrySet()) { + Status.Code code = testCase.getKey(); + Boolean expectedResult = testCase.getValue(); + + for (Boolean retryableField : List.of(true, false)) { + ApiException e = + ApiExceptionFactory.create(new Throwable(), GrpcStatusCode.of(code), retryableField); + assertEquals( + expectedResult, BigtableSchemaManager.SchemaApiExceptions.isCausedByInputError(e)); + } + } + ; + } +} diff --git a/google-cloud-bigtable-kafka-connect-sink/src/test/java/com/google/cloud/kafka/connect/bigtable/config/BigtableSinkConfigTest.java b/google-cloud-bigtable-kafka-connect-sink/src/test/java/com/google/cloud/kafka/connect/bigtable/config/BigtableSinkConfigTest.java new file mode 100644 index 0000000000..a2c4629248 --- /dev/null +++ b/google-cloud-bigtable-kafka-connect-sink/src/test/java/com/google/cloud/kafka/connect/bigtable/config/BigtableSinkConfigTest.java @@ -0,0 +1,157 @@ +/* + * Copyright 2024 Google LLC + * + * Licensed under the Apache License, Version 2.0 (the "License"); + * you may not use this file except in compliance with the License. + * You may obtain a copy of the License at + * + * https://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ +package com.google.cloud.kafka.connect.bigtable.config; + +import static com.google.cloud.kafka.connect.bigtable.config.BigtableSinkConfig.CONFIG_AUTO_CREATE_COLUMN_FAMILIES; +import static com.google.cloud.kafka.connect.bigtable.config.BigtableSinkConfig.CONFIG_AUTO_CREATE_TABLES; +import static com.google.cloud.kafka.connect.bigtable.config.BigtableSinkConfig.CONFIG_ERROR_MODE; +import static com.google.cloud.kafka.connect.bigtable.config.BigtableSinkConfig.CONFIG_INSERT_MODE; +import static com.google.cloud.kafka.connect.bigtable.config.BigtableSinkConfig.CONFIG_MAX_BATCH_SIZE; +import static com.google.cloud.kafka.connect.bigtable.config.BigtableSinkConfig.CONFIG_TABLE_NAME_FORMAT; +import static com.google.cloud.kafka.connect.bigtable.config.BigtableSinkConfig.CONFIG_VALUE_NULL_MODE; +import static java.util.Collections.emptyList; +import static org.junit.Assert.assertEquals; +import static org.junit.Assert.assertFalse; +import static org.junit.Assert.assertThrows; +import static org.junit.Assert.assertTrue; +import static org.mockito.Mockito.any; +import static org.mockito.Mockito.doReturn; +import static org.mockito.Mockito.doThrow; +import static org.mockito.Mockito.mock; +import static org.mockito.Mockito.spy; +import static org.mockito.Mockito.times; +import static org.mockito.Mockito.verify; + +import com.google.cloud.bigtable.admin.v2.BigtableTableAdminClient; +import com.google.cloud.kafka.connect.bigtable.util.BasicPropertiesFactory; +import java.util.HashMap; +import java.util.List; +import java.util.Map; +import org.apache.kafka.common.config.ConfigException; +import org.junit.Test; +import org.junit.runner.RunWith; +import org.junit.runners.JUnit4; + +@RunWith(JUnit4.class) +public class BigtableSinkConfigTest { + public static boolean configIsValid(BigtableSinkConfig config) { + return BigtableSinkConfig.validate(config.originalsStrings(), false).configValues().stream() + .allMatch(v -> v.errorMessages().isEmpty()); + } + + @Test + public void testBasicSuccess() { + BigtableSinkConfig config = new BigtableSinkConfig(BasicPropertiesFactory.getSinkProps()); + assertTrue(configIsValid(config)); + } + + @Test + public void testBasicValidationFailure() { + assertThrows(ConfigException.class, () -> new BigtableSinkConfig(new HashMap<>())); + for (String configName : + List.of( + CONFIG_TABLE_NAME_FORMAT, + CONFIG_AUTO_CREATE_TABLES, + CONFIG_AUTO_CREATE_COLUMN_FAMILIES)) { + Map props = BasicPropertiesFactory.getSinkProps(); + props.put(configName, null); + assertThrows(ConfigException.class, () -> new BigtableSinkConfig(new HashMap<>())); + } + for (String configName : + List.of(CONFIG_INSERT_MODE, CONFIG_VALUE_NULL_MODE, CONFIG_ERROR_MODE)) { + Map props = BasicPropertiesFactory.getSinkProps(); + props.put(configName, "invalid"); + assertThrows(ConfigException.class, () -> new BigtableSinkConfig(new HashMap<>())); + } + } + + @Test + public void testDefaults() { + BigtableSinkConfig config = new BigtableSinkConfig(BasicPropertiesFactory.getSinkProps()); + assertEquals(config.getString(CONFIG_INSERT_MODE), InsertMode.INSERT.name()); + assertEquals((long) config.getInt(CONFIG_MAX_BATCH_SIZE), 1); + assertEquals(config.getString(CONFIG_VALUE_NULL_MODE), NullValueMode.WRITE.name()); + } + + @Test + public void testMultipleValuesValidationInsert() { + Map props = BasicPropertiesFactory.getSinkProps(); + props.put(BigtableSinkConfig.CONFIG_INSERT_MODE, InsertMode.INSERT.name()); + props.put(BigtableSinkConfig.CONFIG_MAX_BATCH_SIZE, "2"); + BigtableSinkConfig config = new BigtableSinkConfig(props); + assertFalse(configIsValid(config)); + } + + @Test + public void testMultipleValuesValidationCredentials() { + Map props = BasicPropertiesFactory.getSinkProps(); + props.put(BigtableSinkConfig.CONFIG_GCP_CREDENTIALS_JSON, "nonempty"); + props.put(BigtableSinkConfig.CONFIG_GCP_CREDENTIALS_PATH, "nonempty"); + BigtableSinkConfig config = new BigtableSinkConfig(props); + assertFalse(configIsValid(config)); + } + + @Test + public void testGetBigtableDataClient() { + BigtableSinkConfig config = new BigtableSinkConfig(BasicPropertiesFactory.getSinkProps()); + config.getBigtableDataClient(); + } + + @Test + public void testGetBigtableAdminClient() { + BigtableSinkConfig config = new BigtableSinkConfig(BasicPropertiesFactory.getSinkProps()); + config.getBigtableAdminClient(); + } + + @Test + public void testEnumCaseInsensitivity() { + Map props = BasicPropertiesFactory.getSinkProps(); + props.put(CONFIG_INSERT_MODE, "uPsErT"); + props.put(CONFIG_ERROR_MODE, "IGNORE"); + props.put(CONFIG_VALUE_NULL_MODE, "delete"); + BigtableSinkConfig config = new BigtableSinkConfig(props); + } + + @Test + public void testIsBigtableConfigurationValidBasicSuccess() { + Map props = BasicPropertiesFactory.getSinkProps(); + BigtableSinkConfig config = spy(new BigtableSinkConfig(props)); + BigtableTableAdminClient bigtable = mock(BigtableTableAdminClient.class); + doReturn(emptyList()).when(bigtable).listTables(); + doReturn(bigtable).when(config).getBigtableAdminClient(any()); + assertTrue(config.isBigtableConfigurationValid()); + verify(bigtable, times(1)).close(); + } + + @Test + public void testIsBigtableConfigurationValidClientConstructorError() { + Map props = BasicPropertiesFactory.getSinkProps(); + BigtableSinkConfig config = spy(new BigtableSinkConfig(props)); + doThrow(new RuntimeException()).when(config).getBigtableAdminClient(); + assertFalse(config.isBigtableConfigurationValid()); + } + + @Test + public void testIsBigtableConfigurationValidOperationError() { + Map props = BasicPropertiesFactory.getSinkProps(); + BigtableSinkConfig config = spy(new BigtableSinkConfig(props)); + BigtableTableAdminClient bigtable = mock(BigtableTableAdminClient.class); + doThrow(new RuntimeException()).when(bigtable).listTables(); + doReturn(bigtable).when(config).getBigtableAdminClient(any()); + assertFalse(config.isBigtableConfigurationValid()); + verify(bigtable, times(1)).close(); + } +} diff --git a/google-cloud-bigtable-kafka-connect-sink/src/test/java/com/google/cloud/kafka/connect/bigtable/integration/BaseIT.java b/google-cloud-bigtable-kafka-connect-sink/src/test/java/com/google/cloud/kafka/connect/bigtable/integration/BaseIT.java new file mode 100644 index 0000000000..aeec5a6503 --- /dev/null +++ b/google-cloud-bigtable-kafka-connect-sink/src/test/java/com/google/cloud/kafka/connect/bigtable/integration/BaseIT.java @@ -0,0 +1,142 @@ +/* + * Copyright 2024 Google LLC + * + * Licensed under the Apache License, Version 2.0 (the "License"); + * you may not use this file except in compliance with the License. + * You may obtain a copy of the License at + * + * https://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ +package com.google.cloud.kafka.connect.bigtable.integration; + +import static com.google.cloud.kafka.connect.bigtable.config.BigtableSinkConfig.CONFIG_BIGTABLE_INSTANCE_ID; +import static com.google.cloud.kafka.connect.bigtable.config.BigtableSinkConfig.CONFIG_GCP_PROJECT_ID; +import static org.apache.kafka.connect.runtime.ConnectorConfig.CONNECTOR_CLASS_CONFIG; +import static org.apache.kafka.connect.runtime.ConnectorConfig.TASKS_MAX_CONFIG; +import static org.apache.kafka.connect.runtime.WorkerConfig.KEY_CONVERTER_CLASS_CONFIG; +import static org.apache.kafka.connect.runtime.WorkerConfig.VALUE_CONVERTER_CLASS_CONFIG; + +import com.google.cloud.bigtable.data.v2.BigtableDataClient; +import com.google.cloud.bigtable.data.v2.models.Query; +import com.google.cloud.bigtable.data.v2.models.Row; +import com.google.cloud.kafka.connect.bigtable.BigtableSinkConnector; +import com.google.cloud.kafka.connect.bigtable.config.BigtableSinkConfig; +import com.google.cloud.kafka.connect.bigtable.util.TestId; +import java.util.HashMap; +import java.util.List; +import java.util.Map; +import java.util.Properties; +import java.util.stream.Collectors; +import org.apache.kafka.clients.admin.Admin; +import org.apache.kafka.common.utils.Utils; +import org.apache.kafka.connect.runtime.WorkerConfig; +import org.apache.kafka.connect.runtime.isolation.PluginDiscoveryMode; +import org.apache.kafka.connect.storage.StringConverter; +import org.apache.kafka.connect.util.clusters.EmbeddedConnectCluster; +import org.junit.After; +import org.junit.Before; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +public abstract class BaseIT { + private final Logger logger = LoggerFactory.getLogger(BaseIT.class); + protected EmbeddedConnectCluster connect; + private Admin kafkaAdminClient; + protected int numWorkers = 1; + protected int numBrokers = 1; + protected int numTasks = 1; + + @Before + public void setUp() { + startConnect(); + } + + @After + public void tearDown() { + stopConnect(); + } + + protected void startConnect() { + logger.info("Starting embedded Kafka Connect cluster..."); + Map workerProps = new HashMap<>(); + workerProps.put(WorkerConfig.OFFSET_COMMIT_INTERVAL_MS_CONFIG, Long.toString(10000)); + workerProps.put(WorkerConfig.PLUGIN_DISCOVERY_CONFIG, PluginDiscoveryMode.HYBRID_WARN.name()); + + Properties brokerProps = new Properties(); + brokerProps.put("message.max.bytes", 10 * 1024 * 1024); + brokerProps.put("auto.create.topics.enable", "false"); + brokerProps.put("delete.topic.enable", "true"); + connect = + new EmbeddedConnectCluster.Builder() + .name("kcbt-connect-cluster-" + getTestClassId()) + .numWorkers(numWorkers) + .numBrokers(numBrokers) + .brokerProps(brokerProps) + .workerProps(workerProps) + .build(); + + // Start the clusters + connect.start(); + try { + connect + .assertions() + .assertAtLeastNumWorkersAreUp(1, "Initial group of workers did not start in time."); + } catch (InterruptedException e) { + throw new RuntimeException(e); + } + kafkaAdminClient = connect.kafka().createAdminClient(); + } + + protected void stopConnect() { + logger.info("Stopping embedded Kafka Connect cluster..."); + if (kafkaAdminClient != null) { + Utils.closeQuietly(kafkaAdminClient, "Admin client for embedded Kafka cluster"); + kafkaAdminClient = null; + } + + // Stop all Connect, Kafka and Zk threads. + if (connect != null) { + Utils.closeQuietly(connect::stop, "Embedded Connect, Kafka, and Zookeeper clusters"); + connect = null; + } + } + + protected Map baseConnectorProps() { + Map result = new HashMap<>(); + + result.put(CONNECTOR_CLASS_CONFIG, BigtableSinkConnector.class.getCanonicalName()); + result.put(TASKS_MAX_CONFIG, Integer.toString(numTasks)); + result.put(KEY_CONVERTER_CLASS_CONFIG, StringConverter.class.getName()); + result.put(VALUE_CONVERTER_CLASS_CONFIG, StringConverter.class.getName()); + + // TODO: get it from environment variables after migrating to kokoro. + result.put(CONFIG_GCP_PROJECT_ID, "todotodo"); + result.put(CONFIG_BIGTABLE_INSTANCE_ID, "todotodo"); + + return result; + } + + protected BigtableDataClient getBigtableDataClient() { + BigtableSinkConfig config = new BigtableSinkConfig(baseConnectorProps()); + return config.getBigtableDataClient(); + } + + protected List readAllRows(BigtableDataClient bigtable, String table) { + Query query = Query.create(table); + return bigtable.readRows(query).stream().collect(Collectors.toList()); + } + + protected String getTestClassId() { + return TestId.getTestClassId(this.getClass()); + } + + protected String getTestCaseId() { + return TestId.getTestCaseId(this.getClass()); + } +} diff --git a/google-cloud-bigtable-kafka-connect-sink/src/test/java/com/google/cloud/kafka/connect/bigtable/integration/BasicIT.java b/google-cloud-bigtable-kafka-connect-sink/src/test/java/com/google/cloud/kafka/connect/bigtable/integration/BasicIT.java new file mode 100644 index 0000000000..0fd454f5d6 --- /dev/null +++ b/google-cloud-bigtable-kafka-connect-sink/src/test/java/com/google/cloud/kafka/connect/bigtable/integration/BasicIT.java @@ -0,0 +1,81 @@ +/* + * Copyright 2024 Google LLC + * + * Licensed under the Apache License, Version 2.0 (the "License"); + * you may not use this file except in compliance with the License. + * You may obtain a copy of the License at + * + * https://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ +package com.google.cloud.kafka.connect.bigtable.integration; + +import static org.apache.kafka.test.TestUtils.waitForCondition; + +import com.google.cloud.bigtable.data.v2.BigtableDataClient; +import com.google.cloud.bigtable.data.v2.models.Row; +import com.google.cloud.bigtable.data.v2.models.RowCell; +import com.google.cloud.kafka.connect.bigtable.config.BigtableSinkConfig; +import com.google.cloud.kafka.connect.bigtable.config.InsertMode; +import com.google.protobuf.ByteString; +import java.nio.charset.StandardCharsets; +import java.util.List; +import java.util.Map; +import org.apache.kafka.connect.runtime.SinkConnectorConfig; +import org.apache.kafka.test.TestCondition; +import org.junit.Test; +import org.junit.runner.RunWith; +import org.junit.runners.JUnit4; + +@RunWith(JUnit4.class) +public class BasicIT extends BaseIT { + private static final int TASK_CONSUME_TIMEOUT_MS = 10000; + + @Test + public void testSimpleWrite() throws InterruptedException { + BigtableDataClient bigtable = getBigtableDataClient(); + String topic = getTestCaseId(); + String connectorName = "connector-" + topic; + connect.kafka().createTopic(topic, numTasks); + Map props = baseConnectorProps(); + props.put(SinkConnectorConfig.TOPICS_CONFIG, topic); + props.put(BigtableSinkConfig.CONFIG_AUTO_CREATE_TABLES, "true"); + props.put(BigtableSinkConfig.CONFIG_AUTO_CREATE_COLUMN_FAMILIES, "true"); + props.put(BigtableSinkConfig.CONFIG_INSERT_MODE, InsertMode.UPSERT.name()); + connect.configureConnector(connectorName, props); + connect + .assertions() + .assertConnectorAndAtLeastNumTasksAreRunning( + connectorName, numTasks, "Connector start timeout"); + + int numberOfRecords = 1; + String key = "key"; + String value = "value"; + connect.kafka().produce(topic, key, value); + + TestCondition testCondition = + () -> { + List allRows = readAllRows(bigtable, topic); + if (numberOfRecords != allRows.size()) { + return false; + } + Row row = allRows.get(0); + if (!ByteString.copyFrom(key.getBytes(StandardCharsets.UTF_8)).equals(row.getKey())) { + return false; + } + List rowCells = row.getCells("default", "KAFKA_VALUE"); + if (numberOfRecords != rowCells.size()) { + return false; + } + return ByteString.copyFrom(value.getBytes(StandardCharsets.UTF_8)) + .equals(rowCells.get(0).getValue()); + }; + waitForCondition( + testCondition, TASK_CONSUME_TIMEOUT_MS, "Correct results not received in time"); + } +} diff --git a/google-cloud-bigtable-kafka-connect-sink/src/test/java/com/google/cloud/kafka/connect/bigtable/integration/ConfigIT.java b/google-cloud-bigtable-kafka-connect-sink/src/test/java/com/google/cloud/kafka/connect/bigtable/integration/ConfigIT.java new file mode 100644 index 0000000000..ca616fa490 --- /dev/null +++ b/google-cloud-bigtable-kafka-connect-sink/src/test/java/com/google/cloud/kafka/connect/bigtable/integration/ConfigIT.java @@ -0,0 +1,46 @@ +/* + * Copyright 2024 Google LLC + * + * Licensed under the Apache License, Version 2.0 (the "License"); + * you may not use this file except in compliance with the License. + * You may obtain a copy of the License at + * + * https://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ +package com.google.cloud.kafka.connect.bigtable.integration; + +import java.util.Map; +import org.apache.kafka.connect.runtime.SinkConnectorConfig; +import org.apache.kafka.connect.runtime.rest.errors.ConnectRestException; +import org.junit.Test; +import org.junit.runner.RunWith; +import org.junit.runners.JUnit4; + +@RunWith(JUnit4.class) +public class ConfigIT extends BaseIT { + @Test + public void testBaseSuccess() throws InterruptedException { + String topic = getTestCaseId(); + String connectorName = "connector-" + topic; + connect.kafka().createTopic(topic, numTasks); + Map props = baseConnectorProps(); + props.put(SinkConnectorConfig.TOPICS_CONFIG, topic); + connect.configureConnector(connectorName, props); + connect + .assertions() + .assertConnectorAndAtLeastNumTasksAreRunning( + connectorName, numTasks, "Connector start timeout"); + } + + @Test(expected = ConnectRestException.class) + public void testUnconfiguredError() { + Map props = baseConnectorProps(); + connect.configureConnector(getTestCaseId(), props); + } +} diff --git a/google-cloud-bigtable-kafka-connect-sink/src/test/java/com/google/cloud/kafka/connect/bigtable/integration/VersionIT.java b/google-cloud-bigtable-kafka-connect-sink/src/test/java/com/google/cloud/kafka/connect/bigtable/integration/VersionIT.java new file mode 100644 index 0000000000..5b51056b04 --- /dev/null +++ b/google-cloud-bigtable-kafka-connect-sink/src/test/java/com/google/cloud/kafka/connect/bigtable/integration/VersionIT.java @@ -0,0 +1,53 @@ +/* + * Copyright 2025 Google LLC + * + * Licensed under the Apache License, Version 2.0 (the "License"); + * you may not use this file except in compliance with the License. + * You may obtain a copy of the License at + * + * https://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ +package com.google.cloud.kafka.connect.bigtable.integration; + +import static org.junit.Assert.assertNotEquals; + +import com.fasterxml.jackson.databind.MapperFeature; +import com.fasterxml.jackson.databind.ObjectMapper; +import com.google.cloud.kafka.connect.bigtable.BigtableSinkConnector; +import com.google.cloud.kafka.connect.bigtable.version.PackageMetadata; +import java.io.IOException; +import java.net.URI; +import java.net.http.HttpClient; +import java.net.http.HttpRequest; +import java.net.http.HttpResponse; +import java.util.Arrays; +import org.apache.kafka.connect.runtime.rest.entities.PluginInfo; +import org.junit.Test; + +public class VersionIT extends BaseIT { + @Test + public void testVersionIsSet() throws IOException, InterruptedException { + String url = connect.endpointForResource("connector-plugins"); + HttpClient http = HttpClient.newHttpClient(); + HttpRequest req = HttpRequest.newBuilder(URI.create(url)).GET().build(); + HttpResponse response = http.send(req, HttpResponse.BodyHandlers.ofString()); + ObjectMapper mapper = + new ObjectMapper() + .enable(MapperFeature.ACCEPT_CASE_INSENSITIVE_ENUMS) + .enable(MapperFeature.ACCEPT_CASE_INSENSITIVE_VALUES) + .enable(MapperFeature.ACCEPT_CASE_INSENSITIVE_PROPERTIES); + PluginInfo[] pluginInfos = mapper.readValue(response.body(), PluginInfo[].class); + PluginInfo pluginInfo = + Arrays.stream(pluginInfos) + .filter(i -> i.className().equals(BigtableSinkConnector.class.getCanonicalName())) + .findFirst() + .get(); + assertNotEquals(PackageMetadata.UNKNOWN_VERSION, pluginInfo.version()); + } +} diff --git a/google-cloud-bigtable-kafka-connect-sink/src/test/java/com/google/cloud/kafka/connect/bigtable/mapping/KeyMapperTest.java b/google-cloud-bigtable-kafka-connect-sink/src/test/java/com/google/cloud/kafka/connect/bigtable/mapping/KeyMapperTest.java new file mode 100644 index 0000000000..01d26ce3ae --- /dev/null +++ b/google-cloud-bigtable-kafka-connect-sink/src/test/java/com/google/cloud/kafka/connect/bigtable/mapping/KeyMapperTest.java @@ -0,0 +1,835 @@ +/* + * Copyright 2024 Google LLC + * + * Licensed under the Apache License, Version 2.0 (the "License"); + * you may not use this file except in compliance with the License. + * You may obtain a copy of the License at + * + * https://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ +package com.google.cloud.kafka.connect.bigtable.mapping; + +/* + * This software contains code derived from the BigQuery Connector for Apache Kafka, + * Copyright Aiven Oy, which in turn contains code derived from the Confluent BigQuery + * Kafka Connector, Copyright Confluent, Inc, which in turn contains code derived from + * the WePay BigQuery Kafka Connector, Copyright WePay, Inc. + */ + +import static org.junit.Assert.assertArrayEquals; +import static org.junit.Assert.assertEquals; +import static org.junit.Assert.assertThrows; +import static org.junit.Assert.assertTrue; + +import com.google.cloud.kafka.connect.bigtable.util.JsonConverterFactory; +import java.io.ByteArrayOutputStream; +import java.io.IOException; +import java.math.BigDecimal; +import java.nio.ByteBuffer; +import java.nio.charset.StandardCharsets; +import java.util.Arrays; +import java.util.Date; +import java.util.HashMap; +import java.util.List; +import java.util.Map; +import org.apache.kafka.connect.data.Schema; +import org.apache.kafka.connect.data.SchemaAndValue; +import org.apache.kafka.connect.data.SchemaBuilder; +import org.apache.kafka.connect.data.Struct; +import org.apache.kafka.connect.errors.DataException; +import org.apache.kafka.connect.json.JsonConverter; +import org.junit.Test; +import org.junit.runner.RunWith; +import org.junit.runners.JUnit4; + +// Note that in many tests when we compare with toString() result, we most probably compare our and +// Confluent sink's implementations. +@RunWith(JUnit4.class) +public class KeyMapperTest { + private static String DELIMITER = "##"; + + @Test + public void testBoolean() { + final String fieldName = "Boolean"; + final Boolean fieldValue = true; + + Schema kafkaConnectSchema = + SchemaBuilder.struct().field(fieldName, Schema.BOOLEAN_SCHEMA).build(); + + Struct kafkaConnectStruct = new Struct(kafkaConnectSchema); + kafkaConnectStruct.put(fieldName, fieldValue); + + assertTrue( + Arrays.equals( + calculateKey(List.of(), DELIMITER, fieldValue), + fieldValue.toString().getBytes(StandardCharsets.UTF_8))); + assertTrue( + Arrays.equals( + calculateKey(List.of(), DELIMITER, kafkaConnectStruct), + fieldValue.toString().getBytes(StandardCharsets.UTF_8))); + assertTrue( + Arrays.equals( + calculateKey(List.of(fieldName), DELIMITER, kafkaConnectStruct), + fieldValue.toString().getBytes(StandardCharsets.UTF_8))); + } + + @Test + public void testInteger() { + final String fieldName = "Integer"; + final Byte fieldByteValue = (byte) 42; + Schema kafkaConnectSchema = SchemaBuilder.struct().field(fieldName, Schema.INT8_SCHEMA).build(); + + Struct kafkaConnectStruct = new Struct(kafkaConnectSchema); + kafkaConnectStruct.put(fieldName, fieldByteValue); + assertTrue( + Arrays.equals( + calculateKey(List.of(), DELIMITER, fieldByteValue), + fieldByteValue.toString().getBytes(StandardCharsets.UTF_8))); + assertTrue( + Arrays.equals( + calculateKey(List.of(), DELIMITER, kafkaConnectStruct), + fieldByteValue.toString().getBytes(StandardCharsets.UTF_8))); + assertTrue( + Arrays.equals( + calculateKey(List.of(fieldName), DELIMITER, kafkaConnectStruct), + fieldByteValue.toString().getBytes(StandardCharsets.UTF_8))); + + final Short fieldShortValue = (short) 4242; + kafkaConnectSchema = SchemaBuilder.struct().field(fieldName, Schema.INT16_SCHEMA).build(); + + kafkaConnectStruct = new Struct(kafkaConnectSchema); + kafkaConnectStruct.put(fieldName, fieldShortValue); + assertTrue( + Arrays.equals( + calculateKey(List.of(), DELIMITER, fieldShortValue), + fieldShortValue.toString().getBytes(StandardCharsets.UTF_8))); + assertTrue( + Arrays.equals( + calculateKey(List.of(), DELIMITER, kafkaConnectStruct), + fieldShortValue.toString().getBytes(StandardCharsets.UTF_8))); + assertTrue( + Arrays.equals( + calculateKey(List.of(fieldName), DELIMITER, kafkaConnectStruct), + fieldShortValue.toString().getBytes(StandardCharsets.UTF_8))); + + final Integer fieldIntegerValue = 424242; + kafkaConnectSchema = SchemaBuilder.struct().field(fieldName, Schema.INT32_SCHEMA).build(); + + kafkaConnectStruct = new Struct(kafkaConnectSchema); + kafkaConnectStruct.put(fieldName, fieldIntegerValue); + assertTrue( + Arrays.equals( + calculateKey(List.of(), DELIMITER, fieldIntegerValue), + fieldIntegerValue.toString().getBytes(StandardCharsets.UTF_8))); + assertTrue( + Arrays.equals( + calculateKey(List.of(), DELIMITER, kafkaConnectStruct), + fieldIntegerValue.toString().getBytes(StandardCharsets.UTF_8))); + assertTrue( + Arrays.equals( + calculateKey(List.of(fieldName), DELIMITER, kafkaConnectStruct), + fieldIntegerValue.toString().getBytes(StandardCharsets.UTF_8))); + + final Long fieldLongValue = 424242424242L; + kafkaConnectSchema = SchemaBuilder.struct().field(fieldName, Schema.INT64_SCHEMA).build(); + + kafkaConnectStruct = new Struct(kafkaConnectSchema); + kafkaConnectStruct.put(fieldName, fieldLongValue); + assertTrue( + Arrays.equals( + calculateKey(List.of(), DELIMITER, fieldLongValue), + fieldLongValue.toString().getBytes(StandardCharsets.UTF_8))); + assertTrue( + Arrays.equals( + calculateKey(List.of(), DELIMITER, kafkaConnectStruct), + fieldLongValue.toString().getBytes(StandardCharsets.UTF_8))); + assertTrue( + Arrays.equals( + calculateKey(List.of(fieldName), DELIMITER, kafkaConnectStruct), + fieldLongValue.toString().getBytes(StandardCharsets.UTF_8))); + } + + @Test + public void testFloat() { + final String fieldName = "Float"; + final Float fieldFloatValue = 4242424242.4242F; + Schema kafkaConnectSchema = + SchemaBuilder.struct().field(fieldName, Schema.FLOAT32_SCHEMA).build(); + + Struct kafkaConnectStruct = new Struct(kafkaConnectSchema); + kafkaConnectStruct.put(fieldName, fieldFloatValue); + assertTrue( + Arrays.equals( + calculateKey(List.of(), DELIMITER, fieldFloatValue), + fieldFloatValue.toString().getBytes(StandardCharsets.UTF_8))); + assertTrue( + Arrays.equals( + calculateKey(List.of(), DELIMITER, kafkaConnectStruct), + fieldFloatValue.toString().getBytes(StandardCharsets.UTF_8))); + assertTrue( + Arrays.equals( + calculateKey(List.of(fieldName), DELIMITER, kafkaConnectStruct), + fieldFloatValue.toString().getBytes(StandardCharsets.UTF_8))); + + final Double fieldDoubleValue = 4242424242.4242; + kafkaConnectSchema = SchemaBuilder.struct().field(fieldName, Schema.FLOAT64_SCHEMA).build(); + + kafkaConnectStruct = new Struct(kafkaConnectSchema); + kafkaConnectStruct.put(fieldName, fieldDoubleValue); + assertTrue( + Arrays.equals( + calculateKey(List.of(), DELIMITER, fieldDoubleValue), + fieldDoubleValue.toString().getBytes(StandardCharsets.UTF_8))); + assertTrue( + Arrays.equals( + calculateKey(List.of(), DELIMITER, kafkaConnectStruct), + fieldDoubleValue.toString().getBytes(StandardCharsets.UTF_8))); + assertTrue( + Arrays.equals( + calculateKey(List.of(fieldName), DELIMITER, kafkaConnectStruct), + fieldDoubleValue.toString().getBytes(StandardCharsets.UTF_8))); + } + + @Test + public void testDoubleSpecial() { + final String fieldName = "Double"; + + List testValues = + Arrays.asList(Double.POSITIVE_INFINITY, Double.NEGATIVE_INFINITY, Double.NaN); + List expectedValues = + Arrays.asList(Double.MAX_VALUE, Double.MIN_VALUE, Double.MIN_VALUE); + assertEquals(testValues.size(), expectedValues.size()); + + for (int test = 0; test < testValues.size(); ++test) { + Schema kafkaConnectSchema = + SchemaBuilder.struct().field(fieldName, Schema.FLOAT64_SCHEMA).build(); + Double testValue = testValues.get(test); + + Struct kafkaConnectStruct = new Struct(kafkaConnectSchema); + kafkaConnectStruct.put(fieldName, testValue); + assertTrue( + Arrays.equals( + calculateKey(List.of(), DELIMITER, testValue), + testValue.toString().getBytes(StandardCharsets.UTF_8))); + assertTrue( + Arrays.equals( + calculateKey(List.of(), DELIMITER, kafkaConnectStruct), + testValue.toString().getBytes(StandardCharsets.UTF_8))); + assertTrue( + Arrays.equals( + calculateKey(List.of(fieldName), DELIMITER, kafkaConnectStruct), + testValue.toString().getBytes(StandardCharsets.UTF_8))); + } + } + + @Test + public void testString() { + final String fieldName = "String"; + final String fieldValue = "42424242424242424242424242424242"; + Schema kafkaConnectSchema = + SchemaBuilder.struct().field(fieldName, Schema.STRING_SCHEMA).build(); + + Struct kafkaConnectStruct = new Struct(kafkaConnectSchema); + kafkaConnectStruct.put(fieldName, fieldValue); + assertTrue( + Arrays.equals( + calculateKey(List.of(), DELIMITER, fieldValue), + fieldValue.toString().getBytes(StandardCharsets.UTF_8))); + assertTrue( + Arrays.equals( + calculateKey(List.of(), DELIMITER, kafkaConnectStruct), + fieldValue.toString().getBytes(StandardCharsets.UTF_8))); + assertTrue( + Arrays.equals( + calculateKey(List.of(fieldName), DELIMITER, kafkaConnectStruct), + fieldValue.toString().getBytes(StandardCharsets.UTF_8))); + } + + @Test + public void testStruct() { + final String middleFieldStructName = "MiddleStruct"; + final String middleFieldArrayName = "MiddleArray"; + final String innerFieldStructName = "InnerStruct"; + final String innerFieldStringName = "InnerString"; + final String innerFieldIntegerName = "InnerInt"; + final String innerStringValue = "forty two"; + final Integer innerIntegerValue = 42; + final List middleArrayValue = Arrays.asList(42.0f, 42.4f, 42.42f, 42.424f, 42.4242f); + + Schema kafkaConnectInnerSchema = + SchemaBuilder.struct() + .field(innerFieldStringName, Schema.STRING_SCHEMA) + .field(innerFieldIntegerName, Schema.INT32_SCHEMA) + .build(); + + Struct kafkaConnectInnerStruct = new Struct(kafkaConnectInnerSchema); + kafkaConnectInnerStruct.put(innerFieldStringName, innerStringValue); + kafkaConnectInnerStruct.put(innerFieldIntegerName, innerIntegerValue); + + assertTrue( + Arrays.equals( + calculateKey(List.of(), DELIMITER, kafkaConnectInnerStruct), + (innerStringValue + DELIMITER + innerIntegerValue).getBytes(StandardCharsets.UTF_8))); + assertTrue( + Arrays.equals( + calculateKey(List.of(innerFieldStringName), DELIMITER, kafkaConnectInnerStruct), + innerStringValue.toString().getBytes(StandardCharsets.UTF_8))); + assertTrue( + Arrays.equals( + calculateKey(List.of(innerFieldIntegerName), DELIMITER, kafkaConnectInnerStruct), + innerIntegerValue.toString().getBytes(StandardCharsets.UTF_8))); + + Schema kafkaConnectMiddleSchema = + SchemaBuilder.struct() + .field(innerFieldStructName, kafkaConnectInnerSchema) + .field(middleFieldArrayName, SchemaBuilder.array(Schema.FLOAT32_SCHEMA).build()) + .build(); + + Struct kafkaConnectMiddleStruct = new Struct(kafkaConnectMiddleSchema); + kafkaConnectMiddleStruct.put(innerFieldStructName, kafkaConnectInnerStruct); + kafkaConnectMiddleStruct.put(middleFieldArrayName, middleArrayValue); + assertTrue( + Arrays.equals( + calculateKey(List.of(), DELIMITER, kafkaConnectMiddleStruct), + (kafkaConnectInnerStruct.toString() + DELIMITER + middleArrayValue.toString()) + .getBytes(StandardCharsets.UTF_8))); + assertTrue( + Arrays.equals( + calculateKey(List.of(innerFieldStructName), DELIMITER, kafkaConnectMiddleStruct), + kafkaConnectInnerStruct.toString().getBytes(StandardCharsets.UTF_8))); + assertTrue( + Arrays.equals( + calculateKey(List.of(middleFieldArrayName), DELIMITER, kafkaConnectMiddleStruct), + middleArrayValue.toString().getBytes(StandardCharsets.UTF_8))); + assertTrue( + Arrays.equals( + calculateKey( + List.of(innerFieldStructName + "." + innerFieldStringName), + DELIMITER, + kafkaConnectMiddleStruct), + innerStringValue.toString().getBytes(StandardCharsets.UTF_8))); + assertTrue( + Arrays.equals( + calculateKey( + List.of(innerFieldStructName + "." + innerFieldIntegerName), + DELIMITER, + kafkaConnectMiddleStruct), + innerIntegerValue.toString().getBytes(StandardCharsets.UTF_8))); + + Schema kafkaConnectOuterSchema = + SchemaBuilder.struct() + .field(innerFieldStructName, kafkaConnectInnerSchema) + .field(middleFieldStructName, kafkaConnectMiddleSchema) + .build(); + + Struct kafkaConnectOuterStruct = new Struct(kafkaConnectOuterSchema); + kafkaConnectOuterStruct.put(innerFieldStructName, kafkaConnectInnerStruct); + kafkaConnectOuterStruct.put(middleFieldStructName, kafkaConnectMiddleStruct); + + assertTrue( + Arrays.equals( + calculateKey(List.of(), DELIMITER, kafkaConnectOuterStruct), + (kafkaConnectInnerStruct.toString() + DELIMITER + kafkaConnectMiddleStruct.toString()) + .getBytes(StandardCharsets.UTF_8))); + assertTrue( + Arrays.equals( + calculateKey(List.of(innerFieldStructName), DELIMITER, kafkaConnectOuterStruct), + kafkaConnectInnerStruct.toString().getBytes(StandardCharsets.UTF_8))); + assertTrue( + Arrays.equals( + calculateKey(List.of(middleFieldStructName), DELIMITER, kafkaConnectOuterStruct), + kafkaConnectMiddleStruct.toString().getBytes(StandardCharsets.UTF_8))); + assertTrue( + Arrays.equals( + calculateKey( + List.of(innerFieldStructName + "." + innerFieldStringName), + DELIMITER, + kafkaConnectOuterStruct), + innerStringValue.toString().getBytes(StandardCharsets.UTF_8))); + assertTrue( + Arrays.equals( + calculateKey( + List.of( + middleFieldStructName + + "." + + innerFieldStructName + + "." + + innerFieldIntegerName), + DELIMITER, + kafkaConnectOuterStruct), + innerIntegerValue.toString().getBytes(StandardCharsets.UTF_8))); + } + + @Test + public void testEmptyInnerStruct() { + final String innerFieldStructName = "InnerStruct"; + final String innerFieldStringName = "InnerString"; + final String innerStringValue = "forty two"; + + Schema kafkaConnectInnerSchema = SchemaBuilder.struct().build(); + + Struct kafkaConnectInnerStruct = new Struct(kafkaConnectInnerSchema); + + Schema kafkaConnectOuterSchema = + SchemaBuilder.struct() + .field(innerFieldStructName, kafkaConnectInnerSchema) + .field(innerFieldStringName, Schema.STRING_SCHEMA) + .build(); + + Struct kafkaConnectOuterStruct = new Struct(kafkaConnectOuterSchema); + kafkaConnectOuterStruct.put(innerFieldStructName, kafkaConnectInnerStruct); + kafkaConnectOuterStruct.put(innerFieldStringName, innerStringValue); + assertTrue( + Arrays.equals( + calculateKey(List.of(), DELIMITER, kafkaConnectOuterStruct), + (kafkaConnectInnerStruct + DELIMITER + innerStringValue) + .getBytes(StandardCharsets.UTF_8))); + assertTrue( + Arrays.equals( + calculateKey(List.of(innerFieldStructName), DELIMITER, kafkaConnectOuterStruct), + kafkaConnectInnerStruct.toString().getBytes(StandardCharsets.UTF_8))); + assertTrue( + Arrays.equals( + calculateKey(List.of(innerFieldStructName), DELIMITER, kafkaConnectOuterStruct), + "Struct{}".getBytes(StandardCharsets.UTF_8))); + } + + @Test + public void testNull() { + Schema structOnlyOptionalFieldsSchema = + SchemaBuilder.struct().field("f", SchemaBuilder.bool().optional()).build(); + Struct structNoOptionalFields = new Struct(structOnlyOptionalFieldsSchema); + + assertThrows(DataException.class, () -> calculateKey(List.of(), DELIMITER, null)); + assertThrows( + DataException.class, () -> calculateKey(List.of(), DELIMITER, structNoOptionalFields)); + } + + @Test + public void testUnmappableValues() { + Schema structNoFieldsSchema = SchemaBuilder.struct().build(); + Struct structNoFields = new Struct(structNoFieldsSchema); + + byte[] expected = new byte[0]; + assertArrayEquals(expected, calculateKey(List.of(), DELIMITER, "")); + assertArrayEquals(expected, calculateKey(List.of(), DELIMITER, new byte[0])); + assertArrayEquals(expected, calculateKey(List.of(), DELIMITER, new HashMap<>())); + assertArrayEquals(expected, calculateKey(List.of(), DELIMITER, structNoFields)); + } + + @Test + public void testDifferentStructMappings() { + final String innerFieldStringName = "InnerString"; + final String innerFieldIntegerName = "InnerInt"; + final String innerStringValue = "forty two"; + final Integer innerIntegerValue = 42; + + Schema kafkaConnectInnerSchema = + SchemaBuilder.struct() + .field(innerFieldStringName, Schema.STRING_SCHEMA) + .field(innerFieldIntegerName, Schema.INT32_SCHEMA) + .build(); + Struct kafkaConnectInnerStruct = new Struct(kafkaConnectInnerSchema); + kafkaConnectInnerStruct.put(innerFieldStringName, innerStringValue); + kafkaConnectInnerStruct.put(innerFieldIntegerName, innerIntegerValue); + + // Note that it preserves field order from the Schema. + assertTrue( + Arrays.equals( + calculateKey(List.of(), DELIMITER, kafkaConnectInnerStruct), + (innerStringValue + DELIMITER + innerIntegerValue).getBytes(StandardCharsets.UTF_8))); + // Force another order. + assertTrue( + Arrays.equals( + calculateKey( + List.of(innerFieldIntegerName, innerFieldStringName), + DELIMITER, + kafkaConnectInnerStruct), + (innerIntegerValue + DELIMITER + innerStringValue).getBytes(StandardCharsets.UTF_8))); + // Use the same field twice. + assertTrue( + Arrays.equals( + calculateKey( + List.of(innerFieldIntegerName, innerFieldIntegerName), + DELIMITER, + kafkaConnectInnerStruct), + (innerIntegerValue + DELIMITER + innerIntegerValue).getBytes(StandardCharsets.UTF_8))); + // Try accessing nonexistent key. + assertThrows( + DataException.class, + () -> calculateKey(List.of("invalid"), DELIMITER, kafkaConnectInnerStruct)); + } + + @Test + public void testMap() { + final String fieldNameIntegerMap = "IntegerMap"; + final String fieldNameStringMap = "StringMap"; + final Map integerMap = new HashMap<>(); + final Map stringMap = new HashMap<>(); + + for (int n = 2; n <= 10; n++) { + boolean isPrime = true; + for (int d : integerMap.keySet()) { + if (n % d == 0) { + isPrime = false; + break; + } + } + integerMap.put(n, isPrime); + } + for (int n = 2; n <= 10; n++) { + boolean isPrime = true; + for (String s : stringMap.keySet()) { + Integer d = Integer.parseInt(s); + if (n % d == 0) { + isPrime = false; + break; + } + } + stringMap.put(Integer.toString(n), isPrime); + } + Schema kafkaConnectSchema = + SchemaBuilder.struct() + .field( + fieldNameIntegerMap, SchemaBuilder.map(Schema.INT32_SCHEMA, Schema.BOOLEAN_SCHEMA)) + .field( + fieldNameStringMap, SchemaBuilder.map(Schema.STRING_SCHEMA, Schema.BOOLEAN_SCHEMA)) + .build(); + + Struct kafkaConnectStruct = new Struct(kafkaConnectSchema); + kafkaConnectStruct.put(fieldNameIntegerMap, integerMap); + kafkaConnectStruct.put(fieldNameStringMap, stringMap); + assertTrue( + Arrays.equals( + calculateKey(List.of(), DELIMITER, kafkaConnectStruct), + (integerMap.toString() + DELIMITER + stringMap.toString()) + .getBytes(StandardCharsets.UTF_8))); + assertTrue( + Arrays.equals( + calculateKey(List.of(fieldNameIntegerMap), DELIMITER, kafkaConnectStruct), + integerMap.toString().getBytes(StandardCharsets.UTF_8))); + assertTrue( + Arrays.equals( + calculateKey(List.of(fieldNameStringMap), DELIMITER, kafkaConnectStruct), + stringMap.toString().getBytes(StandardCharsets.UTF_8))); + // The key is Integer, not String - we don't support it + assertThrows( + DataException.class, + () -> calculateKey(List.of(fieldNameIntegerMap + ".3"), DELIMITER, kafkaConnectStruct)); + assertTrue( + Arrays.equals( + calculateKey(List.of(fieldNameStringMap + ".3"), DELIMITER, kafkaConnectStruct), + "true".getBytes(StandardCharsets.UTF_8))); + } + + @Test + public void testIntegerArray() { + final String fieldName = "IntegerArray"; + final List fieldValue = Arrays.asList(42, 4242, 424242, 42424242); + + Schema kafkaConnectSchema = + SchemaBuilder.struct() + .field(fieldName, SchemaBuilder.array(Schema.INT32_SCHEMA).build()) + .build(); + + Struct kafkaConnectStruct = new Struct(kafkaConnectSchema); + kafkaConnectStruct.put(fieldName, fieldValue); + assertTrue( + Arrays.equals( + calculateKey(List.of(), DELIMITER, kafkaConnectStruct), + fieldValue.toString().getBytes(StandardCharsets.UTF_8))); + assertTrue( + Arrays.equals( + calculateKey(List.of(fieldName), DELIMITER, kafkaConnectStruct), + fieldValue.toString().getBytes(StandardCharsets.UTF_8))); + } + + @Test + public void testStructArray() { + final String innerFieldStringName = "InnerString"; + final String innerFieldIntegerName = "InnerInt"; + final String innerStringValue = "42"; + final Integer innerIntegerValue = 42; + Schema kafkaConnectInnerSchema = + SchemaBuilder.struct() + .field(innerFieldStringName, Schema.STRING_SCHEMA) + .field(innerFieldIntegerName, Schema.INT32_SCHEMA) + .build(); + Struct kafkaConnectInnerStruct = new Struct(kafkaConnectInnerSchema); + kafkaConnectInnerStruct.put(innerFieldStringName, innerStringValue); + kafkaConnectInnerStruct.put(innerFieldIntegerName, innerIntegerValue); + + final String middleFieldArrayName = "MiddleArray"; + Schema kafkaConnectSchema = + SchemaBuilder.struct() + .field(middleFieldArrayName, SchemaBuilder.array(kafkaConnectInnerSchema).build()) + .build(); + + Struct kafkaConnectStruct = new Struct(kafkaConnectSchema); + List innerStructList = List.of(kafkaConnectInnerStruct); + kafkaConnectStruct.put(middleFieldArrayName, innerStructList); + + assertTrue( + Arrays.equals( + calculateKey(List.of(), DELIMITER, kafkaConnectStruct), + innerStructList.toString().getBytes(StandardCharsets.UTF_8))); + assertTrue( + Arrays.equals( + calculateKey(List.of(middleFieldArrayName), DELIMITER, kafkaConnectStruct), + innerStructList.toString().getBytes(StandardCharsets.UTF_8))); + } + + @Test + public void testStringArray() { + final String fieldName = "StringArray"; + final List fieldValue = + Arrays.asList("Forty-two", "forty-two", "Forty two", "forty two"); + + Schema kafkaConnectSchema = + SchemaBuilder.struct() + .field(fieldName, SchemaBuilder.array(Schema.STRING_SCHEMA).build()) + .build(); + + Struct kafkaConnectStruct = new Struct(kafkaConnectSchema); + kafkaConnectStruct.put(fieldName, fieldValue); + + assertTrue( + Arrays.equals( + calculateKey(List.of(), DELIMITER, kafkaConnectStruct), + fieldValue.toString().getBytes(StandardCharsets.UTF_8))); + assertTrue( + Arrays.equals( + calculateKey(List.of(fieldName), DELIMITER, kafkaConnectStruct), + fieldValue.toString().getBytes(StandardCharsets.UTF_8))); + } + + @Test + public void testBytes() { + final String fieldName = "Bytes"; + final byte[] fieldBytes = new byte[] {42, 43, 44, 45, 46, 47, 48, 49, 50, 51, 52, 53, 54}; + final ByteBuffer fieldValueKafkaConnect = ByteBuffer.wrap(fieldBytes); + Schema kafkaConnectSchema = + SchemaBuilder.struct().field(fieldName, Schema.BYTES_SCHEMA).build(); + + Struct kafkaConnectStruct = new Struct(kafkaConnectSchema); + kafkaConnectStruct.put(fieldName, fieldValueKafkaConnect); + assertTrue(Arrays.equals(calculateKey(List.of(), DELIMITER, kafkaConnectStruct), fieldBytes)); + assertTrue( + Arrays.equals(calculateKey(List.of(fieldName), DELIMITER, kafkaConnectStruct), fieldBytes)); + } + + @Test + public void testBytesInStruct() throws IOException { + final String innerFieldStructName = "InnerStruct"; + final String innerFieldBytesName = "InnerBytes"; + ByteArrayOutputStream inputBuilder = new ByteArrayOutputStream(); + for (int i = -128; i < 128; i++) { + inputBuilder.write(i); + } + byte[] innerBytesValue = inputBuilder.toByteArray(); + + Schema kafkaConnectInnerSchema = + SchemaBuilder.struct().field(innerFieldBytesName, Schema.BYTES_SCHEMA).build(); + Struct kafkaConnectInnerStruct = new Struct(kafkaConnectInnerSchema); + kafkaConnectInnerStruct.put(innerFieldBytesName, innerBytesValue); + + Schema kafkaConnectOuterSchema = + SchemaBuilder.struct().field(innerFieldStructName, kafkaConnectInnerSchema).build(); + + Struct kafkaConnectOuterStruct = new Struct(kafkaConnectOuterSchema); + kafkaConnectOuterStruct.put(innerFieldStructName, kafkaConnectInnerStruct); + + ByteArrayOutputStream expectedBuilder = new ByteArrayOutputStream(); + expectedBuilder.write(("Struct{" + innerFieldBytesName + "=").getBytes(StandardCharsets.UTF_8)); + expectedBuilder.write(innerBytesValue); + expectedBuilder.write("}".getBytes(StandardCharsets.UTF_8)); + byte[] expectedStructSerialization = expectedBuilder.toByteArray(); + + assertTrue( + Arrays.equals( + calculateKey(List.of(), DELIMITER, kafkaConnectOuterStruct), + expectedStructSerialization)); + assertTrue( + Arrays.equals( + calculateKey(List.of(innerFieldStructName), DELIMITER, kafkaConnectOuterStruct), + expectedStructSerialization)); + assertTrue( + Arrays.equals( + calculateKey( + List.of(innerFieldStructName + "." + innerFieldBytesName), + DELIMITER, + kafkaConnectOuterStruct), + innerBytesValue)); + } + + @Test + public void testKafkaLogicalTypes() { + final String dateFieldName = "KafkaDate"; + final String timestampFieldName = "KafkaTimestamp"; + final String timeFieldName = "KafkaTime"; + final String decimalFieldName = "KafkaDecimal"; + final Long dateLong = 1488406838808L; + final Date date = new Date(dateLong); + final String decimalString = "0.30000000000000004"; + final Integer decimalScale = 0; + final BigDecimal decimal = new BigDecimal(decimalString); + + Schema kafkaConnectSchema = + SchemaBuilder.struct() + .field(dateFieldName, org.apache.kafka.connect.data.Date.SCHEMA) + .field(timestampFieldName, org.apache.kafka.connect.data.Timestamp.SCHEMA) + .field(timeFieldName, org.apache.kafka.connect.data.Timestamp.SCHEMA) + .field(decimalFieldName, org.apache.kafka.connect.data.Decimal.schema(decimalScale)) + .build(); + + Struct kafkaConnectStruct = new Struct(kafkaConnectSchema); + kafkaConnectStruct.put(dateFieldName, date); + kafkaConnectStruct.put(timestampFieldName, date); + kafkaConnectStruct.put(timeFieldName, date); + kafkaConnectStruct.put(decimalFieldName, decimal); + // TODO: test in practice whether the Confluent sink works exactly like this. + assertTrue( + Arrays.equals( + calculateKey(List.of(dateFieldName), DELIMITER, kafkaConnectStruct), + date.toString().getBytes(StandardCharsets.UTF_8))); + assertTrue( + Arrays.equals( + calculateKey(List.of(timestampFieldName), DELIMITER, kafkaConnectStruct), + date.toString().getBytes(StandardCharsets.UTF_8))); + assertTrue( + Arrays.equals( + calculateKey(List.of(timeFieldName), DELIMITER, kafkaConnectStruct), + date.toString().getBytes(StandardCharsets.UTF_8))); + assertTrue( + Arrays.equals( + calculateKey(List.of(decimalFieldName), DELIMITER, kafkaConnectStruct), + decimalString.getBytes(StandardCharsets.UTF_8))); + } + + @Test + public void testNullable() { + final String nullableFieldName = "nullable"; + final String requiredFieldName = "required"; + final Integer nullableFieldValue = null; + final Integer requiredFieldValue = 42; + + Schema kafkaConnectSchema = + SchemaBuilder.struct() + .field(nullableFieldName, SchemaBuilder.int32().optional().build()) + .field(requiredFieldName, SchemaBuilder.int32().required().build()) + .build(); + + Struct kafkaConnectStruct = new Struct(kafkaConnectSchema); + kafkaConnectStruct.put(nullableFieldName, nullableFieldValue); + kafkaConnectStruct.put(requiredFieldName, requiredFieldValue); + assertThrows(DataException.class, () -> calculateKey(List.of(), DELIMITER, kafkaConnectStruct)); + assertThrows( + DataException.class, + () -> calculateKey(List.of(nullableFieldName), DELIMITER, kafkaConnectStruct)); + assertTrue( + Arrays.equals( + calculateKey(List.of(requiredFieldName), DELIMITER, kafkaConnectStruct), + requiredFieldValue.toString().getBytes(StandardCharsets.UTF_8))); + } + + @Test + public void testNullableStruct() { + final String nullableFieldName = "nullableStruct"; + final String innerStructFieldName = "foobar"; + + Schema kafkaConnectSchema = + SchemaBuilder.struct() + .field( + nullableFieldName, + SchemaBuilder.struct() + .field(innerStructFieldName, SchemaBuilder.bool().build()) + .optional() + .build()) + .build(); + + Struct kafkaConnectStruct = new Struct(kafkaConnectSchema); + kafkaConnectStruct.put(nullableFieldName, null); + assertThrows(DataException.class, () -> calculateKey(List.of(), DELIMITER, kafkaConnectStruct)); + assertThrows( + DataException.class, + () -> calculateKey(List.of(nullableFieldName), DELIMITER, kafkaConnectStruct)); + assertThrows( + DataException.class, + () -> + calculateKey( + List.of(nullableFieldName + "." + innerStructFieldName), + DELIMITER, + kafkaConnectStruct)); + } + + @Test + public void testSchemalessRecordSuccesses() { + JsonConverter jsonConverter = JsonConverterFactory.create(false, true); + + String topic = "topic"; + String delimiter = "##"; + + for (Object[] testCase : + List.of( + // Default key definition and all kinds of types. + // I know of no way to pass unserialized bytes or logical types here. I think it's only + // possible using some kind of schema. + new Object[] {List.of(), "2.130", "2.13"}, + new Object[] {List.of(), "7", "7"}, + new Object[] {List.of(), "\"x\"", "x"}, + new Object[] {List.of(), "true", "true"}, + new Object[] {List.of(), "[]", "[]"}, + new Object[] {List.of(), "[1,\"s\",true]", "[1, s, true]"}, + // Default key definition when using on a map (schemaless data is converted into Map not + // Struct!). + new Object[] {List.of(), "{\"a\":1,\"b\":true,\"c\":\"str\"}", "1##true##str"}, + new Object[] { + List.of(), "{\"b\":1,\"a\":3}", "3##1" + }, // Note it doesn't keep key ordering. + new Object[] { + List.of(), + "{\"b\":[1,2],\"a\":3,\"c\":{\"x\":\"D\",\"y\":2137}}", + "3##[1, 2]##{x=D, y=2137}" + }, + // Key extraction and serialization of nested beings. + new Object[] {List.of("f"), "{\"f\":{}}", "{}"}, + new Object[] {List.of("f"), "{\"f\":1}", "1"}, + new Object[] {List.of("f"), "{\"f\":true}", "true"}, + new Object[] {List.of("f"), "{\"f\":\"s\"}", "s"}, + new Object[] {List.of("f"), "{\"f\":[]}", "[]"}, + new Object[] {List.of("f"), "{\"f\":[1,\"a\"]}", "[1, a]"}, + new Object[] {List.of("f"), "{\"f\":{\"b\":1,\"a\":3}}", "{a=3, b=1}"}, + new Object[] {List.of("f"), "{\"f\":{\"a\":{\"b\": true}}}", "{a={b=true}}"}, + new Object[] { + List.of("f"), "{\"f\":{\"a\":{\"b\": true,\"c\":2}}}", "{a={b=true, c=2}}" + }, + new Object[] {List.of("f.a"), "{\"f\":{\"b\":1,\"a\":3}}", "3"})) { + KeyMapper mapper = new KeyMapper(delimiter, (List) testCase[0]); + SchemaAndValue connectData = + jsonConverter.toConnectData( + topic, ((String) testCase[1]).getBytes(StandardCharsets.UTF_8)); + byte[] expectedResult = ((String) testCase[2]).getBytes(StandardCharsets.UTF_8); + byte[] result = mapper.getKey(connectData.value()); + assertTrue(Arrays.equals(expectedResult, result)); + } + ; + } + + @Test + public void testAccessingSchemalessPrimitiveField() { + KeyMapper mapper = new KeyMapper("#", List.of("fieldName")); + assertThrows(DataException.class, () -> mapper.getKey("primitiveString")); + } + + private static byte[] calculateKey( + List mapperDefinition, String mapperDelimiter, Object kafkaKey) { + KeyMapper mapper = new KeyMapper(mapperDelimiter, mapperDefinition); + return mapper.getKey(kafkaKey); + } +} diff --git a/google-cloud-bigtable-kafka-connect-sink/src/test/java/com/google/cloud/kafka/connect/bigtable/mapping/MutationDataBuilderTest.java b/google-cloud-bigtable-kafka-connect-sink/src/test/java/com/google/cloud/kafka/connect/bigtable/mapping/MutationDataBuilderTest.java new file mode 100644 index 0000000000..62fc0d0011 --- /dev/null +++ b/google-cloud-bigtable-kafka-connect-sink/src/test/java/com/google/cloud/kafka/connect/bigtable/mapping/MutationDataBuilderTest.java @@ -0,0 +1,102 @@ +/* + * Copyright 2024 Google LLC + * + * Licensed under the Apache License, Version 2.0 (the "License"); + * you may not use this file except in compliance with the License. + * You may obtain a copy of the License at + * + * https://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ +package com.google.cloud.kafka.connect.bigtable.mapping; + +import static org.junit.Assert.assertEquals; +import static org.junit.Assert.assertTrue; +import static org.mockito.Mockito.mock; +import static org.mockito.Mockito.times; +import static org.mockito.Mockito.verify; + +import com.google.cloud.bigtable.data.v2.models.Mutation; +import com.google.cloud.bigtable.data.v2.models.Range; +import com.google.protobuf.ByteString; +import java.nio.charset.StandardCharsets; +import java.util.Optional; +import java.util.Set; +import org.junit.Before; +import org.junit.Test; +import org.junit.runner.RunWith; +import org.junit.runners.JUnit4; + +@RunWith(JUnit4.class) +public class MutationDataBuilderTest { + private static final ByteString ROW_KEY = + ByteString.copyFrom("ROW_KEY".getBytes(StandardCharsets.UTF_8)); + private static final String TARGET_TABLE_NAME = "table"; + private static final String COLUMN_FAMILY = "family"; + private static final ByteString COLUMN_QUALIFIER = + ByteString.copyFrom("COLUMN".getBytes(StandardCharsets.UTF_8)); + private static final ByteString VALUE = + ByteString.copyFrom("VALUE".getBytes(StandardCharsets.UTF_8)); + private static final Long TIMESTAMP = 2024L; + private static final Range.TimestampRange TIMESTAMP_RANGE = + Range.TimestampRange.create(0, TIMESTAMP); + + Mutation mutation; + MutationDataBuilder mutationDataBuilder; + + @Before + public void setUp() { + mutation = mock(Mutation.class); + mutationDataBuilder = new MutationDataBuilder(mutation); + } + + @Test + public void testEmpty() { + assertTrue(mutationDataBuilder.maybeBuild(TARGET_TABLE_NAME, ROW_KEY).isEmpty()); + } + + @Test + public void testDeleteRow() { + mutationDataBuilder.deleteRow(); + verify(mutation, times(1)).deleteRow(); + Optional mutationData = + mutationDataBuilder.maybeBuild(TARGET_TABLE_NAME, ROW_KEY); + assertTrue(mutationData.isPresent()); + assertEquals(mutationData.get().getRequiredColumnFamilies(), Set.of()); + } + + @Test + public void testDeleteCells() { + mutationDataBuilder.deleteCells(COLUMN_FAMILY, COLUMN_QUALIFIER, TIMESTAMP_RANGE); + verify(mutation, times(1)).deleteCells(COLUMN_FAMILY, COLUMN_QUALIFIER, TIMESTAMP_RANGE); + Optional mutationData = + mutationDataBuilder.maybeBuild(TARGET_TABLE_NAME, ROW_KEY); + assertTrue(mutationData.isPresent()); + assertTrue(mutationData.get().getRequiredColumnFamilies().isEmpty()); + } + + @Test + public void testDeleteFamily() { + mutationDataBuilder.deleteFamily(COLUMN_FAMILY); + verify(mutation, times(1)).deleteFamily(COLUMN_FAMILY); + Optional mutationData = + mutationDataBuilder.maybeBuild(TARGET_TABLE_NAME, ROW_KEY); + assertTrue(mutationData.isPresent()); + assertTrue(mutationData.get().getRequiredColumnFamilies().isEmpty()); + } + + @Test + public void testSetCell() { + mutationDataBuilder.setCell(COLUMN_FAMILY, COLUMN_QUALIFIER, TIMESTAMP, VALUE); + verify(mutation, times(1)).setCell(COLUMN_FAMILY, COLUMN_QUALIFIER, TIMESTAMP, VALUE); + Optional mutationData = + mutationDataBuilder.maybeBuild(TARGET_TABLE_NAME, ROW_KEY); + assertTrue(mutationData.isPresent()); + assertEquals(mutationData.get().getRequiredColumnFamilies(), Set.of(COLUMN_FAMILY)); + } +} diff --git a/google-cloud-bigtable-kafka-connect-sink/src/test/java/com/google/cloud/kafka/connect/bigtable/mapping/ValueMapperTest.java b/google-cloud-bigtable-kafka-connect-sink/src/test/java/com/google/cloud/kafka/connect/bigtable/mapping/ValueMapperTest.java new file mode 100644 index 0000000000..ce2762fd76 --- /dev/null +++ b/google-cloud-bigtable-kafka-connect-sink/src/test/java/com/google/cloud/kafka/connect/bigtable/mapping/ValueMapperTest.java @@ -0,0 +1,751 @@ +/* + * Copyright 2024 Google LLC + * + * Licensed under the Apache License, Version 2.0 (the "License"); + * you may not use this file except in compliance with the License. + * You may obtain a copy of the License at + * + * https://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ +package com.google.cloud.kafka.connect.bigtable.mapping; + +import static com.google.cloud.kafka.connect.bigtable.util.MockUtil.assertTotalNumberOfInvocations; +import static org.junit.Assert.assertTrue; +import static org.mockito.Mockito.spy; +import static org.mockito.Mockito.times; +import static org.mockito.Mockito.verify; + +import com.google.cloud.bigtable.data.v2.models.Range; +import com.google.cloud.kafka.connect.bigtable.config.NullValueMode; +import com.google.cloud.kafka.connect.bigtable.util.JsonConverterFactory; +import com.google.protobuf.ByteString; +import java.math.BigDecimal; +import java.nio.charset.StandardCharsets; +import java.util.Date; +import java.util.HashMap; +import java.util.List; +import java.util.Map; +import java.util.TreeMap; +import org.apache.hadoop.hbase.util.Bytes; +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.json.JsonConverter; +import org.junit.Ignore; +import org.junit.Test; +import org.junit.runner.RunWith; +import org.junit.runners.JUnit4; + +@RunWith(JUnit4.class) +public class ValueMapperTest { + private static final String DEFAULT_COLUMN_FAMILY = "COLUMN_FAMILY"; + private static final String DEFAULT_COLUMN = "COLUMN_QUALIFIER"; + private static final ByteString DEFAULT_COLUMN_BYTES = + ByteString.copyFrom(DEFAULT_COLUMN.getBytes(StandardCharsets.UTF_8)); + private static final ByteString ROW_KEY = + ByteString.copyFrom("ROW_KEY".getBytes(StandardCharsets.UTF_8)); + private static final String TARGET_TABLE_NAME = "table"; + private static final Long TIMESTAMP = 2024L; + private static final Range.TimestampRange TIMESTAMP_RANGE = + Range.TimestampRange.create(0, TIMESTAMP); + private static final String DEFAULT_TOPIC = "topic"; + + private static final String NESTED_NULL_STRUCT_FIELD_NAME = "struct"; + private static final ByteString NESTED_NULL_STRUCT_FIELD_NAME_BYTES = + ByteString.copyFrom(NESTED_NULL_STRUCT_FIELD_NAME.getBytes(StandardCharsets.UTF_8)); + + private static final JsonConverter jsonConverter = JsonConverterFactory.create(false, false); + + @Test + public void testBoolean() { + Boolean value = true; + ByteString expected = ByteString.copyFrom(Bytes.toBytes(value)); + ValueMapper mapper = + new TestValueMapper(DEFAULT_COLUMN_FAMILY, DEFAULT_COLUMN, NullValueMode.IGNORE); + MutationDataBuilder mutationDataBuilder = mapper.getRecordMutationDataBuilder(value, TIMESTAMP); + verify(mutationDataBuilder, times(1)) + .setCell(DEFAULT_COLUMN_FAMILY, DEFAULT_COLUMN_BYTES, TIMESTAMP, expected); + assertTotalNumberOfInvocations(mutationDataBuilder, 1); + } + + @Test + public void testString() { + String value = "rrrrrrr"; + ByteString expected = ByteString.copyFrom(Bytes.toBytes(value)); + ValueMapper mapper = + new TestValueMapper(DEFAULT_COLUMN_FAMILY, DEFAULT_COLUMN, NullValueMode.IGNORE); + MutationDataBuilder mutationDataBuilder = mapper.getRecordMutationDataBuilder(value, TIMESTAMP); + verify(mutationDataBuilder, times(1)) + .setCell(DEFAULT_COLUMN_FAMILY, DEFAULT_COLUMN_BYTES, TIMESTAMP, expected); + assertTotalNumberOfInvocations(mutationDataBuilder, 1); + } + + @Test + public void testLong() { + Long value = 9223372036854775807L; + ByteString expected = ByteString.copyFrom(Bytes.toBytes(value)); + ValueMapper mapper = + new TestValueMapper(DEFAULT_COLUMN_FAMILY, DEFAULT_COLUMN, NullValueMode.IGNORE); + MutationDataBuilder mutationDataBuilder = mapper.getRecordMutationDataBuilder(value, TIMESTAMP); + verify(mutationDataBuilder, times(1)) + .setCell(DEFAULT_COLUMN_FAMILY, DEFAULT_COLUMN_BYTES, TIMESTAMP, expected); + assertTotalNumberOfInvocations(mutationDataBuilder, 1); + } + + @Test + public void testInteger() { + Integer value = -2147483648; + ByteString expected = ByteString.copyFrom(Bytes.toBytes(value)); + ValueMapper mapper = + new TestValueMapper(DEFAULT_COLUMN_FAMILY, DEFAULT_COLUMN, NullValueMode.IGNORE); + MutationDataBuilder mutationDataBuilder = mapper.getRecordMutationDataBuilder(value, TIMESTAMP); + verify(mutationDataBuilder, times(1)) + .setCell(DEFAULT_COLUMN_FAMILY, DEFAULT_COLUMN_BYTES, TIMESTAMP, expected); + assertTotalNumberOfInvocations(mutationDataBuilder, 1); + } + + @Test + public void testShort() { + Short value = 32767; + ByteString expected = ByteString.copyFrom(Bytes.toBytes(value)); + ValueMapper mapper = + new TestValueMapper(DEFAULT_COLUMN_FAMILY, DEFAULT_COLUMN, NullValueMode.IGNORE); + MutationDataBuilder mutationDataBuilder = mapper.getRecordMutationDataBuilder(value, TIMESTAMP); + verify(mutationDataBuilder, times(1)) + .setCell(DEFAULT_COLUMN_FAMILY, DEFAULT_COLUMN_BYTES, TIMESTAMP, expected); + assertTotalNumberOfInvocations(mutationDataBuilder, 1); + } + + @Test + public void testByte() { + Byte value = -128; + ByteString expected = ByteString.copyFrom(Bytes.toBytes(value)); + ValueMapper mapper = + new TestValueMapper(DEFAULT_COLUMN_FAMILY, DEFAULT_COLUMN, NullValueMode.IGNORE); + MutationDataBuilder mutationDataBuilder = mapper.getRecordMutationDataBuilder(value, TIMESTAMP); + verify(mutationDataBuilder, times(1)) + .setCell(DEFAULT_COLUMN_FAMILY, DEFAULT_COLUMN_BYTES, TIMESTAMP, expected); + assertTotalNumberOfInvocations(mutationDataBuilder, 1); + } + + @Test + public void testBytes() { + byte[] value = new byte[] {(byte) 37, (byte) 21}; + ByteString expected = ByteString.copyFrom(value); + ValueMapper mapper = + new TestValueMapper(DEFAULT_COLUMN_FAMILY, DEFAULT_COLUMN, NullValueMode.IGNORE); + MutationDataBuilder mutationDataBuilder = mapper.getRecordMutationDataBuilder(value, TIMESTAMP); + verify(mutationDataBuilder, times(1)) + .setCell(DEFAULT_COLUMN_FAMILY, DEFAULT_COLUMN_BYTES, TIMESTAMP, expected); + assertTotalNumberOfInvocations(mutationDataBuilder, 1); + } + + @Test + public void testFloat() { + Float value = 128.37157f; + ByteString expected = ByteString.copyFrom(Bytes.toBytes(value)); + ValueMapper mapper = + new TestValueMapper(DEFAULT_COLUMN_FAMILY, DEFAULT_COLUMN, NullValueMode.IGNORE); + MutationDataBuilder mutationDataBuilder = mapper.getRecordMutationDataBuilder(value, TIMESTAMP); + verify(mutationDataBuilder, times(1)) + .setCell(DEFAULT_COLUMN_FAMILY, DEFAULT_COLUMN_BYTES, TIMESTAMP, expected); + assertTotalNumberOfInvocations(mutationDataBuilder, 1); + } + + @Test + public void testDouble() { + Double value = 128.37157; + ByteString expected = ByteString.copyFrom(Bytes.toBytes(value)); + ValueMapper mapper = + new TestValueMapper(DEFAULT_COLUMN_FAMILY, DEFAULT_COLUMN, NullValueMode.IGNORE); + MutationDataBuilder mutationDataBuilder = mapper.getRecordMutationDataBuilder(value, TIMESTAMP); + verify(mutationDataBuilder, times(1)) + .setCell(DEFAULT_COLUMN_FAMILY, DEFAULT_COLUMN_BYTES, TIMESTAMP, expected); + assertTotalNumberOfInvocations(mutationDataBuilder, 1); + } + + @Test + public void testDoubleSpecial() { + Double value = Double.NaN; + ByteString expected = ByteString.copyFrom(Bytes.toBytes(value)); + ValueMapper mapper = + new TestValueMapper(DEFAULT_COLUMN_FAMILY, DEFAULT_COLUMN, NullValueMode.IGNORE); + MutationDataBuilder mutationDataBuilder = mapper.getRecordMutationDataBuilder(value, TIMESTAMP); + verify(mutationDataBuilder, times(1)) + .setCell(DEFAULT_COLUMN_FAMILY, DEFAULT_COLUMN_BYTES, TIMESTAMP, expected); + assertTotalNumberOfInvocations(mutationDataBuilder, 1); + } + + @Ignore // TODO: fix it. + @Test + public void testDate() { + // TODO: is it correct? Or maybe should the implementation first convert it into logical value? + Long value = 1732822801000L; + ByteString expected = ByteString.copyFrom(Bytes.toBytes(value)); + ValueMapper mapper = + new TestValueMapper(DEFAULT_COLUMN_FAMILY, DEFAULT_COLUMN, NullValueMode.IGNORE); + MutationDataBuilder mutationDataBuilder = + mapper.getRecordMutationDataBuilder(new Date(value), TIMESTAMP); + verify(mutationDataBuilder, times(1)) + .setCell(DEFAULT_COLUMN_FAMILY, DEFAULT_COLUMN_BYTES, TIMESTAMP, expected); + assertTotalNumberOfInvocations(mutationDataBuilder, 1); + } + + @Ignore // TODO: fix it. + @Test + public void testDecimal() { + // TODO: is it correct? Or maybe should the implementation first convert it into logical value? + BigDecimal value = new BigDecimal("0.30000000000000000004"); + ByteString expected = ByteString.copyFrom(Bytes.toBytes(value)); + ValueMapper mapper = + new TestValueMapper(DEFAULT_COLUMN_FAMILY, DEFAULT_COLUMN, NullValueMode.IGNORE); + MutationDataBuilder mutationDataBuilder = mapper.getRecordMutationDataBuilder(value, TIMESTAMP); + verify(mutationDataBuilder, times(1)) + .setCell(DEFAULT_COLUMN_FAMILY, DEFAULT_COLUMN_BYTES, TIMESTAMP, expected); + assertTotalNumberOfInvocations(mutationDataBuilder, 1); + } + + @Test + public void testArray() { + List value = List.of("1", 2, true); + ValueMapper mapper = + new TestValueMapper(DEFAULT_COLUMN_FAMILY, DEFAULT_COLUMN, NullValueMode.IGNORE); + MutationDataBuilder mutationDataBuilder = mapper.getRecordMutationDataBuilder(value, TIMESTAMP); + verify(mutationDataBuilder, times(1)) + .setCell( + DEFAULT_COLUMN_FAMILY, + DEFAULT_COLUMN_BYTES, + TIMESTAMP, + ByteString.copyFrom("[\"1\",2,true]".getBytes(StandardCharsets.UTF_8))); + assertTotalNumberOfInvocations(mutationDataBuilder, 1); + } + + @Test + public void testRootValueNeedsBothDefaultColumns() { + Integer value = 123; + for (ValueMapper mapper : + List.of( + new TestValueMapper(null, null, NullValueMode.WRITE), + new TestValueMapper(DEFAULT_COLUMN_FAMILY, null, NullValueMode.WRITE), + new TestValueMapper(null, DEFAULT_COLUMN, NullValueMode.WRITE))) { + MutationDataBuilder mutationDataBuilder = + mapper.getRecordMutationDataBuilder(value, TIMESTAMP); + verify(mutationDataBuilder, times(0)) + .setCell( + DEFAULT_COLUMN_FAMILY, + DEFAULT_COLUMN_BYTES, + TIMESTAMP, + ByteString.copyFrom(Bytes.toBytes(value))); + } + ValueMapper mapper = + new TestValueMapper(DEFAULT_COLUMN_FAMILY, DEFAULT_COLUMN, NullValueMode.WRITE); + MutationDataBuilder mutationDataBuilder = mapper.getRecordMutationDataBuilder(value, TIMESTAMP); + verify(mutationDataBuilder, times(1)) + .setCell( + DEFAULT_COLUMN_FAMILY, + DEFAULT_COLUMN_BYTES, + TIMESTAMP, + ByteString.copyFrom(Bytes.toBytes(value))); + assertTotalNumberOfInvocations(mutationDataBuilder, 1); + } + + @Test + public void testValueNestedOnceNeedsOnlyDefaultColumnFamily() { + Object value = fromJson("{\"key\": 2}"); + ValueMapper mapper = new TestValueMapper(DEFAULT_COLUMN_FAMILY, null, NullValueMode.WRITE); + MutationDataBuilder mutationDataBuilder = mapper.getRecordMutationDataBuilder(value, TIMESTAMP); + verify(mutationDataBuilder, times(1)) + .setCell( + DEFAULT_COLUMN_FAMILY, + ByteString.copyFrom("key".getBytes(StandardCharsets.UTF_8)), + TIMESTAMP, + ByteString.copyFrom(Bytes.toBytes(2L))); + + assertTotalNumberOfInvocations(mutationDataBuilder, 1); + } + + @Test + public void testMultipleOperationsAtOnce() { + Object value = fromJson("{\"a\":{\"b\":789},\"c\":true,\"x\":{\"y\":null},\"z\":null}"); + ValueMapper mapper = + new TestValueMapper(DEFAULT_COLUMN_FAMILY, DEFAULT_COLUMN, NullValueMode.DELETE); + MutationDataBuilder mutationDataBuilder = mapper.getRecordMutationDataBuilder(value, TIMESTAMP); + verify(mutationDataBuilder, times(1)) + .setCell( + "a", + ByteString.copyFrom("b".getBytes(StandardCharsets.UTF_8)), + TIMESTAMP, + ByteString.copyFrom(Bytes.toBytes(789L))); + verify(mutationDataBuilder, times(1)) + .setCell( + DEFAULT_COLUMN_FAMILY, + ByteString.copyFrom("c".getBytes(StandardCharsets.UTF_8)), + TIMESTAMP, + ByteString.copyFrom(Bytes.toBytes(true))); + verify(mutationDataBuilder, times(1)) + .deleteCells( + "x", + ByteString.copyFrom("y".getBytes(StandardCharsets.UTF_8)), + Range.TimestampRange.create(0, TIMESTAMP)); + verify(mutationDataBuilder, times(1)).deleteFamily("z"); + assertTotalNumberOfInvocations(mutationDataBuilder, 4); + } + + @Test + public void testMap() { + Object outerMapKey = 123456; + Object innerMapKey = "innerMapKey"; + String familyToBeDeleted = "familyToBeDeleted"; + String columnToBeDeleted = "columnToBeDeleted"; + ByteString columnToBeDeletedBytes = + ByteString.copyFrom(columnToBeDeleted.getBytes(StandardCharsets.UTF_8)); + Object innermostNullKey = "innermostNullKey"; + + Object value = "value"; + ByteString valueBytes = ByteString.copyFrom(((String) value).getBytes(StandardCharsets.UTF_8)); + Object valueKey = "valueKey"; + ByteString valueKeyBytes = + ByteString.copyFrom(((String) valueKey).getBytes(StandardCharsets.UTF_8)); + + Map innermostMap = new HashMap<>(); + Map innerMap = new HashMap<>(); + Map outerMap = new HashMap<>(); + + outerMap.put(outerMapKey, innerMap); + innerMap.put(innerMapKey, innermostMap); + + outerMap.put(valueKey, value); + innerMap.put(valueKey, value); + innermostMap.put(valueKey, value); + + outerMap.put(familyToBeDeleted, null); + innerMap.put(columnToBeDeleted, null); + innermostMap.put(innermostNullKey, null); + + /* + { + outerMapKey: { + innerMapKey: { + valueKey: value, + innermostNullKey: null, + } + valueKey: value, + columnToBeDeleted: null, + } + valueKey: value, + familyToBeDeleted: null, + } + */ + String expectedJsonification = "{\"innermostNullKey\":null,\"valueKey\":\"value\"}"; + ByteString expectedJsonificationBytes = + ByteString.copyFrom(expectedJsonification.getBytes(StandardCharsets.UTF_8)); + + ValueMapper mapper = + new TestValueMapper(DEFAULT_COLUMN_FAMILY, DEFAULT_COLUMN, NullValueMode.DELETE); + MutationDataBuilder mutationDataBuilder = + mapper.getRecordMutationDataBuilder(outerMap, TIMESTAMP); + verify(mutationDataBuilder, times(1)).deleteFamily(familyToBeDeleted); + verify(mutationDataBuilder, times(1)) + .setCell(DEFAULT_COLUMN_FAMILY, valueKeyBytes, TIMESTAMP, valueBytes); + verify(mutationDataBuilder, times(1)) + .deleteCells( + outerMapKey.toString(), + columnToBeDeletedBytes, + Range.TimestampRange.create(0, TIMESTAMP)); + verify(mutationDataBuilder, times(1)) + .setCell(outerMapKey.toString(), valueKeyBytes, TIMESTAMP, valueBytes); + verify(mutationDataBuilder, times(1)) + .setCell( + outerMapKey.toString(), + ByteString.copyFrom(innerMapKey.toString().getBytes(StandardCharsets.UTF_8)), + TIMESTAMP, + expectedJsonificationBytes); + assertTotalNumberOfInvocations(mutationDataBuilder, 5); + } + + @Test + public void testJsonificationOfNonJsonNativeTypes() { + final String dateFieldName = "KafkaDate"; + final String timestampFieldName = "KafkaTimestamp"; + final String timeFieldName = "KafkaTime"; + final String decimalFieldName = "KafkaDecimal"; + final String bytesFieldName = "KafkaBytes"; + final Long dateLong = 1488406838808L; + final Date date = new Date(dateLong); + final String decimalString = "0.30000000000000004"; + final Integer decimalScale = 0; + final BigDecimal decimal = new BigDecimal(decimalString); + final byte[] bytes = "bytes\0".getBytes(StandardCharsets.UTF_8); + final String schemaStructFieldName = "schema"; + final ByteString schemaStructFieldNameBytes = + ByteString.copyFrom(schemaStructFieldName.getBytes(StandardCharsets.UTF_8)); + final String schemalessMapFieldName = "schemaless"; + final ByteString schemalessMapFieldNameBytes = + ByteString.copyFrom(schemalessMapFieldName.getBytes(StandardCharsets.UTF_8)); + + Schema structSchema = + SchemaBuilder.struct() + .field(dateFieldName, org.apache.kafka.connect.data.Date.SCHEMA) + .field(timestampFieldName, org.apache.kafka.connect.data.Timestamp.SCHEMA) + .field(timeFieldName, org.apache.kafka.connect.data.Timestamp.SCHEMA) + .field(decimalFieldName, org.apache.kafka.connect.data.Decimal.schema(decimalScale)) + .field(bytesFieldName, Schema.BYTES_SCHEMA) + .build(); + Struct struct = new Struct(structSchema); + Map map = new TreeMap<>(); // Note we need this map to be ordered! + + Map outerMap = new HashMap<>(); + Map innerMap = new HashMap<>(); + + outerMap.put(DEFAULT_COLUMN_FAMILY, innerMap); + innerMap.put(schemaStructFieldName, struct); + innerMap.put(schemalessMapFieldName, map); + struct.put(dateFieldName, date); + map.put(dateFieldName, date); + struct.put(timestampFieldName, date); + map.put(timestampFieldName, date); + struct.put(timeFieldName, date); + map.put(timeFieldName, date); + struct.put(decimalFieldName, decimal); + map.put(decimalFieldName, decimal); + struct.put(bytesFieldName, bytes); + map.put(bytesFieldName, bytes); + + String expectedStringificationWithoutSchema = + "{\"KafkaBytes\":\"Ynl0ZXMA\",\"KafkaDate\":1488406838808,\"KafkaDecimal\":0.30000000000000004,\"KafkaTime\":1488406838808,\"KafkaTimestamp\":1488406838808}"; + ByteString expectedStringificationWithoutSchemaBytes = + ByteString.copyFrom(expectedStringificationWithoutSchema.getBytes(StandardCharsets.UTF_8)); + // TODO: shouldn't it be different than schemaless serialization? (e.g., count 'time' modulo + // 24h) + String expectedStringificationWithSchema = + "{\"KafkaDate\":1488406838808,\"KafkaTimestamp\":1488406838808,\"KafkaTime\":1488406838808,\"KafkaDecimal\":0.30000000000000004,\"KafkaBytes\":\"Ynl0ZXMA\"}"; + ByteString expectedStringificationWithSchemaBytes = + ByteString.copyFrom(expectedStringificationWithSchema.getBytes(StandardCharsets.UTF_8)); + + ValueMapper mapper = new TestValueMapper(null, null, NullValueMode.DELETE); + MutationDataBuilder mutationDataBuilder = + mapper.getRecordMutationDataBuilder(outerMap, TIMESTAMP); + verify(mutationDataBuilder, times(1)) + .setCell( + DEFAULT_COLUMN_FAMILY, + schemalessMapFieldNameBytes, + TIMESTAMP, + expectedStringificationWithoutSchemaBytes); + verify(mutationDataBuilder, times(1)) + .setCell( + DEFAULT_COLUMN_FAMILY, + schemaStructFieldNameBytes, + TIMESTAMP, + expectedStringificationWithSchemaBytes); + assertTotalNumberOfInvocations(mutationDataBuilder, 2); + } + + @Test + public void testStruct() { + final String structFieldName = "struct"; + final ByteString structFieldNameBytes = + ByteString.copyFrom(structFieldName.getBytes(StandardCharsets.UTF_8)); + final String valueFieldName = "value"; + final ByteString valueFieldNameBytes = + ByteString.copyFrom(valueFieldName.getBytes(StandardCharsets.UTF_8)); + final String optionalFieldName = "optional"; + final ByteString optionalFieldNameBytes = + ByteString.copyFrom(optionalFieldName.getBytes(StandardCharsets.UTF_8)); + final byte[] value = "value\0".getBytes(StandardCharsets.UTF_8); + + Schema innermostStructSchema = + SchemaBuilder.struct() + .field(valueFieldName, Schema.BYTES_SCHEMA) + .field(optionalFieldName, Schema.OPTIONAL_INT8_SCHEMA) + .build(); + Schema innerStructSchema = + SchemaBuilder.struct() + .field(structFieldName, innermostStructSchema) + .field(valueFieldName, Schema.BYTES_SCHEMA) + .field(optionalFieldName, Schema.OPTIONAL_INT8_SCHEMA) + .build(); + Schema outerStructSchema = + SchemaBuilder.struct() + .field(structFieldName, innerStructSchema) + .field(valueFieldName, Schema.BYTES_SCHEMA) + .field(optionalFieldName, Schema.OPTIONAL_INT8_SCHEMA) + .build(); + + Struct innermostStruct = new Struct(innermostStructSchema); + innermostStruct.put(valueFieldName, value); + + String expectedInnermostStringification = "{\"value\":\"dmFsdWUA\",\"optional\":null}"; + ByteString expectedInnermostStringificationBytes = + ByteString.copyFrom(expectedInnermostStringification.getBytes(StandardCharsets.UTF_8)); + + Struct innerStruct = new Struct(innerStructSchema); + innerStruct.put(structFieldName, innermostStruct); + innerStruct.put(valueFieldName, value); + + Struct struct = new Struct(outerStructSchema); + struct.put(structFieldName, innerStruct); + struct.put(valueFieldName, value); + + /* + { + struct: { + struct: { + optionalFieldName: null, + valueFieldName: value, + } + optionalFieldName: null, + valueFieldName: value, + } + optionalFieldName: null, + valueFieldName: value, + } + */ + ValueMapper mapper = + new TestValueMapper(DEFAULT_COLUMN_FAMILY, DEFAULT_COLUMN, NullValueMode.DELETE); + MutationDataBuilder mutationDataBuilder = + mapper.getRecordMutationDataBuilder(struct, TIMESTAMP); + verify(mutationDataBuilder, times(1)) + .setCell(DEFAULT_COLUMN_FAMILY, valueFieldNameBytes, TIMESTAMP, ByteString.copyFrom(value)); + verify(mutationDataBuilder, times(1)).deleteFamily(optionalFieldName); + verify(mutationDataBuilder, times(1)) + .setCell(structFieldName, valueFieldNameBytes, TIMESTAMP, ByteString.copyFrom(value)); + verify(mutationDataBuilder, times(1)) + .deleteCells( + structFieldName, optionalFieldNameBytes, Range.TimestampRange.create(0, TIMESTAMP)); + verify(mutationDataBuilder, times(1)) + .setCell( + structFieldName, + structFieldNameBytes, + TIMESTAMP, + expectedInnermostStringificationBytes); + assertTotalNumberOfInvocations(mutationDataBuilder, 5); + assertTrue(mutationDataBuilder.maybeBuild(TARGET_TABLE_NAME, ROW_KEY).isPresent()); + } + + @Test + public void testEmpty() { + Schema emptyStructSchema = SchemaBuilder.struct().build(); + Struct emptyStruct = new Struct(emptyStructSchema); + Map emptyMap = new HashMap<>(); + ValueMapper mapper = + new TestValueMapper(DEFAULT_COLUMN_FAMILY, DEFAULT_COLUMN, NullValueMode.WRITE); + for (Object value : List.of(emptyMap, emptyStruct)) { + MutationDataBuilder mutationDataBuilder = + mapper.getRecordMutationDataBuilder(value, TIMESTAMP); + assertTotalNumberOfInvocations(mutationDataBuilder, 0); + assertTrue(mutationDataBuilder.maybeBuild(TARGET_TABLE_NAME, ROW_KEY).isEmpty()); + } + } + + @Test + public void testSimpleCase1() { + Object value = fromJson("{\"foo\": {\"bar\": 1}}"); + ValueMapper mapper = new TestValueMapper(null, null, NullValueMode.IGNORE); + MutationDataBuilder mutationDataBuilder = mapper.getRecordMutationDataBuilder(value, TIMESTAMP); + verify(mutationDataBuilder, times(1)) + .setCell( + "foo", + ByteString.copyFrom("bar".getBytes(StandardCharsets.UTF_8)), + TIMESTAMP, + ByteString.copyFrom(Bytes.toBytes(1L))); + assertTotalNumberOfInvocations(mutationDataBuilder, 1); + assertTrue(mutationDataBuilder.maybeBuild(TARGET_TABLE_NAME, ROW_KEY).isPresent()); + } + + @Test + public void testSimpleCase2() { + Object value = fromJson("{\"foo\": {\"bar\": {\"fizz\": 1}}}"); + ValueMapper mapper = new TestValueMapper(null, null, NullValueMode.IGNORE); + MutationDataBuilder mutationDataBuilder = mapper.getRecordMutationDataBuilder(value, TIMESTAMP); + verify(mutationDataBuilder, times(1)) + .setCell( + "foo", + ByteString.copyFrom("bar".getBytes(StandardCharsets.UTF_8)), + TIMESTAMP, + ByteString.copyFrom("{\"fizz\":1}".getBytes(StandardCharsets.UTF_8))); + assertTrue(mutationDataBuilder.maybeBuild(TARGET_TABLE_NAME, ROW_KEY).isPresent()); + } + + @Test + public void testSimpleCase3() { + Object value = fromJson("{\"foo\": 1}"); + ValueMapper mapper = new TestValueMapper(DEFAULT_COLUMN_FAMILY, null, NullValueMode.IGNORE); + MutationDataBuilder mutationDataBuilder = mapper.getRecordMutationDataBuilder(value, TIMESTAMP); + verify(mutationDataBuilder, times(1)) + .setCell( + DEFAULT_COLUMN_FAMILY, + ByteString.copyFrom("foo".getBytes(StandardCharsets.UTF_8)), + TIMESTAMP, + ByteString.copyFrom(Bytes.toBytes(1L))); + assertTotalNumberOfInvocations(mutationDataBuilder, 1); + assertTrue(mutationDataBuilder.maybeBuild(TARGET_TABLE_NAME, ROW_KEY).isPresent()); + } + + @Test + public void testNullModeIgnoreRoot() { + ValueMapper mapper = + new TestValueMapper(DEFAULT_COLUMN_FAMILY, DEFAULT_COLUMN, NullValueMode.IGNORE); + MutationDataBuilder mutationDataBuilder = mapper.getRecordMutationDataBuilder(null, TIMESTAMP); + assertTotalNumberOfInvocations(mutationDataBuilder, 0); + assertTrue(mutationDataBuilder.maybeBuild(TARGET_TABLE_NAME, ROW_KEY).isEmpty()); + } + + @Test + public void testNullModeIgnoreNestedOnce() { + ValueMapper mapper = + new TestValueMapper(DEFAULT_COLUMN_FAMILY, DEFAULT_COLUMN, NullValueMode.IGNORE); + MutationDataBuilder mutationDataBuilder = + mapper.getRecordMutationDataBuilder(getStructhWithNullOnNthNestingLevel(1), TIMESTAMP); + assertTotalNumberOfInvocations(mutationDataBuilder, 0); + assertTrue(mutationDataBuilder.maybeBuild(TARGET_TABLE_NAME, ROW_KEY).isEmpty()); + } + + @Test + public void testNullModeIgnoreNestedTwice() { + ValueMapper mapper = + new TestValueMapper(DEFAULT_COLUMN_FAMILY, DEFAULT_COLUMN, NullValueMode.IGNORE); + MutationDataBuilder mutationDataBuilder = + mapper.getRecordMutationDataBuilder(getStructhWithNullOnNthNestingLevel(2), TIMESTAMP); + assertTotalNumberOfInvocations(mutationDataBuilder, 0); + assertTrue(mutationDataBuilder.maybeBuild(TARGET_TABLE_NAME, ROW_KEY).isEmpty()); + } + + @Test + public void testNullModeWriteRoot() { + ValueMapper mapper = + new TestValueMapper(DEFAULT_COLUMN_FAMILY, DEFAULT_COLUMN, NullValueMode.WRITE); + MutationDataBuilder mutationDataBuilder = mapper.getRecordMutationDataBuilder(null, TIMESTAMP); + verify(mutationDataBuilder, times(1)) + .setCell(DEFAULT_COLUMN_FAMILY, DEFAULT_COLUMN_BYTES, TIMESTAMP, ByteString.empty()); + assertTotalNumberOfInvocations(mutationDataBuilder, 1); + assertTrue(mutationDataBuilder.maybeBuild(TARGET_TABLE_NAME, ROW_KEY).isPresent()); + } + + @Test + public void testNullModeWriteNestedOnce() { + ValueMapper mapper = + new TestValueMapper(DEFAULT_COLUMN_FAMILY, DEFAULT_COLUMN, NullValueMode.WRITE); + MutationDataBuilder mutationDataBuilder = + mapper.getRecordMutationDataBuilder(getStructhWithNullOnNthNestingLevel(1), TIMESTAMP); + verify(mutationDataBuilder, times(1)) + .setCell( + DEFAULT_COLUMN_FAMILY, + NESTED_NULL_STRUCT_FIELD_NAME_BYTES, + TIMESTAMP, + ByteString.empty()); + assertTotalNumberOfInvocations(mutationDataBuilder, 1); + assertTrue(mutationDataBuilder.maybeBuild(TARGET_TABLE_NAME, ROW_KEY).isPresent()); + } + + @Test + public void testNullModeWriteNestedTwice() { + ValueMapper mapper = + new TestValueMapper(DEFAULT_COLUMN_FAMILY, DEFAULT_COLUMN, NullValueMode.WRITE); + MutationDataBuilder mutationDataBuilder = + mapper.getRecordMutationDataBuilder(getStructhWithNullOnNthNestingLevel(2), TIMESTAMP); + verify(mutationDataBuilder, times(1)) + .setCell( + NESTED_NULL_STRUCT_FIELD_NAME, + NESTED_NULL_STRUCT_FIELD_NAME_BYTES, + TIMESTAMP, + ByteString.empty()); + assertTotalNumberOfInvocations(mutationDataBuilder, 1); + assertTrue(mutationDataBuilder.maybeBuild(TARGET_TABLE_NAME, ROW_KEY).isPresent()); + } + + @Test + public void testNullModeDeleteRoot() { + ValueMapper mapper = + new TestValueMapper(DEFAULT_COLUMN_FAMILY, DEFAULT_COLUMN, NullValueMode.DELETE); + MutationDataBuilder mutationDataBuilder = mapper.getRecordMutationDataBuilder(null, TIMESTAMP); + verify(mutationDataBuilder, times(1)).deleteRow(); + assertTotalNumberOfInvocations(mutationDataBuilder, 1); + assertTrue(mutationDataBuilder.maybeBuild(TARGET_TABLE_NAME, ROW_KEY).isPresent()); + } + + @Test + public void testNullModeDeleteNestedOnce() { + ValueMapper mapper = + new TestValueMapper(DEFAULT_COLUMN_FAMILY, DEFAULT_COLUMN, NullValueMode.DELETE); + MutationDataBuilder mutationDataBuilder = + mapper.getRecordMutationDataBuilder(getStructhWithNullOnNthNestingLevel(1), TIMESTAMP); + verify(mutationDataBuilder, times(1)).deleteFamily(NESTED_NULL_STRUCT_FIELD_NAME); + assertTotalNumberOfInvocations(mutationDataBuilder, 1); + assertTrue(mutationDataBuilder.maybeBuild(TARGET_TABLE_NAME, ROW_KEY).isPresent()); + } + + @Test + public void testNullModeDeleteNestedTwice() { + ValueMapper mapper = + new TestValueMapper(DEFAULT_COLUMN_FAMILY, DEFAULT_COLUMN, NullValueMode.DELETE); + MutationDataBuilder mutationDataBuilder = + mapper.getRecordMutationDataBuilder(getStructhWithNullOnNthNestingLevel(2), TIMESTAMP); + verify(mutationDataBuilder, times(1)) + .deleteCells( + NESTED_NULL_STRUCT_FIELD_NAME, NESTED_NULL_STRUCT_FIELD_NAME_BYTES, TIMESTAMP_RANGE); + assertTotalNumberOfInvocations(mutationDataBuilder, 1); + assertTrue(mutationDataBuilder.maybeBuild(TARGET_TABLE_NAME, ROW_KEY).isPresent()); + } + + @Test + public void testNullModeNestedThrice() { + ValueMapper mapper = + new TestValueMapper(DEFAULT_COLUMN_FAMILY, DEFAULT_COLUMN, NullValueMode.IGNORE); + String expectedJsonification = "{\"struct\":null}"; + ByteString expectedJsonificationBytes = + ByteString.copyFrom(expectedJsonification.getBytes(StandardCharsets.UTF_8)); + MutationDataBuilder mutationDataBuilder = + mapper.getRecordMutationDataBuilder(getStructhWithNullOnNthNestingLevel(3), TIMESTAMP); + verify(mutationDataBuilder, times(1)) + .setCell( + NESTED_NULL_STRUCT_FIELD_NAME, + NESTED_NULL_STRUCT_FIELD_NAME_BYTES, + TIMESTAMP, + expectedJsonificationBytes); + assertTotalNumberOfInvocations(mutationDataBuilder, 1); + assertTrue(mutationDataBuilder.maybeBuild(TARGET_TABLE_NAME, ROW_KEY).isPresent()); + } + + private static Struct getStructhWithNullOnNthNestingLevel(int n) { + assert n > 0; + + Schema schema = + SchemaBuilder.struct() + .field(NESTED_NULL_STRUCT_FIELD_NAME, SchemaBuilder.struct().optional()) + .build(); + // We consider a Struct with a null child to be a level 1 nested struct. + Struct struct = new Struct(schema); + + while (n > 1) { + n -= 1; + schema = + SchemaBuilder.struct().field(NESTED_NULL_STRUCT_FIELD_NAME, schema).optional().build(); + final Struct outerStruct = new Struct(schema); + outerStruct.put(NESTED_NULL_STRUCT_FIELD_NAME, struct); + struct = outerStruct; + } + return struct; + } + + private static Object fromJson(String s) { + return jsonConverter.toConnectData(DEFAULT_TOPIC, s.getBytes(StandardCharsets.UTF_8)).value(); + } + + private static class TestValueMapper extends ValueMapper { + public TestValueMapper( + String defaultColumnFamily, String defaultColumnQualifier, NullValueMode nullMode) { + super(defaultColumnFamily, defaultColumnQualifier, nullMode); + } + + @Override + protected MutationDataBuilder createMutationDataBuilder() { + return spy(super.createMutationDataBuilder()); + } + } +} diff --git a/google-cloud-bigtable-kafka-connect-sink/src/test/java/com/google/cloud/kafka/connect/bigtable/util/ApiExceptionFactory.java b/google-cloud-bigtable-kafka-connect-sink/src/test/java/com/google/cloud/kafka/connect/bigtable/util/ApiExceptionFactory.java new file mode 100644 index 0000000000..f09740dbe4 --- /dev/null +++ b/google-cloud-bigtable-kafka-connect-sink/src/test/java/com/google/cloud/kafka/connect/bigtable/util/ApiExceptionFactory.java @@ -0,0 +1,35 @@ +/* + * Copyright 2024 Google LLC + * + * Licensed under the Apache License, Version 2.0 (the "License"); + * you may not use this file except in compliance with the License. + * You may obtain a copy of the License at + * + * https://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ +package com.google.cloud.kafka.connect.bigtable.util; + +import com.google.api.gax.grpc.GrpcStatusCode; +import com.google.api.gax.rpc.ApiException; +import com.google.api.gax.rpc.StatusCode; +import io.grpc.Status; + +public class ApiExceptionFactory { + public static ApiException create() { + return create(Status.Code.NOT_FOUND); + } + + public static ApiException create(Status.Code code) { + return create(new Throwable(), GrpcStatusCode.of(code), true); + } + + public static ApiException create(Throwable cause, StatusCode code, boolean retryable) { + return com.google.api.gax.rpc.ApiExceptionFactory.createException(cause, code, retryable); + } +} diff --git a/google-cloud-bigtable-kafka-connect-sink/src/test/java/com/google/cloud/kafka/connect/bigtable/util/BasicPropertiesFactory.java b/google-cloud-bigtable-kafka-connect-sink/src/test/java/com/google/cloud/kafka/connect/bigtable/util/BasicPropertiesFactory.java new file mode 100644 index 0000000000..3cdc7a2a7d --- /dev/null +++ b/google-cloud-bigtable-kafka-connect-sink/src/test/java/com/google/cloud/kafka/connect/bigtable/util/BasicPropertiesFactory.java @@ -0,0 +1,36 @@ +/* + * Copyright 2024 Google LLC + * + * Licensed under the Apache License, Version 2.0 (the "License"); + * you may not use this file except in compliance with the License. + * You may obtain a copy of the License at + * + * https://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ +package com.google.cloud.kafka.connect.bigtable.util; + +import com.google.cloud.kafka.connect.bigtable.config.BigtableSinkConfig; +import com.google.cloud.kafka.connect.bigtable.config.BigtableSinkTaskConfig; +import java.util.HashMap; +import java.util.Map; + +public class BasicPropertiesFactory { + public static Map getSinkProps() { + Map props = new HashMap<>(); + props.put(BigtableSinkConfig.CONFIG_GCP_PROJECT_ID, "project"); + props.put(BigtableSinkConfig.CONFIG_BIGTABLE_INSTANCE_ID, "instance"); + return props; + } + + public static Map getTaskProps() { + Map props = getSinkProps(); + props.put(BigtableSinkTaskConfig.CONFIG_TASK_ID, "1"); + return props; + } +} diff --git a/google-cloud-bigtable-kafka-connect-sink/src/test/java/com/google/cloud/kafka/connect/bigtable/util/FutureUtil.java b/google-cloud-bigtable-kafka-connect-sink/src/test/java/com/google/cloud/kafka/connect/bigtable/util/FutureUtil.java new file mode 100644 index 0000000000..a4d4c7875d --- /dev/null +++ b/google-cloud-bigtable-kafka-connect-sink/src/test/java/com/google/cloud/kafka/connect/bigtable/util/FutureUtil.java @@ -0,0 +1,33 @@ +/* + * Copyright 2024 Google LLC + * + * Licensed under the Apache License, Version 2.0 (the "License"); + * you may not use this file except in compliance with the License. + * You may obtain a copy of the License at + * + * https://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ +package com.google.cloud.kafka.connect.bigtable.util; + +import com.google.api.core.ApiFuture; +import com.google.api.core.SettableApiFuture; + +public class FutureUtil { + public static ApiFuture completedApiFuture(T value) { + SettableApiFuture future = SettableApiFuture.create(); + future.set(value); + return future; + } + + public static ApiFuture failedApiFuture(Exception exception) { + SettableApiFuture future = SettableApiFuture.create(); + future.setException(exception); + return future; + } +} diff --git a/google-cloud-bigtable-kafka-connect-sink/src/test/java/com/google/cloud/kafka/connect/bigtable/util/JsonConverterFactory.java b/google-cloud-bigtable-kafka-connect-sink/src/test/java/com/google/cloud/kafka/connect/bigtable/util/JsonConverterFactory.java new file mode 100644 index 0000000000..4192fec3cc --- /dev/null +++ b/google-cloud-bigtable-kafka-connect-sink/src/test/java/com/google/cloud/kafka/connect/bigtable/util/JsonConverterFactory.java @@ -0,0 +1,29 @@ +/* + * Copyright 2024 Google LLC + * + * Licensed under the Apache License, Version 2.0 (the "License"); + * you may not use this file except in compliance with the License. + * You may obtain a copy of the License at + * + * https://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ +package com.google.cloud.kafka.connect.bigtable.util; + +import java.util.Map; +import org.apache.kafka.connect.json.JsonConverter; + +public class JsonConverterFactory { + public static JsonConverter create(boolean schemasEnable, boolean isKafkaKeyConverter) { + Map jsonConverterProps = + Map.of("schemas.enable", Boolean.toString(schemasEnable)); + JsonConverter jsonConverter = new JsonConverter(); + jsonConverter.configure(jsonConverterProps, isKafkaKeyConverter); + return jsonConverter; + } +} diff --git a/google-cloud-bigtable-kafka-connect-sink/src/test/java/com/google/cloud/kafka/connect/bigtable/util/MockUtil.java b/google-cloud-bigtable-kafka-connect-sink/src/test/java/com/google/cloud/kafka/connect/bigtable/util/MockUtil.java new file mode 100644 index 0000000000..03f7534005 --- /dev/null +++ b/google-cloud-bigtable-kafka-connect-sink/src/test/java/com/google/cloud/kafka/connect/bigtable/util/MockUtil.java @@ -0,0 +1,29 @@ +/* + * Copyright 2024 Google LLC + * + * Licensed under the Apache License, Version 2.0 (the "License"); + * you may not use this file except in compliance with the License. + * You may obtain a copy of the License at + * + * https://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ +package com.google.cloud.kafka.connect.bigtable.util; + +import static org.junit.Assert.assertEquals; +import static org.mockito.Mockito.mockingDetails; + +import java.util.Collection; +import org.mockito.invocation.Invocation; + +public class MockUtil { + public static void assertTotalNumberOfInvocations(Object mock, int expected) { + Collection invocations = mockingDetails(mock).getInvocations(); + assertEquals(expected, invocations.size()); + } +} diff --git a/google-cloud-bigtable-kafka-connect-sink/src/test/java/com/google/cloud/kafka/connect/bigtable/util/TestId.java b/google-cloud-bigtable-kafka-connect-sink/src/test/java/com/google/cloud/kafka/connect/bigtable/util/TestId.java new file mode 100644 index 0000000000..f32e8a63d1 --- /dev/null +++ b/google-cloud-bigtable-kafka-connect-sink/src/test/java/com/google/cloud/kafka/connect/bigtable/util/TestId.java @@ -0,0 +1,32 @@ +/* + * Copyright 2025 Google LLC + * + * Licensed under the Apache License, Version 2.0 (the "License"); + * you may not use this file except in compliance with the License. + * You may obtain a copy of the License at + * + * https://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ +package com.google.cloud.kafka.connect.bigtable.util; + +import com.google.common.collect.Streams; + +public class TestId { + public static String getTestClassId(Class testClass) { + return testClass.getSimpleName(); + } + + public static String getTestCaseId(Class testClass) { + StackWalker.StackFrame frame = + StackWalker.getInstance(StackWalker.Option.RETAIN_CLASS_REFERENCE) + .walk(s -> Streams.findLast(s.filter(f -> f.getDeclaringClass().equals(testClass)))) + .get(); + return getTestClassId(frame.getDeclaringClass()) + frame.getMethodName(); + } +} diff --git a/google-cloud-bigtable-kafka-connect-sink/src/test/resources/fake_service_key.json b/google-cloud-bigtable-kafka-connect-sink/src/test/resources/fake_service_key.json new file mode 100644 index 0000000000..f3eac606f0 --- /dev/null +++ b/google-cloud-bigtable-kafka-connect-sink/src/test/resources/fake_service_key.json @@ -0,0 +1,9 @@ +{ + "type": "service_account", + "private_key_id": "abc", + "private_key": "-----BEGIN PRIVATE KEY-----\nMIIEvgIBADANBgkqhkiG9w0BAQEFAASCBKgwggSkAgEAAoIBAQDY3E8o1NEFcjMM\nHW/5ZfFJw29/8NEqpViNjQIx95Xx5KDtJ+nWn9+OW0uqsSqKlKGhAdAo+Q6bjx2c\nuXVsXTu7XrZUY5Kltvj94DvUa1wjNXs606r/RxWTJ58bfdC+gLLxBfGnB6CwK0YQ\nxnfpjNbkUfVVzO0MQD7UP0Hl5ZcY0Puvxd/yHuONQn/rIAieTHH1pqgW+zrH/y3c\n59IGThC9PPtugI9ea8RSnVj3PWz1bX2UkCDpy9IRh9LzJLaYYX9RUd7++dULUlat\nAaXBh1U6emUDzhrIsgApjDVtimOPbmQWmX1S60mqQikRpVYZ8u+NDD+LNw+/Eovn\nxCj2Y3z1AgMBAAECggEAWDBzoqO1IvVXjBA2lqId10T6hXmN3j1ifyH+aAqK+FVl\nGjyWjDj0xWQcJ9ync7bQ6fSeTeNGzP0M6kzDU1+w6FgyZqwdmXWI2VmEizRjwk+/\n/uLQUcL7I55Dxn7KUoZs/rZPmQDxmGLoue60Gg6z3yLzVcKiDc7cnhzhdBgDc8vd\nQorNAlqGPRnm3EqKQ6VQp6fyQmCAxrr45kspRXNLddat3AMsuqImDkqGKBmF3Q1y\nxWGe81LphUiRqvqbyUlh6cdSZ8pLBpc9m0c3qWPKs9paqBIvgUPlvOZMqec6x4S6\nChbdkkTRLnbsRr0Yg/nDeEPlkhRBhasXpxpMUBgPywKBgQDs2axNkFjbU94uXvd5\nznUhDVxPFBuxyUHtsJNqW4p/ujLNimGet5E/YthCnQeC2P3Ym7c3fiz68amM6hiA\nOnW7HYPZ+jKFnefpAtjyOOs46AkftEg07T9XjwWNPt8+8l0DYawPoJgbM5iE0L2O\nx8TU1Vs4mXc+ql9F90GzI0x3VwKBgQDqZOOqWw3hTnNT07Ixqnmd3dugV9S7eW6o\nU9OoUgJB4rYTpG+yFqNqbRT8bkx37iKBMEReppqonOqGm4wtuRR6LSLlgcIU9Iwx\nyfH12UWqVmFSHsgZFqM/cK3wGev38h1WBIOx3/djKn7BdlKVh8kWyx6uC8bmV+E6\nOoK0vJD6kwKBgHAySOnROBZlqzkiKW8c+uU2VATtzJSydrWm0J4wUPJifNBa/hVW\ndcqmAzXC9xznt5AVa3wxHBOfyKaE+ig8CSsjNyNZ3vbmr0X04FoV1m91k2TeXNod\njMTobkPThaNm4eLJMN2SQJuaHGTGERWC0l3T18t+/zrDMDCPiSLX1NAvAoGBAN1T\nVLJYdjvIMxf1bm59VYcepbK7HLHFkRq6xMJMZbtG0ryraZjUzYvB4q4VjHk2UDiC\nlhx13tXWDZH7MJtABzjyg+AI7XWSEQs2cBXACos0M4Myc6lU+eL+iA+OuoUOhmrh\nqmT8YYGu76/IBWUSqWuvcpHPpwl7871i4Ga/I3qnAoGBANNkKAcMoeAbJQK7a/Rn\nwPEJB+dPgNDIaboAsh1nZhVhN5cvdvCWuEYgOGCPQLYQF0zmTLcM+sVxOYgfy8mV\nfbNgPgsP5xmu6dw2COBKdtozw0HrWSRjACd1N4yGu75+wPCcX/gQarcjRcXXZeEa\nNtBLSfcqPULqD+h7br9lEJio\n-----END PRIVATE KEY-----\n", + "client_email": "123-abc@developer.gserviceaccount.com", + "client_id": "123-abc.apps.googleusercontent.com", + "auth_uri": "https://accounts.google.com/o/oauth2/auth", + "token_uri": "http://localhost:8080/token" +} \ No newline at end of file From 3738df2afbb2598de2633f56052be3536019c0b5 Mon Sep 17 00:00:00 2001 From: Adam Czajkowski Date: Thu, 23 Jan 2025 13:05:52 +0100 Subject: [PATCH 02/76] Elaborate on error handling in awaitResourceCreationAndHandleInvalidInputErrors() --- .../connect/bigtable/autocreate/BigtableSchemaManager.java | 4 +++- 1 file changed, 3 insertions(+), 1 deletion(-) diff --git a/google-cloud-bigtable-kafka-connect-sink/src/main/java/com/google/cloud/kafka/connect/bigtable/autocreate/BigtableSchemaManager.java b/google-cloud-bigtable-kafka-connect-sink/src/main/java/com/google/cloud/kafka/connect/bigtable/autocreate/BigtableSchemaManager.java index 7db1f31104..f7d20282ab 100644 --- a/google-cloud-bigtable-kafka-connect-sink/src/main/java/com/google/cloud/kafka/connect/bigtable/autocreate/BigtableSchemaManager.java +++ b/google-cloud-bigtable-kafka-connect-sink/src/main/java/com/google/cloud/kafka/connect/bigtable/autocreate/BigtableSchemaManager.java @@ -365,7 +365,9 @@ private ApiFuture

createColumnFamily(Map.Entry tableNameA * invalid, it is assumed that input {@link SinkRecord SinkRecord(s)} map to invalid values, * so all the {@link SinkRecord SinkRecord(s)} needing the resource whose creation failed * are returned. - *
  • Other resource creation errors are logged. + *
  • Other resource creation errors are only logged. A different section of code is + * responsible for checking whether the resources exist despite these futures' errors. This + * way all errors not caused by invalid input can be handled generally. * * * @param createdColumnFamilyFuturesAndRecords {@link Map} of {@link ApiFuture ApiFuture(s)} and From d488244c59cac92d5a6c59812611ebf84b97ef30 Mon Sep 17 00:00:00 2001 From: Adam Czajkowski Date: Thu, 23 Jan 2025 13:12:06 +0100 Subject: [PATCH 03/76] rename createdColumnFamilyFuturesAndRecords --- .../connect/bigtable/autocreate/BigtableSchemaManager.java | 6 +++--- 1 file changed, 3 insertions(+), 3 deletions(-) diff --git a/google-cloud-bigtable-kafka-connect-sink/src/main/java/com/google/cloud/kafka/connect/bigtable/autocreate/BigtableSchemaManager.java b/google-cloud-bigtable-kafka-connect-sink/src/main/java/com/google/cloud/kafka/connect/bigtable/autocreate/BigtableSchemaManager.java index f7d20282ab..5957be5873 100644 --- a/google-cloud-bigtable-kafka-connect-sink/src/main/java/com/google/cloud/kafka/connect/bigtable/autocreate/BigtableSchemaManager.java +++ b/google-cloud-bigtable-kafka-connect-sink/src/main/java/com/google/cloud/kafka/connect/bigtable/autocreate/BigtableSchemaManager.java @@ -370,7 +370,7 @@ private ApiFuture
  • createColumnFamily(Map.Entry tableNameA * way all errors not caused by invalid input can be handled generally. * * - * @param createdColumnFamilyFuturesAndRecords {@link Map} of {@link ApiFuture ApiFuture(s)} and + * @param resourceCreationFuturesAndRecords {@link Map} of {@link ApiFuture ApiFuture(s)} and * information what resource is created and for which {@link SinkRecord SinkRecord(s)}. * @param errorMessageTemplate The Java format string template of error message with which Cloud * Bigtable exceptions for valid input data are logged. @@ -381,10 +381,10 @@ private ApiFuture
    createColumnFamily(Map.Entry tableNameA */ @VisibleForTesting , Id> Set awaitResourceCreationAndHandleInvalidInputErrors( - Map> createdColumnFamilyFuturesAndRecords, + Map> resourceCreationFuturesAndRecords, String errorMessageTemplate) { Set dataErrors = new HashSet<>(); - createdColumnFamilyFuturesAndRecords.forEach( + resourceCreationFuturesAndRecords.forEach( (fut, resourceAndRecords) -> { Object resource = resourceAndRecords.getResource(); List sinkRecords = resourceAndRecords.getRecords(); From 8f254ef4f86066d4163cb2b20d046995c72d7c1a Mon Sep 17 00:00:00 2001 From: Adam Czajkowski Date: Thu, 23 Jan 2025 13:19:57 +0100 Subject: [PATCH 04/76] Tweak NullValueMode docstring --- .../cloud/kafka/connect/bigtable/config/BigtableSinkConfig.java | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/google-cloud-bigtable-kafka-connect-sink/src/main/java/com/google/cloud/kafka/connect/bigtable/config/BigtableSinkConfig.java b/google-cloud-bigtable-kafka-connect-sink/src/main/java/com/google/cloud/kafka/connect/bigtable/config/BigtableSinkConfig.java index 0fe27aff4a..5fb5a21deb 100644 --- a/google-cloud-bigtable-kafka-connect-sink/src/main/java/com/google/cloud/kafka/connect/bigtable/config/BigtableSinkConfig.java +++ b/google-cloud-bigtable-kafka-connect-sink/src/main/java/com/google/cloud/kafka/connect/bigtable/config/BigtableSinkConfig.java @@ -247,7 +247,7 @@ public static ConfigDef getDefinition() { DEFAULT_NULL_VALUE_MODE.name(), enumValidator(NullValueMode.values()), ConfigDef.Importance.MEDIUM, - "Defines what to do with `null` Kafka values. Supported modes are:" + "Defines what to do with `null`s within Kafka values. Supported modes are:" + "\n- write - Serialize `null`s to empty byte arrays." + "\n- ignore - Ignore `null`s." + "\n- delete - Use them to issue DELETE commands. Root-level `null` deletes a" From 31458bc31a22fcf4d6bd54015b9fab83daac3f91 Mon Sep 17 00:00:00 2001 From: Adam Czajkowski Date: Thu, 23 Jan 2025 13:30:14 +0100 Subject: [PATCH 05/76] Rename test cases valdiating multiple config parameters --- .../kafka/connect/bigtable/config/BigtableSinkConfigTest.java | 4 ++-- 1 file changed, 2 insertions(+), 2 deletions(-) diff --git a/google-cloud-bigtable-kafka-connect-sink/src/test/java/com/google/cloud/kafka/connect/bigtable/config/BigtableSinkConfigTest.java b/google-cloud-bigtable-kafka-connect-sink/src/test/java/com/google/cloud/kafka/connect/bigtable/config/BigtableSinkConfigTest.java index a2c4629248..b22a5df2bb 100644 --- a/google-cloud-bigtable-kafka-connect-sink/src/test/java/com/google/cloud/kafka/connect/bigtable/config/BigtableSinkConfigTest.java +++ b/google-cloud-bigtable-kafka-connect-sink/src/test/java/com/google/cloud/kafka/connect/bigtable/config/BigtableSinkConfigTest.java @@ -87,7 +87,7 @@ public void testDefaults() { } @Test - public void testMultipleValuesValidationInsert() { + public void testInsertModeOnlyAllowsMaxBatchSizeOf1() { Map props = BasicPropertiesFactory.getSinkProps(); props.put(BigtableSinkConfig.CONFIG_INSERT_MODE, InsertMode.INSERT.name()); props.put(BigtableSinkConfig.CONFIG_MAX_BATCH_SIZE, "2"); @@ -96,7 +96,7 @@ public void testMultipleValuesValidationInsert() { } @Test - public void testMultipleValuesValidationCredentials() { + public void testMultipleCredentialsAreDisallowed() { Map props = BasicPropertiesFactory.getSinkProps(); props.put(BigtableSinkConfig.CONFIG_GCP_CREDENTIALS_JSON, "nonempty"); props.put(BigtableSinkConfig.CONFIG_GCP_CREDENTIALS_PATH, "nonempty"); From e79507eea07f250bbc90426a9667e47becc3dbac Mon Sep 17 00:00:00 2001 From: Adam Czajkowski Date: Thu, 23 Jan 2025 14:25:27 +0100 Subject: [PATCH 06/76] Add Map ordering disclaimer --- .../google/cloud/kafka/connect/bigtable/mapping/KeyMapper.java | 1 + 1 file changed, 1 insertion(+) diff --git a/google-cloud-bigtable-kafka-connect-sink/src/main/java/com/google/cloud/kafka/connect/bigtable/mapping/KeyMapper.java b/google-cloud-bigtable-kafka-connect-sink/src/main/java/com/google/cloud/kafka/connect/bigtable/mapping/KeyMapper.java index 81436dc906..179dd00c25 100644 --- a/google-cloud-bigtable-kafka-connect-sink/src/main/java/com/google/cloud/kafka/connect/bigtable/mapping/KeyMapper.java +++ b/google-cloud-bigtable-kafka-connect-sink/src/main/java/com/google/cloud/kafka/connect/bigtable/mapping/KeyMapper.java @@ -187,6 +187,7 @@ private static byte[] serializeKeyElement(Object keyElement) { Map map = (Map) keyElement; return concatenateByteArrays( "{", + // Note that it inherits ordering of entries from the configured converter. map.entrySet().stream() .map( e -> From 6ebc75a4edf862528fd3f88ee160a11e5f8d3731 Mon Sep 17 00:00:00 2001 From: Adam Czajkowski Date: Thu, 23 Jan 2025 14:37:05 +0100 Subject: [PATCH 07/76] Get rid of interpolationish suffix from the record topic's name in testGetTableName --- .../cloud/kafka/connect/bigtable/BigtableSinkTaskTest.java | 5 ++--- 1 file changed, 2 insertions(+), 3 deletions(-) diff --git a/google-cloud-bigtable-kafka-connect-sink/src/test/java/com/google/cloud/kafka/connect/bigtable/BigtableSinkTaskTest.java b/google-cloud-bigtable-kafka-connect-sink/src/test/java/com/google/cloud/kafka/connect/bigtable/BigtableSinkTaskTest.java index 3348d4bf60..e103511108 100644 --- a/google-cloud-bigtable-kafka-connect-sink/src/test/java/com/google/cloud/kafka/connect/bigtable/BigtableSinkTaskTest.java +++ b/google-cloud-bigtable-kafka-connect-sink/src/test/java/com/google/cloud/kafka/connect/bigtable/BigtableSinkTaskTest.java @@ -160,11 +160,10 @@ public void testVersion() { @Test public void testGetTableName() { - SinkRecord record = new SinkRecord("topic${test}", 1, null, null, null, null, 1); + SinkRecord record = new SinkRecord("topic", 1, null, null, null, null, 1); for (Map.Entry test : List.of( - new AbstractMap.SimpleImmutableEntry<>( - "prefix_${topic}_suffix", "prefix_topic${test}_suffix"), + new AbstractMap.SimpleImmutableEntry<>("prefix_${topic}_suffix", "prefix_topic_suffix"), new AbstractMap.SimpleImmutableEntry<>( "prefix_${topic_suffix", "prefix_${topic_suffix"), new AbstractMap.SimpleImmutableEntry<>("prefix_$topic_suffix", "prefix_$topic_suffix"), From 1a70230b400b2bace19b7ca09ac6e95355fa0037 Mon Sep 17 00:00:00 2001 From: Adam Czajkowski Date: Thu, 23 Jan 2025 14:44:05 +0100 Subject: [PATCH 08/76] Remove mocks from testCreateRecordMutationDataNonemptyKey --- .../bigtable/BigtableSinkTaskTest.java | 28 +++++-------------- 1 file changed, 7 insertions(+), 21 deletions(-) diff --git a/google-cloud-bigtable-kafka-connect-sink/src/test/java/com/google/cloud/kafka/connect/bigtable/BigtableSinkTaskTest.java b/google-cloud-bigtable-kafka-connect-sink/src/test/java/com/google/cloud/kafka/connect/bigtable/BigtableSinkTaskTest.java index e103511108..be22c82a82 100644 --- a/google-cloud-bigtable-kafka-connect-sink/src/test/java/com/google/cloud/kafka/connect/bigtable/BigtableSinkTaskTest.java +++ b/google-cloud-bigtable-kafka-connect-sink/src/test/java/com/google/cloud/kafka/connect/bigtable/BigtableSinkTaskTest.java @@ -55,6 +55,7 @@ import com.google.cloud.kafka.connect.bigtable.config.BigtableErrorMode; import com.google.cloud.kafka.connect.bigtable.config.BigtableSinkTaskConfig; import com.google.cloud.kafka.connect.bigtable.config.InsertMode; +import com.google.cloud.kafka.connect.bigtable.config.NullValueMode; import com.google.cloud.kafka.connect.bigtable.exception.InvalidBigtableSchemaModificationException; import com.google.cloud.kafka.connect.bigtable.mapping.KeyMapper; import com.google.cloud.kafka.connect.bigtable.mapping.MutationData; @@ -188,29 +189,14 @@ public void testCreateRecordMutationDataEmptyKey() { @Test public void testCreateRecordMutationDataNonemptyKey() { - SinkRecord record = new SinkRecord("topic", 1, null, new Object(), null, null, 1); + SinkRecord emptyRecord = new SinkRecord("topic", 1, null, "key", null, null, 1); + SinkRecord okRecord = new SinkRecord("topic", 1, null, "key", null, "value", 2); + keyMapper = new KeyMapper("#", List.of()); + valueMapper = new ValueMapper("default", "KAFKA_VALUE", NullValueMode.IGNORE); task = new TestBigtableSinkTask(config, null, null, keyMapper, valueMapper, null, null); - byte[] rowKey = "rowKey".getBytes(StandardCharsets.UTF_8); - doReturn(rowKey).when(keyMapper).getKey(any()); - doAnswer( - i -> { - MutationDataBuilder builder = new MutationDataBuilder(); - return builder; - }) - .when(valueMapper) - .getRecordMutationDataBuilder(any(), anyLong()); - assertTrue(task.createRecordMutationData(record).isEmpty()); - - doAnswer( - i -> { - MutationDataBuilder builder = new MutationDataBuilder(); - builder.deleteRow(); - return builder; - }) - .when(valueMapper) - .getRecordMutationDataBuilder(any(), anyLong()); - assertTrue(task.createRecordMutationData(record).isPresent()); + assertTrue(task.createRecordMutationData(emptyRecord).isEmpty()); + assertTrue(task.createRecordMutationData(okRecord).isPresent()); } @Test From a2b106ea6b32849a075071517012d7f70872070d Mon Sep 17 00:00:00 2001 From: Adam Czajkowski Date: Thu, 23 Jan 2025 14:45:22 +0100 Subject: [PATCH 09/76] remove leftover empty test --- .../connect/bigtable/autocreate/BigtableSchemaManagerTest.java | 3 --- 1 file changed, 3 deletions(-) diff --git a/google-cloud-bigtable-kafka-connect-sink/src/test/java/com/google/cloud/kafka/connect/bigtable/autocreate/BigtableSchemaManagerTest.java b/google-cloud-bigtable-kafka-connect-sink/src/test/java/com/google/cloud/kafka/connect/bigtable/autocreate/BigtableSchemaManagerTest.java index c8b6eaf45d..1b4bebae32 100644 --- a/google-cloud-bigtable-kafka-connect-sink/src/test/java/com/google/cloud/kafka/connect/bigtable/autocreate/BigtableSchemaManagerTest.java +++ b/google-cloud-bigtable-kafka-connect-sink/src/test/java/com/google/cloud/kafka/connect/bigtable/autocreate/BigtableSchemaManagerTest.java @@ -625,9 +625,6 @@ public void testEnsureColumnFamiliesExistMissingTable() { assertTotalNumberOfInvocations(bigtable, expectedBigtableInteractions); } - @Test - public void testErrorsCreatingColumnFamilies() {} - @Test public void testAwaitResourceCreationAndHandleInvalidInputErrors() { int uniqueKafkaOffset = 0; From 02be96cf735ec22293e8f7667d4f07ecc74da900 Mon Sep 17 00:00:00 2001 From: Adam Czajkowski Date: Thu, 23 Jan 2025 14:48:24 +0100 Subject: [PATCH 10/76] Add ALREADY_EXISTS to the list of tested codes in SchemaApiExceptionsTest --- .../bigtable/autocreate/SchemaApiExceptionsTest.java | 7 ++++++- 1 file changed, 6 insertions(+), 1 deletion(-) diff --git a/google-cloud-bigtable-kafka-connect-sink/src/test/java/com/google/cloud/kafka/connect/bigtable/autocreate/SchemaApiExceptionsTest.java b/google-cloud-bigtable-kafka-connect-sink/src/test/java/com/google/cloud/kafka/connect/bigtable/autocreate/SchemaApiExceptionsTest.java index b01595f185..81b0a183f4 100644 --- a/google-cloud-bigtable-kafka-connect-sink/src/test/java/com/google/cloud/kafka/connect/bigtable/autocreate/SchemaApiExceptionsTest.java +++ b/google-cloud-bigtable-kafka-connect-sink/src/test/java/com/google/cloud/kafka/connect/bigtable/autocreate/SchemaApiExceptionsTest.java @@ -44,7 +44,12 @@ public void testIsStatusCodeCausedByInputError() { } for (Code notCausedByInputError : - List.of(Code.NOT_FOUND, Code.RESOURCE_EXHAUSTED, Code.CANCELLED, Code.UNKNOWN)) { + List.of( + Code.ALREADY_EXISTS, + Code.NOT_FOUND, + Code.RESOURCE_EXHAUSTED, + Code.CANCELLED, + Code.UNKNOWN)) { assertFalse( BigtableSchemaManager.SchemaApiExceptions.isStatusCodeCausedByInputError( notCausedByInputError)); From 35aaf50c5592747e50b841257ed59c097fbbb798 Mon Sep 17 00:00:00 2001 From: Adam Czajkowski Date: Thu, 23 Jan 2025 14:50:09 +0100 Subject: [PATCH 11/76] remove a wild semicolon --- .../connect/bigtable/autocreate/SchemaApiExceptionsTest.java | 1 - 1 file changed, 1 deletion(-) diff --git a/google-cloud-bigtable-kafka-connect-sink/src/test/java/com/google/cloud/kafka/connect/bigtable/autocreate/SchemaApiExceptionsTest.java b/google-cloud-bigtable-kafka-connect-sink/src/test/java/com/google/cloud/kafka/connect/bigtable/autocreate/SchemaApiExceptionsTest.java index 81b0a183f4..c0f95e423d 100644 --- a/google-cloud-bigtable-kafka-connect-sink/src/test/java/com/google/cloud/kafka/connect/bigtable/autocreate/SchemaApiExceptionsTest.java +++ b/google-cloud-bigtable-kafka-connect-sink/src/test/java/com/google/cloud/kafka/connect/bigtable/autocreate/SchemaApiExceptionsTest.java @@ -143,6 +143,5 @@ public void testIsCausedByInputErrorIgnoresRetriableField() { expectedResult, BigtableSchemaManager.SchemaApiExceptions.isCausedByInputError(e)); } } - ; } } From c9e630650f2236b781e545f98a9af49704251146 Mon Sep 17 00:00:00 2001 From: Adam Czajkowski Date: Thu, 23 Jan 2025 14:54:56 +0100 Subject: [PATCH 12/76] Split primitive test cases in KeyMapperTest --- .../bigtable/mapping/KeyMapperTest.java | 54 ++++++++++++------- 1 file changed, 36 insertions(+), 18 deletions(-) diff --git a/google-cloud-bigtable-kafka-connect-sink/src/test/java/com/google/cloud/kafka/connect/bigtable/mapping/KeyMapperTest.java b/google-cloud-bigtable-kafka-connect-sink/src/test/java/com/google/cloud/kafka/connect/bigtable/mapping/KeyMapperTest.java index 01d26ce3ae..6ee69f0b8e 100644 --- a/google-cloud-bigtable-kafka-connect-sink/src/test/java/com/google/cloud/kafka/connect/bigtable/mapping/KeyMapperTest.java +++ b/google-cloud-bigtable-kafka-connect-sink/src/test/java/com/google/cloud/kafka/connect/bigtable/mapping/KeyMapperTest.java @@ -58,10 +58,8 @@ public class KeyMapperTest { public void testBoolean() { final String fieldName = "Boolean"; final Boolean fieldValue = true; - Schema kafkaConnectSchema = SchemaBuilder.struct().field(fieldName, Schema.BOOLEAN_SCHEMA).build(); - Struct kafkaConnectStruct = new Struct(kafkaConnectSchema); kafkaConnectStruct.put(fieldName, fieldValue); @@ -80,13 +78,13 @@ public void testBoolean() { } @Test - public void testInteger() { - final String fieldName = "Integer"; + public void testByte() { + final String fieldName = "Byte"; final Byte fieldByteValue = (byte) 42; Schema kafkaConnectSchema = SchemaBuilder.struct().field(fieldName, Schema.INT8_SCHEMA).build(); - Struct kafkaConnectStruct = new Struct(kafkaConnectSchema); kafkaConnectStruct.put(fieldName, fieldByteValue); + assertTrue( Arrays.equals( calculateKey(List.of(), DELIMITER, fieldByteValue), @@ -99,12 +97,17 @@ public void testInteger() { Arrays.equals( calculateKey(List.of(fieldName), DELIMITER, kafkaConnectStruct), fieldByteValue.toString().getBytes(StandardCharsets.UTF_8))); + } + @Test + public void testShort() { + final String fieldName = "Short"; final Short fieldShortValue = (short) 4242; - kafkaConnectSchema = SchemaBuilder.struct().field(fieldName, Schema.INT16_SCHEMA).build(); - - kafkaConnectStruct = new Struct(kafkaConnectSchema); + Schema kafkaConnectSchema = + SchemaBuilder.struct().field(fieldName, Schema.INT16_SCHEMA).build(); + Struct kafkaConnectStruct = new Struct(kafkaConnectSchema); kafkaConnectStruct.put(fieldName, fieldShortValue); + assertTrue( Arrays.equals( calculateKey(List.of(), DELIMITER, fieldShortValue), @@ -117,12 +120,17 @@ public void testInteger() { Arrays.equals( calculateKey(List.of(fieldName), DELIMITER, kafkaConnectStruct), fieldShortValue.toString().getBytes(StandardCharsets.UTF_8))); + } + @Test + public void testInteger() { + String fieldName = "Integer"; final Integer fieldIntegerValue = 424242; - kafkaConnectSchema = SchemaBuilder.struct().field(fieldName, Schema.INT32_SCHEMA).build(); - - kafkaConnectStruct = new Struct(kafkaConnectSchema); + Schema kafkaConnectSchema = + SchemaBuilder.struct().field(fieldName, Schema.INT32_SCHEMA).build(); + Struct kafkaConnectStruct = new Struct(kafkaConnectSchema); kafkaConnectStruct.put(fieldName, fieldIntegerValue); + assertTrue( Arrays.equals( calculateKey(List.of(), DELIMITER, fieldIntegerValue), @@ -135,12 +143,17 @@ public void testInteger() { Arrays.equals( calculateKey(List.of(fieldName), DELIMITER, kafkaConnectStruct), fieldIntegerValue.toString().getBytes(StandardCharsets.UTF_8))); + } + @Test + public void testLong() { + String fieldName = "Long"; final Long fieldLongValue = 424242424242L; - kafkaConnectSchema = SchemaBuilder.struct().field(fieldName, Schema.INT64_SCHEMA).build(); - - kafkaConnectStruct = new Struct(kafkaConnectSchema); + Schema kafkaConnectSchema = + SchemaBuilder.struct().field(fieldName, Schema.INT64_SCHEMA).build(); + Struct kafkaConnectStruct = new Struct(kafkaConnectSchema); kafkaConnectStruct.put(fieldName, fieldLongValue); + assertTrue( Arrays.equals( calculateKey(List.of(), DELIMITER, fieldLongValue), @@ -161,9 +174,9 @@ public void testFloat() { final Float fieldFloatValue = 4242424242.4242F; Schema kafkaConnectSchema = SchemaBuilder.struct().field(fieldName, Schema.FLOAT32_SCHEMA).build(); - Struct kafkaConnectStruct = new Struct(kafkaConnectSchema); kafkaConnectStruct.put(fieldName, fieldFloatValue); + assertTrue( Arrays.equals( calculateKey(List.of(), DELIMITER, fieldFloatValue), @@ -176,12 +189,17 @@ public void testFloat() { Arrays.equals( calculateKey(List.of(fieldName), DELIMITER, kafkaConnectStruct), fieldFloatValue.toString().getBytes(StandardCharsets.UTF_8))); + } + @Test + public void testDouble() { + final String fieldName = "Double"; final Double fieldDoubleValue = 4242424242.4242; - kafkaConnectSchema = SchemaBuilder.struct().field(fieldName, Schema.FLOAT64_SCHEMA).build(); - - kafkaConnectStruct = new Struct(kafkaConnectSchema); + Schema kafkaConnectSchema = + SchemaBuilder.struct().field(fieldName, Schema.FLOAT64_SCHEMA).build(); + Struct kafkaConnectStruct = new Struct(kafkaConnectSchema); kafkaConnectStruct.put(fieldName, fieldDoubleValue); + assertTrue( Arrays.equals( calculateKey(List.of(), DELIMITER, fieldDoubleValue), From 2b9c9c29be7f5a9800d82dac50907ecf1deb9e64 Mon Sep 17 00:00:00 2001 From: Adam Czajkowski Date: Thu, 23 Jan 2025 15:01:17 +0100 Subject: [PATCH 13/76] clean up testDifferentStructMappings --- .../bigtable/mapping/KeyMapperTest.java | 40 ++++++++----------- 1 file changed, 16 insertions(+), 24 deletions(-) diff --git a/google-cloud-bigtable-kafka-connect-sink/src/test/java/com/google/cloud/kafka/connect/bigtable/mapping/KeyMapperTest.java b/google-cloud-bigtable-kafka-connect-sink/src/test/java/com/google/cloud/kafka/connect/bigtable/mapping/KeyMapperTest.java index 6ee69f0b8e..51a8acb62d 100644 --- a/google-cloud-bigtable-kafka-connect-sink/src/test/java/com/google/cloud/kafka/connect/bigtable/mapping/KeyMapperTest.java +++ b/google-cloud-bigtable-kafka-connect-sink/src/test/java/com/google/cloud/kafka/connect/bigtable/mapping/KeyMapperTest.java @@ -443,45 +443,37 @@ public void testUnmappableValues() { @Test public void testDifferentStructMappings() { - final String innerFieldStringName = "InnerString"; - final String innerFieldIntegerName = "InnerInt"; - final String innerStringValue = "forty two"; - final Integer innerIntegerValue = 42; + final String fieldStringName = "String"; + final String fieldIntegerName = "Int"; + final String stringValue = "forty two"; + final Integer integerValue = 42; Schema kafkaConnectInnerSchema = SchemaBuilder.struct() - .field(innerFieldStringName, Schema.STRING_SCHEMA) - .field(innerFieldIntegerName, Schema.INT32_SCHEMA) + .field(fieldStringName, Schema.STRING_SCHEMA) + .field(fieldIntegerName, Schema.INT32_SCHEMA) .build(); - Struct kafkaConnectInnerStruct = new Struct(kafkaConnectInnerSchema); - kafkaConnectInnerStruct.put(innerFieldStringName, innerStringValue); - kafkaConnectInnerStruct.put(innerFieldIntegerName, innerIntegerValue); + Struct struct = new Struct(kafkaConnectInnerSchema); + struct.put(fieldStringName, stringValue); + struct.put(fieldIntegerName, integerValue); // Note that it preserves field order from the Schema. assertTrue( Arrays.equals( - calculateKey(List.of(), DELIMITER, kafkaConnectInnerStruct), - (innerStringValue + DELIMITER + innerIntegerValue).getBytes(StandardCharsets.UTF_8))); + calculateKey(List.of(), DELIMITER, struct), + (stringValue + DELIMITER + integerValue).getBytes(StandardCharsets.UTF_8))); // Force another order. assertTrue( Arrays.equals( - calculateKey( - List.of(innerFieldIntegerName, innerFieldStringName), - DELIMITER, - kafkaConnectInnerStruct), - (innerIntegerValue + DELIMITER + innerStringValue).getBytes(StandardCharsets.UTF_8))); + calculateKey(List.of(fieldIntegerName, fieldStringName), DELIMITER, struct), + (integerValue + DELIMITER + stringValue).getBytes(StandardCharsets.UTF_8))); // Use the same field twice. assertTrue( Arrays.equals( - calculateKey( - List.of(innerFieldIntegerName, innerFieldIntegerName), - DELIMITER, - kafkaConnectInnerStruct), - (innerIntegerValue + DELIMITER + innerIntegerValue).getBytes(StandardCharsets.UTF_8))); + calculateKey(List.of(fieldIntegerName, fieldIntegerName), DELIMITER, struct), + (integerValue + DELIMITER + integerValue).getBytes(StandardCharsets.UTF_8))); // Try accessing nonexistent key. - assertThrows( - DataException.class, - () -> calculateKey(List.of("invalid"), DELIMITER, kafkaConnectInnerStruct)); + assertThrows(DataException.class, () -> calculateKey(List.of("invalid"), DELIMITER, struct)); } @Test From 9fce9ab39ab774ffd3e48aaa1a9ad0ed7fbcf5de Mon Sep 17 00:00:00 2001 From: Adam Czajkowski Date: Thu, 23 Jan 2025 15:13:35 +0100 Subject: [PATCH 14/76] Explain why exceptions are expected to be thrown --- .../kafka/connect/bigtable/mapping/KeyMapperTest.java | 7 +++++++ 1 file changed, 7 insertions(+) diff --git a/google-cloud-bigtable-kafka-connect-sink/src/test/java/com/google/cloud/kafka/connect/bigtable/mapping/KeyMapperTest.java b/google-cloud-bigtable-kafka-connect-sink/src/test/java/com/google/cloud/kafka/connect/bigtable/mapping/KeyMapperTest.java index 51a8acb62d..f939904f95 100644 --- a/google-cloud-bigtable-kafka-connect-sink/src/test/java/com/google/cloud/kafka/connect/bigtable/mapping/KeyMapperTest.java +++ b/google-cloud-bigtable-kafka-connect-sink/src/test/java/com/google/cloud/kafka/connect/bigtable/mapping/KeyMapperTest.java @@ -424,6 +424,8 @@ public void testNull() { SchemaBuilder.struct().field("f", SchemaBuilder.bool().optional()).build(); Struct structNoOptionalFields = new Struct(structOnlyOptionalFieldsSchema); + // The following two invocations throw since `null` cannot be serialized in a key directly (only + // as a member of a Struct/List/Map/...) to preserve compatibility with the Confluent's sink. assertThrows(DataException.class, () -> calculateKey(List.of(), DELIMITER, null)); assertThrows( DataException.class, () -> calculateKey(List.of(), DELIMITER, structNoOptionalFields)); @@ -738,6 +740,8 @@ public void testNullable() { Struct kafkaConnectStruct = new Struct(kafkaConnectSchema); kafkaConnectStruct.put(nullableFieldName, nullableFieldValue); kafkaConnectStruct.put(requiredFieldName, requiredFieldValue); + // The following two invocations throw since `null` cannot be serialized in a key directly (only + // as a member of a Struct/List/Map/...) to preserve compatibility with the Confluent's sink. assertThrows(DataException.class, () -> calculateKey(List.of(), DELIMITER, kafkaConnectStruct)); assertThrows( DataException.class, @@ -765,10 +769,13 @@ public void testNullableStruct() { Struct kafkaConnectStruct = new Struct(kafkaConnectSchema); kafkaConnectStruct.put(nullableFieldName, null); + // The following two invocations throw since `null` cannot be serialized in a key directly (only + // as a member of a Struct/List/Map/...) to preserve compatibility with the Confluent's sink. assertThrows(DataException.class, () -> calculateKey(List.of(), DELIMITER, kafkaConnectStruct)); assertThrows( DataException.class, () -> calculateKey(List.of(nullableFieldName), DELIMITER, kafkaConnectStruct)); + // Access to a field of a nonexistent struct. assertThrows( DataException.class, () -> From af54ebe0d4a83373aecf1a659a015a7898b60890 Mon Sep 17 00:00:00 2001 From: Adam Czajkowski Date: Thu, 30 Jan 2025 13:40:28 +0100 Subject: [PATCH 15/76] split testStruct into three --- .../bigtable/mapping/KeyMapperTest.java | 107 +++++++++++++----- 1 file changed, 78 insertions(+), 29 deletions(-) diff --git a/google-cloud-bigtable-kafka-connect-sink/src/test/java/com/google/cloud/kafka/connect/bigtable/mapping/KeyMapperTest.java b/google-cloud-bigtable-kafka-connect-sink/src/test/java/com/google/cloud/kafka/connect/bigtable/mapping/KeyMapperTest.java index f939904f95..8135ef83b2 100644 --- a/google-cloud-bigtable-kafka-connect-sink/src/test/java/com/google/cloud/kafka/connect/bigtable/mapping/KeyMapperTest.java +++ b/google-cloud-bigtable-kafka-connect-sink/src/test/java/com/google/cloud/kafka/connect/bigtable/mapping/KeyMapperTest.java @@ -270,15 +270,45 @@ public void testString() { } @Test - public void testStruct() { - final String middleFieldStructName = "MiddleStruct"; - final String middleFieldArrayName = "MiddleArray"; + public void testFlatStruct() { + final String fieldStringName = "InnerString"; + final String fieldIntegerName = "InnerInt"; + final String stringValue = "forty two"; + final Integer integerValue = 42; + + Schema kafkaConnectSchema = + SchemaBuilder.struct() + .field(fieldStringName, Schema.STRING_SCHEMA) + .field(fieldIntegerName, Schema.INT32_SCHEMA) + .build(); + + Struct kafkaConnectInnerStruct = new Struct(kafkaConnectSchema); + kafkaConnectInnerStruct.put(fieldStringName, stringValue); + kafkaConnectInnerStruct.put(fieldIntegerName, integerValue); + + assertTrue( + Arrays.equals( + calculateKey(List.of(), DELIMITER, kafkaConnectInnerStruct), + (stringValue + DELIMITER + integerValue).getBytes(StandardCharsets.UTF_8))); + assertTrue( + Arrays.equals( + calculateKey(List.of(fieldStringName), DELIMITER, kafkaConnectInnerStruct), + stringValue.toString().getBytes(StandardCharsets.UTF_8))); + assertTrue( + Arrays.equals( + calculateKey(List.of(fieldIntegerName), DELIMITER, kafkaConnectInnerStruct), + integerValue.toString().getBytes(StandardCharsets.UTF_8))); + } + + @Test + public void testStructNestedOnce() { + final String fieldArrayName = "MiddleArray"; final String innerFieldStructName = "InnerStruct"; final String innerFieldStringName = "InnerString"; final String innerFieldIntegerName = "InnerInt"; final String innerStringValue = "forty two"; final Integer innerIntegerValue = 42; - final List middleArrayValue = Arrays.asList(42.0f, 42.4f, 42.42f, 42.424f, 42.4242f); + final List arrayValue = Arrays.asList(42.0f, 42.4f, 42.42f, 42.424f, 42.4242f); Schema kafkaConnectInnerSchema = SchemaBuilder.struct() @@ -290,55 +320,74 @@ public void testStruct() { kafkaConnectInnerStruct.put(innerFieldStringName, innerStringValue); kafkaConnectInnerStruct.put(innerFieldIntegerName, innerIntegerValue); - assertTrue( - Arrays.equals( - calculateKey(List.of(), DELIMITER, kafkaConnectInnerStruct), - (innerStringValue + DELIMITER + innerIntegerValue).getBytes(StandardCharsets.UTF_8))); - assertTrue( - Arrays.equals( - calculateKey(List.of(innerFieldStringName), DELIMITER, kafkaConnectInnerStruct), - innerStringValue.toString().getBytes(StandardCharsets.UTF_8))); - assertTrue( - Arrays.equals( - calculateKey(List.of(innerFieldIntegerName), DELIMITER, kafkaConnectInnerStruct), - innerIntegerValue.toString().getBytes(StandardCharsets.UTF_8))); - - Schema kafkaConnectMiddleSchema = + Schema kafkaConnectSchema = SchemaBuilder.struct() .field(innerFieldStructName, kafkaConnectInnerSchema) - .field(middleFieldArrayName, SchemaBuilder.array(Schema.FLOAT32_SCHEMA).build()) + .field(fieldArrayName, SchemaBuilder.array(Schema.FLOAT32_SCHEMA).build()) .build(); - Struct kafkaConnectMiddleStruct = new Struct(kafkaConnectMiddleSchema); - kafkaConnectMiddleStruct.put(innerFieldStructName, kafkaConnectInnerStruct); - kafkaConnectMiddleStruct.put(middleFieldArrayName, middleArrayValue); + Struct kafkaConnectStruct = new Struct(kafkaConnectSchema); + kafkaConnectStruct.put(innerFieldStructName, kafkaConnectInnerStruct); + kafkaConnectStruct.put(fieldArrayName, arrayValue); assertTrue( Arrays.equals( - calculateKey(List.of(), DELIMITER, kafkaConnectMiddleStruct), - (kafkaConnectInnerStruct.toString() + DELIMITER + middleArrayValue.toString()) + calculateKey(List.of(), DELIMITER, kafkaConnectStruct), + (kafkaConnectInnerStruct.toString() + DELIMITER + arrayValue.toString()) .getBytes(StandardCharsets.UTF_8))); assertTrue( Arrays.equals( - calculateKey(List.of(innerFieldStructName), DELIMITER, kafkaConnectMiddleStruct), + calculateKey(List.of(innerFieldStructName), DELIMITER, kafkaConnectStruct), kafkaConnectInnerStruct.toString().getBytes(StandardCharsets.UTF_8))); assertTrue( Arrays.equals( - calculateKey(List.of(middleFieldArrayName), DELIMITER, kafkaConnectMiddleStruct), - middleArrayValue.toString().getBytes(StandardCharsets.UTF_8))); + calculateKey(List.of(fieldArrayName), DELIMITER, kafkaConnectStruct), + arrayValue.toString().getBytes(StandardCharsets.UTF_8))); assertTrue( Arrays.equals( calculateKey( List.of(innerFieldStructName + "." + innerFieldStringName), DELIMITER, - kafkaConnectMiddleStruct), + kafkaConnectStruct), innerStringValue.toString().getBytes(StandardCharsets.UTF_8))); assertTrue( Arrays.equals( calculateKey( List.of(innerFieldStructName + "." + innerFieldIntegerName), DELIMITER, - kafkaConnectMiddleStruct), + kafkaConnectStruct), innerIntegerValue.toString().getBytes(StandardCharsets.UTF_8))); + } + + @Test + public void testStructNestedTwice() { + final String middleFieldStructName = "MiddleStruct"; + final String middleFieldArrayName = "MiddleArray"; + final String innerFieldStructName = "InnerStruct"; + final String innerFieldStringName = "InnerString"; + final String innerFieldIntegerName = "InnerInt"; + final String innerStringValue = "forty two"; + final Integer innerIntegerValue = 42; + final List middleArrayValue = Arrays.asList(42.0f, 42.4f, 42.42f, 42.424f, 42.4242f); + + Schema kafkaConnectInnerSchema = + SchemaBuilder.struct() + .field(innerFieldStringName, Schema.STRING_SCHEMA) + .field(innerFieldIntegerName, Schema.INT32_SCHEMA) + .build(); + + Struct kafkaConnectInnerStruct = new Struct(kafkaConnectInnerSchema); + kafkaConnectInnerStruct.put(innerFieldStringName, innerStringValue); + kafkaConnectInnerStruct.put(innerFieldIntegerName, innerIntegerValue); + + Schema kafkaConnectMiddleSchema = + SchemaBuilder.struct() + .field(innerFieldStructName, kafkaConnectInnerSchema) + .field(middleFieldArrayName, SchemaBuilder.array(Schema.FLOAT32_SCHEMA).build()) + .build(); + + Struct kafkaConnectMiddleStruct = new Struct(kafkaConnectMiddleSchema); + kafkaConnectMiddleStruct.put(innerFieldStructName, kafkaConnectInnerStruct); + kafkaConnectMiddleStruct.put(middleFieldArrayName, middleArrayValue); Schema kafkaConnectOuterSchema = SchemaBuilder.struct() From f5660b0445a75acd13ba199602056e6a5e80ae55 Mon Sep 17 00:00:00 2001 From: Adam Czajkowski Date: Thu, 30 Jan 2025 15:19:49 +0100 Subject: [PATCH 16/76] update pom.xml with integration test dependencies --- .../pom.xml | 75 ++++++++++++++++++- 1 file changed, 73 insertions(+), 2 deletions(-) diff --git a/google-cloud-bigtable-kafka-connect-sink/pom.xml b/google-cloud-bigtable-kafka-connect-sink/pom.xml index 30340f29e0..e2d4b27d0e 100644 --- a/google-cloud-bigtable-kafka-connect-sink/pom.xml +++ b/google-cloud-bigtable-kafka-connect-sink/pom.xml @@ -12,16 +12,17 @@ 26.31.0 - 3.8.1 + 3.6.1 2.12 2.6.1 1.7.36 - 2.16.2 + 2.14.2 5.14.2 4.13.2 5.11.3 + 7.6.0 2.43.0 1.19.2 3.11.2 @@ -29,6 +30,7 @@ 3.5.2 3.5.2 3.4.2 + 9.4.57.v20241219 11 11 @@ -134,6 +136,68 @@ ${junit.jupiter.version} test + + io.confluent + kafka-schema-registry + ${confluent.version} + test + + + io.confluent + kafka-schema-registry + tests + test-jar + ${confluent.version} + test + + + io.confluent + kafka-avro-serializer + ${confluent.version} + test + + + io.confluent + kafka-connect-avro-converter + ${confluent.version} + test + + + io.confluent + kafka-connect-protobuf-converter + ${confluent.version} + test + + + io.confluent + kafka-connect-json-schema-converter + ${confluent.version} + test + + + org.eclipse.jetty + jetty-server + ${jetty.version} + test + + + org.eclipse.jetty + jetty-servlet + ${jetty.version} + test + + + org.eclipse.jetty + jetty-servlets + ${jetty.version} + test + + + org.eclipse.jetty + jetty-client + ${jetty.version} + test + @@ -276,4 +340,11 @@ + + + + confluent + https://packages.confluent.io/maven/ + + \ No newline at end of file From 5e278122c34b56a4d1ef47273c76b7d2f755c419 Mon Sep 17 00:00:00 2001 From: Adam Czajkowski Date: Thu, 30 Jan 2025 15:22:30 +0100 Subject: [PATCH 17/76] use config variable in JsonConverterFactory --- .../kafka/connect/bigtable/util/JsonConverterFactory.java | 3 ++- 1 file changed, 2 insertions(+), 1 deletion(-) diff --git a/google-cloud-bigtable-kafka-connect-sink/src/test/java/com/google/cloud/kafka/connect/bigtable/util/JsonConverterFactory.java b/google-cloud-bigtable-kafka-connect-sink/src/test/java/com/google/cloud/kafka/connect/bigtable/util/JsonConverterFactory.java index 4192fec3cc..14ddf4ac03 100644 --- a/google-cloud-bigtable-kafka-connect-sink/src/test/java/com/google/cloud/kafka/connect/bigtable/util/JsonConverterFactory.java +++ b/google-cloud-bigtable-kafka-connect-sink/src/test/java/com/google/cloud/kafka/connect/bigtable/util/JsonConverterFactory.java @@ -17,11 +17,12 @@ import java.util.Map; import org.apache.kafka.connect.json.JsonConverter; +import org.apache.kafka.connect.json.JsonConverterConfig; public class JsonConverterFactory { public static JsonConverter create(boolean schemasEnable, boolean isKafkaKeyConverter) { Map jsonConverterProps = - Map.of("schemas.enable", Boolean.toString(schemasEnable)); + Map.of(JsonConverterConfig.SCHEMAS_ENABLE_CONFIG, Boolean.toString(schemasEnable)); JsonConverter jsonConverter = new JsonConverter(); jsonConverter.configure(jsonConverterProps, isKafkaKeyConverter); return jsonConverter; From d5c95d14659ce7e48f9d467cda400e18a339127f Mon Sep 17 00:00:00 2001 From: Adam Czajkowski Date: Thu, 30 Jan 2025 15:23:25 +0100 Subject: [PATCH 18/76] Unify quoting variables in log messages --- .../connect/bigtable/autocreate/BigtableSchemaManager.java | 4 ++-- 1 file changed, 2 insertions(+), 2 deletions(-) diff --git a/google-cloud-bigtable-kafka-connect-sink/src/main/java/com/google/cloud/kafka/connect/bigtable/autocreate/BigtableSchemaManager.java b/google-cloud-bigtable-kafka-connect-sink/src/main/java/com/google/cloud/kafka/connect/bigtable/autocreate/BigtableSchemaManager.java index 5957be5873..a43a22a52f 100644 --- a/google-cloud-bigtable-kafka-connect-sink/src/main/java/com/google/cloud/kafka/connect/bigtable/autocreate/BigtableSchemaManager.java +++ b/google-cloud-bigtable-kafka-connect-sink/src/main/java/com/google/cloud/kafka/connect/bigtable/autocreate/BigtableSchemaManager.java @@ -337,7 +337,7 @@ private Map, ResourceAndRecords> sendCreateTableRequest } private ApiFuture
    createTable(String tableName) { - logger.info("Creating table '{}'", tableName); + logger.info("Creating table `{}`", tableName); CreateTableRequest createTableRequest = CreateTableRequest.of(tableName); return bigtable.createTableAsync(createTableRequest); } @@ -349,7 +349,7 @@ private ApiFuture
    createTable(String tableName) { private ApiFuture
    createColumnFamily(Map.Entry tableNameAndColumnFamily) { String tableName = tableNameAndColumnFamily.getKey(); String columnFamily = tableNameAndColumnFamily.getValue(); - logger.info("Creating column family '{}' in table '{}'", columnFamily, tableName); + logger.info("Creating column family `{}` in table `{}`", columnFamily, tableName); ModifyColumnFamiliesRequest request = ModifyColumnFamiliesRequest.of(tableName).addFamily(columnFamily); return bigtable.modifyFamiliesAsync(request); From b55ac40f972eb67491111b28397c2fb72301dcb8 Mon Sep 17 00:00:00 2001 From: Adam Czajkowski Date: Thu, 30 Jan 2025 15:26:02 +0100 Subject: [PATCH 19/76] Make column family presence required for delete operations --- .../bigtable/mapping/MutationDataBuilder.java | 18 ++++++++++++++++++ .../mapping/MutationDataBuilderTest.java | 8 ++++---- 2 files changed, 22 insertions(+), 4 deletions(-) diff --git a/google-cloud-bigtable-kafka-connect-sink/src/main/java/com/google/cloud/kafka/connect/bigtable/mapping/MutationDataBuilder.java b/google-cloud-bigtable-kafka-connect-sink/src/main/java/com/google/cloud/kafka/connect/bigtable/mapping/MutationDataBuilder.java index 229853dae4..75fab665ba 100644 --- a/google-cloud-bigtable-kafka-connect-sink/src/main/java/com/google/cloud/kafka/connect/bigtable/mapping/MutationDataBuilder.java +++ b/google-cloud-bigtable-kafka-connect-sink/src/main/java/com/google/cloud/kafka/connect/bigtable/mapping/MutationDataBuilder.java @@ -63,12 +63,30 @@ public void deleteRow() { public void deleteFamily(String columnFamily) { mutationIsEmpty = false; + // Cloud Bigtable (but not the Java-based emulator!) returns an error when a mutation contains + // a deletion of a nonexistent column family, thus rolling not only the deletion, but also other + // operations within the row back. + // Thus, we need to make a choice to either: + // a) ensure that the needed column family exists. + // b) modify the record as if it didn't contain the deletion of columns that don't exist so that + // other operations within the row have a chance to execute successfully. + // I think the option a) is clearer to reason about, so we mark the column family as required. + requiredColumnFamilies.add(columnFamily); mutation.deleteFamily(columnFamily); } public void deleteCells( String columnFamily, ByteString columnQualifier, Range.TimestampRange timestampRange) { mutationIsEmpty = false; + // Cloud Bigtable returns an error when a mutation contains a deletion of a cell within + // a nonexistent column family, thus rolling not only the deletion, but also other + // operations within the row back. + // Thus, we need to make a choice to either: + // a) ensure that the needed column family exists. + // b) modify the record as if it didn't contain the deletion of columns that don't exist so that + // other operations within the row have a chance to execute successfully. + // I think the option a) is clearer to reason about, so we mark the column family as required. + requiredColumnFamilies.add(columnFamily); mutation.deleteCells(columnFamily, columnQualifier, timestampRange); } diff --git a/google-cloud-bigtable-kafka-connect-sink/src/test/java/com/google/cloud/kafka/connect/bigtable/mapping/MutationDataBuilderTest.java b/google-cloud-bigtable-kafka-connect-sink/src/test/java/com/google/cloud/kafka/connect/bigtable/mapping/MutationDataBuilderTest.java index 62fc0d0011..a9ac2fbd8e 100644 --- a/google-cloud-bigtable-kafka-connect-sink/src/test/java/com/google/cloud/kafka/connect/bigtable/mapping/MutationDataBuilderTest.java +++ b/google-cloud-bigtable-kafka-connect-sink/src/test/java/com/google/cloud/kafka/connect/bigtable/mapping/MutationDataBuilderTest.java @@ -67,7 +67,7 @@ public void testDeleteRow() { Optional mutationData = mutationDataBuilder.maybeBuild(TARGET_TABLE_NAME, ROW_KEY); assertTrue(mutationData.isPresent()); - assertEquals(mutationData.get().getRequiredColumnFamilies(), Set.of()); + assertTrue(mutationData.get().getRequiredColumnFamilies().isEmpty()); } @Test @@ -77,7 +77,7 @@ public void testDeleteCells() { Optional mutationData = mutationDataBuilder.maybeBuild(TARGET_TABLE_NAME, ROW_KEY); assertTrue(mutationData.isPresent()); - assertTrue(mutationData.get().getRequiredColumnFamilies().isEmpty()); + assertEquals(Set.of(COLUMN_FAMILY), mutationData.get().getRequiredColumnFamilies()); } @Test @@ -87,7 +87,7 @@ public void testDeleteFamily() { Optional mutationData = mutationDataBuilder.maybeBuild(TARGET_TABLE_NAME, ROW_KEY); assertTrue(mutationData.isPresent()); - assertTrue(mutationData.get().getRequiredColumnFamilies().isEmpty()); + assertEquals(Set.of(COLUMN_FAMILY), mutationData.get().getRequiredColumnFamilies()); } @Test @@ -97,6 +97,6 @@ public void testSetCell() { Optional mutationData = mutationDataBuilder.maybeBuild(TARGET_TABLE_NAME, ROW_KEY); assertTrue(mutationData.isPresent()); - assertEquals(mutationData.get().getRequiredColumnFamilies(), Set.of(COLUMN_FAMILY)); + assertEquals(Set.of(COLUMN_FAMILY), mutationData.get().getRequiredColumnFamilies()); } } From 3e9bec22ce04fb2c17433925bc292c2e6eeeacad Mon Sep 17 00:00:00 2001 From: Adam Czajkowski Date: Thu, 30 Jan 2025 15:37:16 +0100 Subject: [PATCH 20/76] Add string interpolation to default column family config --- .../connect/bigtable/BigtableSinkTask.java | 8 +- .../bigtable/config/BigtableSinkConfig.java | 49 ++++++-- .../bigtable/config/ConfigInterpolation.java | 30 +++++ .../connect/bigtable/mapping/ValueMapper.java | 21 ++-- .../bigtable/BigtableSinkTaskTest.java | 25 ++-- .../config/BigtableSinkConfigTest.java | 16 ++- .../config/ConfigInterpolationTest.java | 54 +++++++++ .../bigtable/mapping/ValueMapperTest.java | 114 ++++++++++++------ 8 files changed, 236 insertions(+), 81 deletions(-) create mode 100644 google-cloud-bigtable-kafka-connect-sink/src/main/java/com/google/cloud/kafka/connect/bigtable/config/ConfigInterpolation.java create mode 100644 google-cloud-bigtable-kafka-connect-sink/src/test/java/com/google/cloud/kafka/connect/bigtable/config/ConfigInterpolationTest.java diff --git a/google-cloud-bigtable-kafka-connect-sink/src/main/java/com/google/cloud/kafka/connect/bigtable/BigtableSinkTask.java b/google-cloud-bigtable-kafka-connect-sink/src/main/java/com/google/cloud/kafka/connect/bigtable/BigtableSinkTask.java index e474518e55..a7c61d6819 100644 --- a/google-cloud-bigtable-kafka-connect-sink/src/main/java/com/google/cloud/kafka/connect/bigtable/BigtableSinkTask.java +++ b/google-cloud-bigtable-kafka-connect-sink/src/main/java/com/google/cloud/kafka/connect/bigtable/BigtableSinkTask.java @@ -29,6 +29,7 @@ import com.google.cloud.kafka.connect.bigtable.config.BigtableErrorMode; import com.google.cloud.kafka.connect.bigtable.config.BigtableSinkConfig; import com.google.cloud.kafka.connect.bigtable.config.BigtableSinkTaskConfig; +import com.google.cloud.kafka.connect.bigtable.config.ConfigInterpolation; import com.google.cloud.kafka.connect.bigtable.exception.BatchException; import com.google.cloud.kafka.connect.bigtable.exception.InvalidBigtableSchemaModificationException; import com.google.cloud.kafka.connect.bigtable.mapping.KeyMapper; @@ -226,7 +227,7 @@ Optional createRecordMutationData(SinkRecord record) { } long timestamp = getTimestampMicros(record); MutationDataBuilder mutationDataBuilder = - valueMapper.getRecordMutationDataBuilder(record.value(), timestamp); + valueMapper.getRecordMutationDataBuilder(record.value(), record.topic(), timestamp); return mutationDataBuilder.maybeBuild(recordTableId, rowKey); } @@ -238,9 +239,8 @@ Optional createRecordMutationData(SinkRecord record) { */ @VisibleForTesting String getTableName(SinkRecord record) { - return config - .getString(BigtableSinkTaskConfig.CONFIG_TABLE_NAME_FORMAT) - .replace("${topic}", record.topic()); + String template = config.getString(BigtableSinkTaskConfig.CONFIG_TABLE_NAME_FORMAT); + return ConfigInterpolation.replace(template, record.topic()); } /** diff --git a/google-cloud-bigtable-kafka-connect-sink/src/main/java/com/google/cloud/kafka/connect/bigtable/config/BigtableSinkConfig.java b/google-cloud-bigtable-kafka-connect-sink/src/main/java/com/google/cloud/kafka/connect/bigtable/config/BigtableSinkConfig.java index 5fb5a21deb..9abe10c7d8 100644 --- a/google-cloud-bigtable-kafka-connect-sink/src/main/java/com/google/cloud/kafka/connect/bigtable/config/BigtableSinkConfig.java +++ b/google-cloud-bigtable-kafka-connect-sink/src/main/java/com/google/cloud/kafka/connect/bigtable/config/BigtableSinkConfig.java @@ -123,9 +123,12 @@ static Config validate(Map props, boolean accessBigtableToValida String credentialsPath = props.get(CONFIG_GCP_CREDENTIALS_PATH); String credentialsJson = props.get(CONFIG_GCP_CREDENTIALS_JSON); String insertMode = props.get(CONFIG_INSERT_MODE); + String nullValueMode = props.get(CONFIG_VALUE_NULL_MODE); String maxBatchSize = props.get(CONFIG_MAX_BATCH_SIZE); String effectiveInsertMode = Optional.ofNullable(insertMode).orElse(DEFAULT_INSERT_MODE.name()).toUpperCase(); + String effectiveNullValueMode = + Optional.ofNullable(nullValueMode).orElse(DEFAULT_NULL_VALUE_MODE.name()).toUpperCase(); String effectiveMaxBatchSize = Optional.ofNullable(maxBatchSize).orElse(DEFAULT_MAX_BATCH_SIZE.toString()).trim(); @@ -144,12 +147,29 @@ static Config validate(Map props, boolean accessBigtableToValida String errorMessage = "When using `" + CONFIG_INSERT_MODE - + "` of `insert`, " + + "` of `" + + InsertMode.INSERT.name() + + "`, " + CONFIG_MAX_BATCH_SIZE + " must be set to `1`."; addErrorMessage(validationResult, CONFIG_INSERT_MODE, insertMode, errorMessage); addErrorMessage(validationResult, CONFIG_MAX_BATCH_SIZE, maxBatchSize, errorMessage); } + if (effectiveInsertMode.equals(InsertMode.INSERT.name()) + && effectiveNullValueMode.equals(NullValueMode.DELETE.name())) { + String errorMessage = + "When using `" + + CONFIG_VALUE_NULL_MODE + + "` of `" + + NullValueMode.DELETE.name() + + "`, " + + CONFIG_INSERT_MODE + + " must not be set to `" + + InsertMode.INSERT.name() + + "`."; + addErrorMessage(validationResult, CONFIG_INSERT_MODE, insertMode, errorMessage); + addErrorMessage(validationResult, CONFIG_VALUE_NULL_MODE, nullValueMode, errorMessage); + } if (accessBigtableToValidateConfiguration && validationResult.values().stream().allMatch(v -> v.errorMessages().isEmpty())) { @@ -271,13 +291,16 @@ public static ConfigDef getDefinition() { .define( CONFIG_TABLE_NAME_FORMAT, ConfigDef.Type.STRING, - "${topic}", + ConfigInterpolation.TOPIC_PLACEHOLDER, ConfigDef.CompositeValidator.of( new ConfigDef.NonNullValidator(), new ConfigDef.NonEmptyString()), ConfigDef.Importance.MEDIUM, - "Name of the destination table. Use `${topic}` within the table name to specify" - + " the originating topic name.\nFor example, `user_${topic}` for the topic `stats`" - + " will map to the table name `user_stats`.") + "Name of the destination table. Use `" + + ConfigInterpolation.TOPIC_PLACEHOLDER + + "` within the table name to specify the originating topic name.\n" + + "For example, `user_" + + ConfigInterpolation.TOPIC_PLACEHOLDER + + "` for the topic `stats` will map to the table name `user_stats`.") .define( CONFIG_ROW_KEY_DEFINITION, ConfigDef.Type.LIST, @@ -320,18 +343,22 @@ public static ConfigDef getDefinition() { new ConfigDef.NonNullValidator(), ConfigDef.Importance.MEDIUM, "Whether to automatically create missing columns families in the table relative to the" - + " record schema." - + "\nDoes not imply auto-creation of tables." - + "\nWhen enabled, the records for which the auto-creation fails, are failed." - + "\nRecreation of column families deleted by other Cloud Bigtable users is not" + + " record schema.\n" + + "Does not imply auto-creation of tables.\n" + + "When enabled, the records for which the auto-creation fails, are failed.\n" + + "When enabled, column families will be created also for deletions of nonexistent" + + " column families and cells within them.\n" + + "Recreation of column families deleted by other Cloud Bigtable users is not" + " supported.") .define( CONFIG_DEFAULT_COLUMN_FAMILY, ConfigDef.Type.STRING, - "default", + ConfigInterpolation.TOPIC_PLACEHOLDER, ConfigDef.Importance.MEDIUM, "Any root-level fields on the SinkRecord that aren't objects will be added to this" - + " column family. If empty, the fields will be ignored.") + + " column family. If empty, the fields will be ignored. Use `" + + ConfigInterpolation.TOPIC_PLACEHOLDER + + "` within the column family name to specify the originating topic name.") .define( CONFIG_DEFAULT_COLUMN_QUALIFIER, ConfigDef.Type.STRING, diff --git a/google-cloud-bigtable-kafka-connect-sink/src/main/java/com/google/cloud/kafka/connect/bigtable/config/ConfigInterpolation.java b/google-cloud-bigtable-kafka-connect-sink/src/main/java/com/google/cloud/kafka/connect/bigtable/config/ConfigInterpolation.java new file mode 100644 index 0000000000..e7cc5ed4be --- /dev/null +++ b/google-cloud-bigtable-kafka-connect-sink/src/main/java/com/google/cloud/kafka/connect/bigtable/config/ConfigInterpolation.java @@ -0,0 +1,30 @@ +/* + * Copyright 2025 Google LLC + * + * Licensed under the Apache License, Version 2.0 (the "License"); + * you may not use this file except in compliance with the License. + * You may obtain a copy of the License at + * + * https://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ +package com.google.cloud.kafka.connect.bigtable.config; + +/** Class handling string interpolation in config values supporting dynamic values. */ +public class ConfigInterpolation { + public static String TOPIC_PLACEHOLDER = "${topic}"; + + /** + * @param template String possibly containing placeholders to be replaced. + * @param topic Name of Kafka topic. + * @return {@code template} with placeholders replaced with corresponding values. + */ + public static String replace(String template, String topic) { + return template.replace(TOPIC_PLACEHOLDER, topic); + } +} diff --git a/google-cloud-bigtable-kafka-connect-sink/src/main/java/com/google/cloud/kafka/connect/bigtable/mapping/ValueMapper.java b/google-cloud-bigtable-kafka-connect-sink/src/main/java/com/google/cloud/kafka/connect/bigtable/mapping/ValueMapper.java index f97fc78f53..5cefd00dde 100644 --- a/google-cloud-bigtable-kafka-connect-sink/src/main/java/com/google/cloud/kafka/connect/bigtable/mapping/ValueMapper.java +++ b/google-cloud-bigtable-kafka-connect-sink/src/main/java/com/google/cloud/kafka/connect/bigtable/mapping/ValueMapper.java @@ -23,6 +23,7 @@ import com.fasterxml.jackson.databind.ser.std.StdSerializer; import com.google.cloud.ByteArray; import com.google.cloud.bigtable.data.v2.models.Range; +import com.google.cloud.kafka.connect.bigtable.config.ConfigInterpolation; import com.google.cloud.kafka.connect.bigtable.config.NullValueMode; import com.google.common.annotations.VisibleForTesting; import com.google.protobuf.ByteString; @@ -49,7 +50,7 @@ * Mutation(s)}. */ public class ValueMapper { - public final String defaultColumnFamily; + public final String defaultColumnFamilyTemplate; public final ByteString defaultColumnQualifier; private final NullValueMode nullMode; private static final ObjectMapper jsonMapper = getJsonMapper(); @@ -64,7 +65,8 @@ public class ValueMapper { */ public ValueMapper( String defaultColumnFamily, String defaultColumnQualifier, @Nonnull NullValueMode nullMode) { - this.defaultColumnFamily = Utils.isBlank(defaultColumnFamily) ? null : defaultColumnFamily; + this.defaultColumnFamilyTemplate = + Utils.isBlank(defaultColumnFamily) ? null : defaultColumnFamily; this.defaultColumnQualifier = Utils.isBlank(defaultColumnQualifier) ? null @@ -78,10 +80,11 @@ public ValueMapper( * * @param rootKafkaValue The value to be converted into Cloud Bigtable {@link * com.google.cloud.bigtable.data.v2.models.Mutation Mutation(s)}. + * @param topic The name of Kafka topic this value originates from. * @param timestampMicros The timestamp the mutations will be created at in microseconds. */ public MutationDataBuilder getRecordMutationDataBuilder( - Object rootKafkaValue, long timestampMicros) { + Object rootKafkaValue, String topic, long timestampMicros) { MutationDataBuilder mutationDataBuilder = createMutationDataBuilder(); if (rootKafkaValue == null && nullMode == NullValueMode.IGNORE) { // Do nothing @@ -116,9 +119,9 @@ public MutationDataBuilder getRecordMutationDataBuilder( } } } else { - if (defaultColumnFamily != null) { + if (defaultColumnFamilyTemplate != null) { mutationDataBuilder.setCell( - defaultColumnFamily, + getDefaultColumnFamily(topic), ByteString.copyFrom(kafkaFieldName.getBytes(StandardCharsets.UTF_8)), timestampMicros, ByteString.copyFrom(serialize(kafkaFieldValue))); @@ -126,9 +129,9 @@ public MutationDataBuilder getRecordMutationDataBuilder( } } } else { - if (defaultColumnFamily != null && defaultColumnQualifier != null) { + if (defaultColumnFamilyTemplate != null && defaultColumnQualifier != null) { mutationDataBuilder.setCell( - defaultColumnFamily, + getDefaultColumnFamily(topic), defaultColumnQualifier, timestampMicros, ByteString.copyFrom(serialize(rootKafkaValue))); @@ -143,6 +146,10 @@ protected MutationDataBuilder createMutationDataBuilder() { return new MutationDataBuilder(); } + protected String getDefaultColumnFamily(String topic) { + return ConfigInterpolation.replace(defaultColumnFamilyTemplate, topic); + } + /** * @param mapOrStruct {@link Map} or {@link Struct} whose children we want to list * @return {@link List} of names or keys of input value's child entries. diff --git a/google-cloud-bigtable-kafka-connect-sink/src/test/java/com/google/cloud/kafka/connect/bigtable/BigtableSinkTaskTest.java b/google-cloud-bigtable-kafka-connect-sink/src/test/java/com/google/cloud/kafka/connect/bigtable/BigtableSinkTaskTest.java index be22c82a82..77cfcf9dc5 100644 --- a/google-cloud-bigtable-kafka-connect-sink/src/test/java/com/google/cloud/kafka/connect/bigtable/BigtableSinkTaskTest.java +++ b/google-cloud-bigtable-kafka-connect-sink/src/test/java/com/google/cloud/kafka/connect/bigtable/BigtableSinkTaskTest.java @@ -66,7 +66,6 @@ import com.google.cloud.kafka.connect.bigtable.util.FutureUtil; import com.google.protobuf.ByteString; import java.nio.charset.StandardCharsets; -import java.util.AbstractMap; import java.util.ArrayList; import java.util.Arrays; import java.util.Collections; @@ -161,22 +160,14 @@ public void testVersion() { @Test public void testGetTableName() { + String tableFormat = "table"; SinkRecord record = new SinkRecord("topic", 1, null, null, null, null, 1); - for (Map.Entry test : - List.of( - new AbstractMap.SimpleImmutableEntry<>("prefix_${topic}_suffix", "prefix_topic_suffix"), - new AbstractMap.SimpleImmutableEntry<>( - "prefix_${topic_suffix", "prefix_${topic_suffix"), - new AbstractMap.SimpleImmutableEntry<>("prefix_$topic_suffix", "prefix_$topic_suffix"), - new AbstractMap.SimpleImmutableEntry<>("prefix_${bad}_suffix", "prefix_${bad}_suffix"), - new AbstractMap.SimpleImmutableEntry<>("noSubstitution", "noSubstitution"))) { - Map props = BasicPropertiesFactory.getTaskProps(); - props.put(CONFIG_TABLE_NAME_FORMAT, test.getKey()); - task = - new TestBigtableSinkTask( - new BigtableSinkTaskConfig(props), null, null, null, null, null, null); - assertEquals(test.getValue(), task.getTableName(record)); - } + Map props = BasicPropertiesFactory.getTaskProps(); + props.put(CONFIG_TABLE_NAME_FORMAT, tableFormat); + task = + new TestBigtableSinkTask( + new BigtableSinkTaskConfig(props), null, null, null, null, null, null); + assertEquals(tableFormat, task.getTableName(record)); } @Test @@ -541,7 +532,7 @@ public void testPutBranches() { return builder; }) .when(valueMapper) - .getRecordMutationDataBuilder(any(), anyLong()); + .getRecordMutationDataBuilder(any(), anyString(), anyLong()); Batcher batcher = mock(Batcher.class); doReturn(completedApiFuture(null)).when(batcher).add(any()); diff --git a/google-cloud-bigtable-kafka-connect-sink/src/test/java/com/google/cloud/kafka/connect/bigtable/config/BigtableSinkConfigTest.java b/google-cloud-bigtable-kafka-connect-sink/src/test/java/com/google/cloud/kafka/connect/bigtable/config/BigtableSinkConfigTest.java index b22a5df2bb..2aeb80e483 100644 --- a/google-cloud-bigtable-kafka-connect-sink/src/test/java/com/google/cloud/kafka/connect/bigtable/config/BigtableSinkConfigTest.java +++ b/google-cloud-bigtable-kafka-connect-sink/src/test/java/com/google/cloud/kafka/connect/bigtable/config/BigtableSinkConfigTest.java @@ -36,6 +36,7 @@ import static org.mockito.Mockito.verify; import com.google.cloud.bigtable.admin.v2.BigtableTableAdminClient; +import com.google.cloud.bigtable.data.v2.BigtableDataClient; import com.google.cloud.kafka.connect.bigtable.util.BasicPropertiesFactory; import java.util.HashMap; import java.util.List; @@ -104,16 +105,27 @@ public void testMultipleCredentialsAreDisallowed() { assertFalse(configIsValid(config)); } + @Test + public void testNullDeletionIsIncompatibleWithInsertMode() { + Map props = BasicPropertiesFactory.getSinkProps(); + props.put(BigtableSinkConfig.CONFIG_INSERT_MODE, InsertMode.INSERT.name()); + props.put(CONFIG_VALUE_NULL_MODE, NullValueMode.DELETE.name()); + BigtableSinkConfig config = new BigtableSinkConfig(props); + assertFalse(configIsValid(config)); + } + @Test public void testGetBigtableDataClient() { BigtableSinkConfig config = new BigtableSinkConfig(BasicPropertiesFactory.getSinkProps()); - config.getBigtableDataClient(); + BigtableDataClient client = config.getBigtableDataClient(); + client.close(); } @Test public void testGetBigtableAdminClient() { BigtableSinkConfig config = new BigtableSinkConfig(BasicPropertiesFactory.getSinkProps()); - config.getBigtableAdminClient(); + BigtableTableAdminClient client = config.getBigtableAdminClient(); + client.close(); } @Test diff --git a/google-cloud-bigtable-kafka-connect-sink/src/test/java/com/google/cloud/kafka/connect/bigtable/config/ConfigInterpolationTest.java b/google-cloud-bigtable-kafka-connect-sink/src/test/java/com/google/cloud/kafka/connect/bigtable/config/ConfigInterpolationTest.java new file mode 100644 index 0000000000..b8caab8482 --- /dev/null +++ b/google-cloud-bigtable-kafka-connect-sink/src/test/java/com/google/cloud/kafka/connect/bigtable/config/ConfigInterpolationTest.java @@ -0,0 +1,54 @@ +/* + * Copyright 2025 Google LLC + * + * Licensed under the Apache License, Version 2.0 (the "License"); + * you may not use this file except in compliance with the License. + * You may obtain a copy of the License at + * + * https://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ +package com.google.cloud.kafka.connect.bigtable.config; + +import static org.junit.Assert.assertEquals; + +import java.util.Arrays; +import java.util.Collection; +import org.junit.Test; +import org.junit.runner.RunWith; +import org.junit.runners.Parameterized; + +@RunWith(Parameterized.class) +public class ConfigInterpolationTest { + private static final String TOPIC = "TOPIC"; + + private final String template; + private final String expected; + + public ConfigInterpolationTest(String template, String expected) { + this.template = template; + this.expected = expected; + } + + @Parameterized.Parameters + public static Collection testCases() { + return Arrays.asList( + new Object[][] { + {"prefix_${topic}_suffix", "prefix_TOPIC_suffix"}, + {"prefix_${topic_suffix", "prefix_${topic_suffix"}, + {"prefix_$topic_suffix", "prefix_$topic_suffix"}, + {"prefix_${bad}_suffix", "prefix_${bad}_suffix"}, + {"noSubstitution", "noSubstitution"}, + }); + } + + @Test + public void testReplace() { + assertEquals(ConfigInterpolation.replace(template, TOPIC), expected); + } +} diff --git a/google-cloud-bigtable-kafka-connect-sink/src/test/java/com/google/cloud/kafka/connect/bigtable/mapping/ValueMapperTest.java b/google-cloud-bigtable-kafka-connect-sink/src/test/java/com/google/cloud/kafka/connect/bigtable/mapping/ValueMapperTest.java index ce2762fd76..49c7d4f876 100644 --- a/google-cloud-bigtable-kafka-connect-sink/src/test/java/com/google/cloud/kafka/connect/bigtable/mapping/ValueMapperTest.java +++ b/google-cloud-bigtable-kafka-connect-sink/src/test/java/com/google/cloud/kafka/connect/bigtable/mapping/ValueMapperTest.java @@ -22,11 +22,13 @@ import static org.mockito.Mockito.verify; import com.google.cloud.bigtable.data.v2.models.Range; +import com.google.cloud.kafka.connect.bigtable.config.ConfigInterpolation; import com.google.cloud.kafka.connect.bigtable.config.NullValueMode; import com.google.cloud.kafka.connect.bigtable.util.JsonConverterFactory; import com.google.protobuf.ByteString; import java.math.BigDecimal; import java.nio.charset.StandardCharsets; +import java.util.AbstractMap; import java.util.Date; import java.util.HashMap; import java.util.List; @@ -68,7 +70,7 @@ public void testBoolean() { ByteString expected = ByteString.copyFrom(Bytes.toBytes(value)); ValueMapper mapper = new TestValueMapper(DEFAULT_COLUMN_FAMILY, DEFAULT_COLUMN, NullValueMode.IGNORE); - MutationDataBuilder mutationDataBuilder = mapper.getRecordMutationDataBuilder(value, TIMESTAMP); + MutationDataBuilder mutationDataBuilder = mapper.getRecordMutationDataBuilder(value, DEFAULT_TOPIC, TIMESTAMP); verify(mutationDataBuilder, times(1)) .setCell(DEFAULT_COLUMN_FAMILY, DEFAULT_COLUMN_BYTES, TIMESTAMP, expected); assertTotalNumberOfInvocations(mutationDataBuilder, 1); @@ -80,7 +82,7 @@ public void testString() { ByteString expected = ByteString.copyFrom(Bytes.toBytes(value)); ValueMapper mapper = new TestValueMapper(DEFAULT_COLUMN_FAMILY, DEFAULT_COLUMN, NullValueMode.IGNORE); - MutationDataBuilder mutationDataBuilder = mapper.getRecordMutationDataBuilder(value, TIMESTAMP); + MutationDataBuilder mutationDataBuilder = mapper.getRecordMutationDataBuilder(value, DEFAULT_TOPIC, TIMESTAMP); verify(mutationDataBuilder, times(1)) .setCell(DEFAULT_COLUMN_FAMILY, DEFAULT_COLUMN_BYTES, TIMESTAMP, expected); assertTotalNumberOfInvocations(mutationDataBuilder, 1); @@ -92,7 +94,7 @@ public void testLong() { ByteString expected = ByteString.copyFrom(Bytes.toBytes(value)); ValueMapper mapper = new TestValueMapper(DEFAULT_COLUMN_FAMILY, DEFAULT_COLUMN, NullValueMode.IGNORE); - MutationDataBuilder mutationDataBuilder = mapper.getRecordMutationDataBuilder(value, TIMESTAMP); + MutationDataBuilder mutationDataBuilder = mapper.getRecordMutationDataBuilder(value, DEFAULT_TOPIC, TIMESTAMP); verify(mutationDataBuilder, times(1)) .setCell(DEFAULT_COLUMN_FAMILY, DEFAULT_COLUMN_BYTES, TIMESTAMP, expected); assertTotalNumberOfInvocations(mutationDataBuilder, 1); @@ -104,7 +106,7 @@ public void testInteger() { ByteString expected = ByteString.copyFrom(Bytes.toBytes(value)); ValueMapper mapper = new TestValueMapper(DEFAULT_COLUMN_FAMILY, DEFAULT_COLUMN, NullValueMode.IGNORE); - MutationDataBuilder mutationDataBuilder = mapper.getRecordMutationDataBuilder(value, TIMESTAMP); + MutationDataBuilder mutationDataBuilder = mapper.getRecordMutationDataBuilder(value, DEFAULT_TOPIC, TIMESTAMP); verify(mutationDataBuilder, times(1)) .setCell(DEFAULT_COLUMN_FAMILY, DEFAULT_COLUMN_BYTES, TIMESTAMP, expected); assertTotalNumberOfInvocations(mutationDataBuilder, 1); @@ -116,7 +118,7 @@ public void testShort() { ByteString expected = ByteString.copyFrom(Bytes.toBytes(value)); ValueMapper mapper = new TestValueMapper(DEFAULT_COLUMN_FAMILY, DEFAULT_COLUMN, NullValueMode.IGNORE); - MutationDataBuilder mutationDataBuilder = mapper.getRecordMutationDataBuilder(value, TIMESTAMP); + MutationDataBuilder mutationDataBuilder = mapper.getRecordMutationDataBuilder(value, DEFAULT_TOPIC, TIMESTAMP); verify(mutationDataBuilder, times(1)) .setCell(DEFAULT_COLUMN_FAMILY, DEFAULT_COLUMN_BYTES, TIMESTAMP, expected); assertTotalNumberOfInvocations(mutationDataBuilder, 1); @@ -128,7 +130,7 @@ public void testByte() { ByteString expected = ByteString.copyFrom(Bytes.toBytes(value)); ValueMapper mapper = new TestValueMapper(DEFAULT_COLUMN_FAMILY, DEFAULT_COLUMN, NullValueMode.IGNORE); - MutationDataBuilder mutationDataBuilder = mapper.getRecordMutationDataBuilder(value, TIMESTAMP); + MutationDataBuilder mutationDataBuilder = mapper.getRecordMutationDataBuilder(value, DEFAULT_TOPIC, TIMESTAMP); verify(mutationDataBuilder, times(1)) .setCell(DEFAULT_COLUMN_FAMILY, DEFAULT_COLUMN_BYTES, TIMESTAMP, expected); assertTotalNumberOfInvocations(mutationDataBuilder, 1); @@ -140,7 +142,7 @@ public void testBytes() { ByteString expected = ByteString.copyFrom(value); ValueMapper mapper = new TestValueMapper(DEFAULT_COLUMN_FAMILY, DEFAULT_COLUMN, NullValueMode.IGNORE); - MutationDataBuilder mutationDataBuilder = mapper.getRecordMutationDataBuilder(value, TIMESTAMP); + MutationDataBuilder mutationDataBuilder = mapper.getRecordMutationDataBuilder(value, DEFAULT_TOPIC, TIMESTAMP); verify(mutationDataBuilder, times(1)) .setCell(DEFAULT_COLUMN_FAMILY, DEFAULT_COLUMN_BYTES, TIMESTAMP, expected); assertTotalNumberOfInvocations(mutationDataBuilder, 1); @@ -152,7 +154,7 @@ public void testFloat() { ByteString expected = ByteString.copyFrom(Bytes.toBytes(value)); ValueMapper mapper = new TestValueMapper(DEFAULT_COLUMN_FAMILY, DEFAULT_COLUMN, NullValueMode.IGNORE); - MutationDataBuilder mutationDataBuilder = mapper.getRecordMutationDataBuilder(value, TIMESTAMP); + MutationDataBuilder mutationDataBuilder = mapper.getRecordMutationDataBuilder(value, DEFAULT_TOPIC, TIMESTAMP); verify(mutationDataBuilder, times(1)) .setCell(DEFAULT_COLUMN_FAMILY, DEFAULT_COLUMN_BYTES, TIMESTAMP, expected); assertTotalNumberOfInvocations(mutationDataBuilder, 1); @@ -164,7 +166,7 @@ public void testDouble() { ByteString expected = ByteString.copyFrom(Bytes.toBytes(value)); ValueMapper mapper = new TestValueMapper(DEFAULT_COLUMN_FAMILY, DEFAULT_COLUMN, NullValueMode.IGNORE); - MutationDataBuilder mutationDataBuilder = mapper.getRecordMutationDataBuilder(value, TIMESTAMP); + MutationDataBuilder mutationDataBuilder = mapper.getRecordMutationDataBuilder(value, DEFAULT_TOPIC, TIMESTAMP); verify(mutationDataBuilder, times(1)) .setCell(DEFAULT_COLUMN_FAMILY, DEFAULT_COLUMN_BYTES, TIMESTAMP, expected); assertTotalNumberOfInvocations(mutationDataBuilder, 1); @@ -176,7 +178,7 @@ public void testDoubleSpecial() { ByteString expected = ByteString.copyFrom(Bytes.toBytes(value)); ValueMapper mapper = new TestValueMapper(DEFAULT_COLUMN_FAMILY, DEFAULT_COLUMN, NullValueMode.IGNORE); - MutationDataBuilder mutationDataBuilder = mapper.getRecordMutationDataBuilder(value, TIMESTAMP); + MutationDataBuilder mutationDataBuilder = mapper.getRecordMutationDataBuilder(value, DEFAULT_TOPIC, TIMESTAMP); verify(mutationDataBuilder, times(1)) .setCell(DEFAULT_COLUMN_FAMILY, DEFAULT_COLUMN_BYTES, TIMESTAMP, expected); assertTotalNumberOfInvocations(mutationDataBuilder, 1); @@ -191,7 +193,7 @@ public void testDate() { ValueMapper mapper = new TestValueMapper(DEFAULT_COLUMN_FAMILY, DEFAULT_COLUMN, NullValueMode.IGNORE); MutationDataBuilder mutationDataBuilder = - mapper.getRecordMutationDataBuilder(new Date(value), TIMESTAMP); + mapper.getRecordMutationDataBuilder(new Date(value), DEFAULT_TOPIC, TIMESTAMP); verify(mutationDataBuilder, times(1)) .setCell(DEFAULT_COLUMN_FAMILY, DEFAULT_COLUMN_BYTES, TIMESTAMP, expected); assertTotalNumberOfInvocations(mutationDataBuilder, 1); @@ -205,7 +207,7 @@ public void testDecimal() { ByteString expected = ByteString.copyFrom(Bytes.toBytes(value)); ValueMapper mapper = new TestValueMapper(DEFAULT_COLUMN_FAMILY, DEFAULT_COLUMN, NullValueMode.IGNORE); - MutationDataBuilder mutationDataBuilder = mapper.getRecordMutationDataBuilder(value, TIMESTAMP); + MutationDataBuilder mutationDataBuilder = mapper.getRecordMutationDataBuilder(value, DEFAULT_TOPIC, TIMESTAMP); verify(mutationDataBuilder, times(1)) .setCell(DEFAULT_COLUMN_FAMILY, DEFAULT_COLUMN_BYTES, TIMESTAMP, expected); assertTotalNumberOfInvocations(mutationDataBuilder, 1); @@ -216,7 +218,7 @@ public void testArray() { List value = List.of("1", 2, true); ValueMapper mapper = new TestValueMapper(DEFAULT_COLUMN_FAMILY, DEFAULT_COLUMN, NullValueMode.IGNORE); - MutationDataBuilder mutationDataBuilder = mapper.getRecordMutationDataBuilder(value, TIMESTAMP); + MutationDataBuilder mutationDataBuilder = mapper.getRecordMutationDataBuilder(value, DEFAULT_TOPIC, TIMESTAMP); verify(mutationDataBuilder, times(1)) .setCell( DEFAULT_COLUMN_FAMILY, @@ -235,7 +237,7 @@ public void testRootValueNeedsBothDefaultColumns() { new TestValueMapper(DEFAULT_COLUMN_FAMILY, null, NullValueMode.WRITE), new TestValueMapper(null, DEFAULT_COLUMN, NullValueMode.WRITE))) { MutationDataBuilder mutationDataBuilder = - mapper.getRecordMutationDataBuilder(value, TIMESTAMP); + mapper.getRecordMutationDataBuilder(value, DEFAULT_TOPIC, TIMESTAMP); verify(mutationDataBuilder, times(0)) .setCell( DEFAULT_COLUMN_FAMILY, @@ -245,7 +247,7 @@ public void testRootValueNeedsBothDefaultColumns() { } ValueMapper mapper = new TestValueMapper(DEFAULT_COLUMN_FAMILY, DEFAULT_COLUMN, NullValueMode.WRITE); - MutationDataBuilder mutationDataBuilder = mapper.getRecordMutationDataBuilder(value, TIMESTAMP); + MutationDataBuilder mutationDataBuilder = mapper.getRecordMutationDataBuilder(value, DEFAULT_TOPIC, TIMESTAMP); verify(mutationDataBuilder, times(1)) .setCell( DEFAULT_COLUMN_FAMILY, @@ -259,7 +261,7 @@ public void testRootValueNeedsBothDefaultColumns() { public void testValueNestedOnceNeedsOnlyDefaultColumnFamily() { Object value = fromJson("{\"key\": 2}"); ValueMapper mapper = new TestValueMapper(DEFAULT_COLUMN_FAMILY, null, NullValueMode.WRITE); - MutationDataBuilder mutationDataBuilder = mapper.getRecordMutationDataBuilder(value, TIMESTAMP); + MutationDataBuilder mutationDataBuilder = mapper.getRecordMutationDataBuilder(value, DEFAULT_TOPIC, TIMESTAMP); verify(mutationDataBuilder, times(1)) .setCell( DEFAULT_COLUMN_FAMILY, @@ -270,12 +272,36 @@ public void testValueNestedOnceNeedsOnlyDefaultColumnFamily() { assertTotalNumberOfInvocations(mutationDataBuilder, 1); } + @Test + public void testDefaultColumnFamilyInterpolation() { + for (Map.Entry test : + List.of( + new AbstractMap.SimpleImmutableEntry<>("prefix_${topic}_suffix", "prefix_topic_suffix"), + new AbstractMap.SimpleImmutableEntry<>( + "prefix_${topic_suffix", "prefix_${topic_suffix"), + new AbstractMap.SimpleImmutableEntry<>("prefix_$topic_suffix", "prefix_$topic_suffix"), + new AbstractMap.SimpleImmutableEntry<>("prefix_${bad}_suffix", "prefix_${bad}_suffix"), + new AbstractMap.SimpleImmutableEntry<>("noSubstitution", "noSubstitution"))) { + String topic = "topic"; + String value = "value"; + ValueMapper mapper = new TestValueMapper(test.getKey(), DEFAULT_COLUMN, NullValueMode.WRITE); + MutationDataBuilder mutationDataBuilder = mapper.getRecordMutationDataBuilder(value, topic, TIMESTAMP); + verify(mutationDataBuilder, times(1)) + .setCell( + test.getValue(), + DEFAULT_COLUMN_BYTES, + TIMESTAMP, + ByteString.copyFrom(value.getBytes(StandardCharsets.UTF_8))); + assertTotalNumberOfInvocations(mutationDataBuilder, 1); + } + } + @Test public void testMultipleOperationsAtOnce() { Object value = fromJson("{\"a\":{\"b\":789},\"c\":true,\"x\":{\"y\":null},\"z\":null}"); ValueMapper mapper = new TestValueMapper(DEFAULT_COLUMN_FAMILY, DEFAULT_COLUMN, NullValueMode.DELETE); - MutationDataBuilder mutationDataBuilder = mapper.getRecordMutationDataBuilder(value, TIMESTAMP); + MutationDataBuilder mutationDataBuilder = mapper.getRecordMutationDataBuilder(value, DEFAULT_TOPIC, TIMESTAMP); verify(mutationDataBuilder, times(1)) .setCell( "a", @@ -349,7 +375,7 @@ public void testMap() { ValueMapper mapper = new TestValueMapper(DEFAULT_COLUMN_FAMILY, DEFAULT_COLUMN, NullValueMode.DELETE); MutationDataBuilder mutationDataBuilder = - mapper.getRecordMutationDataBuilder(outerMap, TIMESTAMP); + mapper.getRecordMutationDataBuilder(outerMap, DEFAULT_TOPIC, TIMESTAMP); verify(mutationDataBuilder, times(1)).deleteFamily(familyToBeDeleted); verify(mutationDataBuilder, times(1)) .setCell(DEFAULT_COLUMN_FAMILY, valueKeyBytes, TIMESTAMP, valueBytes); @@ -430,7 +456,7 @@ public void testJsonificationOfNonJsonNativeTypes() { ValueMapper mapper = new TestValueMapper(null, null, NullValueMode.DELETE); MutationDataBuilder mutationDataBuilder = - mapper.getRecordMutationDataBuilder(outerMap, TIMESTAMP); + mapper.getRecordMutationDataBuilder(outerMap, DEFAULT_TOPIC, TIMESTAMP); verify(mutationDataBuilder, times(1)) .setCell( DEFAULT_COLUMN_FAMILY, @@ -509,7 +535,7 @@ public void testStruct() { ValueMapper mapper = new TestValueMapper(DEFAULT_COLUMN_FAMILY, DEFAULT_COLUMN, NullValueMode.DELETE); MutationDataBuilder mutationDataBuilder = - mapper.getRecordMutationDataBuilder(struct, TIMESTAMP); + mapper.getRecordMutationDataBuilder(struct, DEFAULT_TOPIC, TIMESTAMP); verify(mutationDataBuilder, times(1)) .setCell(DEFAULT_COLUMN_FAMILY, valueFieldNameBytes, TIMESTAMP, ByteString.copyFrom(value)); verify(mutationDataBuilder, times(1)).deleteFamily(optionalFieldName); @@ -537,7 +563,7 @@ public void testEmpty() { new TestValueMapper(DEFAULT_COLUMN_FAMILY, DEFAULT_COLUMN, NullValueMode.WRITE); for (Object value : List.of(emptyMap, emptyStruct)) { MutationDataBuilder mutationDataBuilder = - mapper.getRecordMutationDataBuilder(value, TIMESTAMP); + mapper.getRecordMutationDataBuilder(value, DEFAULT_TOPIC, TIMESTAMP); assertTotalNumberOfInvocations(mutationDataBuilder, 0); assertTrue(mutationDataBuilder.maybeBuild(TARGET_TABLE_NAME, ROW_KEY).isEmpty()); } @@ -547,7 +573,7 @@ public void testEmpty() { public void testSimpleCase1() { Object value = fromJson("{\"foo\": {\"bar\": 1}}"); ValueMapper mapper = new TestValueMapper(null, null, NullValueMode.IGNORE); - MutationDataBuilder mutationDataBuilder = mapper.getRecordMutationDataBuilder(value, TIMESTAMP); + MutationDataBuilder mutationDataBuilder = mapper.getRecordMutationDataBuilder(value, DEFAULT_TOPIC, TIMESTAMP); verify(mutationDataBuilder, times(1)) .setCell( "foo", @@ -562,7 +588,7 @@ public void testSimpleCase1() { public void testSimpleCase2() { Object value = fromJson("{\"foo\": {\"bar\": {\"fizz\": 1}}}"); ValueMapper mapper = new TestValueMapper(null, null, NullValueMode.IGNORE); - MutationDataBuilder mutationDataBuilder = mapper.getRecordMutationDataBuilder(value, TIMESTAMP); + MutationDataBuilder mutationDataBuilder = mapper.getRecordMutationDataBuilder(value, DEFAULT_TOPIC, TIMESTAMP); verify(mutationDataBuilder, times(1)) .setCell( "foo", @@ -576,7 +602,7 @@ public void testSimpleCase2() { public void testSimpleCase3() { Object value = fromJson("{\"foo\": 1}"); ValueMapper mapper = new TestValueMapper(DEFAULT_COLUMN_FAMILY, null, NullValueMode.IGNORE); - MutationDataBuilder mutationDataBuilder = mapper.getRecordMutationDataBuilder(value, TIMESTAMP); + MutationDataBuilder mutationDataBuilder = mapper.getRecordMutationDataBuilder(value, DEFAULT_TOPIC, TIMESTAMP); verify(mutationDataBuilder, times(1)) .setCell( DEFAULT_COLUMN_FAMILY, @@ -591,7 +617,7 @@ public void testSimpleCase3() { public void testNullModeIgnoreRoot() { ValueMapper mapper = new TestValueMapper(DEFAULT_COLUMN_FAMILY, DEFAULT_COLUMN, NullValueMode.IGNORE); - MutationDataBuilder mutationDataBuilder = mapper.getRecordMutationDataBuilder(null, TIMESTAMP); + MutationDataBuilder mutationDataBuilder = mapper.getRecordMutationDataBuilder(null, DEFAULT_TOPIC, TIMESTAMP); assertTotalNumberOfInvocations(mutationDataBuilder, 0); assertTrue(mutationDataBuilder.maybeBuild(TARGET_TABLE_NAME, ROW_KEY).isEmpty()); } @@ -601,7 +627,7 @@ public void testNullModeIgnoreNestedOnce() { ValueMapper mapper = new TestValueMapper(DEFAULT_COLUMN_FAMILY, DEFAULT_COLUMN, NullValueMode.IGNORE); MutationDataBuilder mutationDataBuilder = - mapper.getRecordMutationDataBuilder(getStructhWithNullOnNthNestingLevel(1), TIMESTAMP); + mapper.getRecordMutationDataBuilder(getStructhWithNullOnNthNestingLevel(1), DEFAULT_TOPIC, TIMESTAMP); assertTotalNumberOfInvocations(mutationDataBuilder, 0); assertTrue(mutationDataBuilder.maybeBuild(TARGET_TABLE_NAME, ROW_KEY).isEmpty()); } @@ -611,7 +637,7 @@ public void testNullModeIgnoreNestedTwice() { ValueMapper mapper = new TestValueMapper(DEFAULT_COLUMN_FAMILY, DEFAULT_COLUMN, NullValueMode.IGNORE); MutationDataBuilder mutationDataBuilder = - mapper.getRecordMutationDataBuilder(getStructhWithNullOnNthNestingLevel(2), TIMESTAMP); + mapper.getRecordMutationDataBuilder(getStructhWithNullOnNthNestingLevel(2), DEFAULT_TOPIC, TIMESTAMP); assertTotalNumberOfInvocations(mutationDataBuilder, 0); assertTrue(mutationDataBuilder.maybeBuild(TARGET_TABLE_NAME, ROW_KEY).isEmpty()); } @@ -620,7 +646,7 @@ public void testNullModeIgnoreNestedTwice() { public void testNullModeWriteRoot() { ValueMapper mapper = new TestValueMapper(DEFAULT_COLUMN_FAMILY, DEFAULT_COLUMN, NullValueMode.WRITE); - MutationDataBuilder mutationDataBuilder = mapper.getRecordMutationDataBuilder(null, TIMESTAMP); + MutationDataBuilder mutationDataBuilder = mapper.getRecordMutationDataBuilder(null, DEFAULT_TOPIC, TIMESTAMP); verify(mutationDataBuilder, times(1)) .setCell(DEFAULT_COLUMN_FAMILY, DEFAULT_COLUMN_BYTES, TIMESTAMP, ByteString.empty()); assertTotalNumberOfInvocations(mutationDataBuilder, 1); @@ -632,7 +658,7 @@ public void testNullModeWriteNestedOnce() { ValueMapper mapper = new TestValueMapper(DEFAULT_COLUMN_FAMILY, DEFAULT_COLUMN, NullValueMode.WRITE); MutationDataBuilder mutationDataBuilder = - mapper.getRecordMutationDataBuilder(getStructhWithNullOnNthNestingLevel(1), TIMESTAMP); + mapper.getRecordMutationDataBuilder(getStructhWithNullOnNthNestingLevel(1), DEFAULT_TOPIC, TIMESTAMP); verify(mutationDataBuilder, times(1)) .setCell( DEFAULT_COLUMN_FAMILY, @@ -648,7 +674,7 @@ public void testNullModeWriteNestedTwice() { ValueMapper mapper = new TestValueMapper(DEFAULT_COLUMN_FAMILY, DEFAULT_COLUMN, NullValueMode.WRITE); MutationDataBuilder mutationDataBuilder = - mapper.getRecordMutationDataBuilder(getStructhWithNullOnNthNestingLevel(2), TIMESTAMP); + mapper.getRecordMutationDataBuilder(getStructhWithNullOnNthNestingLevel(2), DEFAULT_TOPIC, TIMESTAMP); verify(mutationDataBuilder, times(1)) .setCell( NESTED_NULL_STRUCT_FIELD_NAME, @@ -663,7 +689,7 @@ public void testNullModeWriteNestedTwice() { public void testNullModeDeleteRoot() { ValueMapper mapper = new TestValueMapper(DEFAULT_COLUMN_FAMILY, DEFAULT_COLUMN, NullValueMode.DELETE); - MutationDataBuilder mutationDataBuilder = mapper.getRecordMutationDataBuilder(null, TIMESTAMP); + MutationDataBuilder mutationDataBuilder = mapper.getRecordMutationDataBuilder(null, DEFAULT_TOPIC, TIMESTAMP); verify(mutationDataBuilder, times(1)).deleteRow(); assertTotalNumberOfInvocations(mutationDataBuilder, 1); assertTrue(mutationDataBuilder.maybeBuild(TARGET_TABLE_NAME, ROW_KEY).isPresent()); @@ -674,7 +700,7 @@ public void testNullModeDeleteNestedOnce() { ValueMapper mapper = new TestValueMapper(DEFAULT_COLUMN_FAMILY, DEFAULT_COLUMN, NullValueMode.DELETE); MutationDataBuilder mutationDataBuilder = - mapper.getRecordMutationDataBuilder(getStructhWithNullOnNthNestingLevel(1), TIMESTAMP); + mapper.getRecordMutationDataBuilder(getStructhWithNullOnNthNestingLevel(1), DEFAULT_TOPIC, TIMESTAMP); verify(mutationDataBuilder, times(1)).deleteFamily(NESTED_NULL_STRUCT_FIELD_NAME); assertTotalNumberOfInvocations(mutationDataBuilder, 1); assertTrue(mutationDataBuilder.maybeBuild(TARGET_TABLE_NAME, ROW_KEY).isPresent()); @@ -685,7 +711,7 @@ public void testNullModeDeleteNestedTwice() { ValueMapper mapper = new TestValueMapper(DEFAULT_COLUMN_FAMILY, DEFAULT_COLUMN, NullValueMode.DELETE); MutationDataBuilder mutationDataBuilder = - mapper.getRecordMutationDataBuilder(getStructhWithNullOnNthNestingLevel(2), TIMESTAMP); + mapper.getRecordMutationDataBuilder(getStructhWithNullOnNthNestingLevel(2), DEFAULT_TOPIC, TIMESTAMP); verify(mutationDataBuilder, times(1)) .deleteCells( NESTED_NULL_STRUCT_FIELD_NAME, NESTED_NULL_STRUCT_FIELD_NAME_BYTES, TIMESTAMP_RANGE); @@ -701,7 +727,7 @@ public void testNullModeNestedThrice() { ByteString expectedJsonificationBytes = ByteString.copyFrom(expectedJsonification.getBytes(StandardCharsets.UTF_8)); MutationDataBuilder mutationDataBuilder = - mapper.getRecordMutationDataBuilder(getStructhWithNullOnNthNestingLevel(3), TIMESTAMP); + mapper.getRecordMutationDataBuilder(getStructhWithNullOnNthNestingLevel(3), DEFAULT_TOPIC, TIMESTAMP); verify(mutationDataBuilder, times(1)) .setCell( NESTED_NULL_STRUCT_FIELD_NAME, @@ -715,17 +741,13 @@ public void testNullModeNestedThrice() { private static Struct getStructhWithNullOnNthNestingLevel(int n) { assert n > 0; - Schema schema = - SchemaBuilder.struct() - .field(NESTED_NULL_STRUCT_FIELD_NAME, SchemaBuilder.struct().optional()) - .build(); + Schema schema = SchemaBuilder.struct().field(NESTED_NULL_STRUCT_FIELD_NAME, SchemaBuilder.struct().optional()).build(); // We consider a Struct with a null child to be a level 1 nested struct. Struct struct = new Struct(schema); while (n > 1) { n -= 1; - schema = - SchemaBuilder.struct().field(NESTED_NULL_STRUCT_FIELD_NAME, schema).optional().build(); + schema = SchemaBuilder.struct().field(NESTED_NULL_STRUCT_FIELD_NAME, schema).optional().build(); final Struct outerStruct = new Struct(schema); outerStruct.put(NESTED_NULL_STRUCT_FIELD_NAME, struct); struct = outerStruct; @@ -733,6 +755,18 @@ private static Struct getStructhWithNullOnNthNestingLevel(int n) { return struct; } + @Test + public void testDefaultColumnFamilySubstitution() { + ValueMapper mapper = + new TestValueMapper( + ConfigInterpolation.TOPIC_PLACEHOLDER, DEFAULT_COLUMN, NullValueMode.WRITE); + MutationDataBuilder mutationDataBuilder = mapper.getRecordMutationDataBuilder(null, DEFAULT_TOPIC, TIMESTAMP); + verify(mutationDataBuilder, times(1)) + .setCell(DEFAULT_TOPIC, DEFAULT_COLUMN_BYTES, TIMESTAMP, ByteString.empty()); + assertTotalNumberOfInvocations(mutationDataBuilder, 1); + assertTrue(mutationDataBuilder.maybeBuild(TARGET_TABLE_NAME, ROW_KEY).isPresent()); + } + private static Object fromJson(String s) { return jsonConverter.toConnectData(DEFAULT_TOPIC, s.getBytes(StandardCharsets.UTF_8)).value(); } From b1f59b1d7a9b89b056bec41f8e0d92dc6c48ad1e Mon Sep 17 00:00:00 2001 From: Adam Czajkowski Date: Thu, 30 Jan 2025 16:04:14 +0100 Subject: [PATCH 21/76] Support logical types --- .../connect/bigtable/BigtableSinkTask.java | 10 +- .../connect/bigtable/mapping/KeyMapper.java | 119 +++- .../bigtable/mapping/LogicalTypeUtils.java | 59 ++ .../bigtable/mapping/SchemaErrorReporter.java | 45 ++ .../connect/bigtable/mapping/SchemaUtils.java | 110 ++++ .../connect/bigtable/mapping/ValueMapper.java | 128 ++-- .../bigtable/mapping/KeyMapperTest.java | 612 ++++++++---------- .../mapping/LogicalTypeUtilsTest.java | 51 ++ .../bigtable/mapping/ValueMapperTest.java | 450 +++++++------ .../util/NestedNullStructFactory.java | 49 ++ 10 files changed, 1038 insertions(+), 595 deletions(-) create mode 100644 google-cloud-bigtable-kafka-connect-sink/src/main/java/com/google/cloud/kafka/connect/bigtable/mapping/LogicalTypeUtils.java create mode 100644 google-cloud-bigtable-kafka-connect-sink/src/main/java/com/google/cloud/kafka/connect/bigtable/mapping/SchemaErrorReporter.java create mode 100644 google-cloud-bigtable-kafka-connect-sink/src/main/java/com/google/cloud/kafka/connect/bigtable/mapping/SchemaUtils.java create mode 100644 google-cloud-bigtable-kafka-connect-sink/src/test/java/com/google/cloud/kafka/connect/bigtable/mapping/LogicalTypeUtilsTest.java create mode 100644 google-cloud-bigtable-kafka-connect-sink/src/test/java/com/google/cloud/kafka/connect/bigtable/util/NestedNullStructFactory.java diff --git a/google-cloud-bigtable-kafka-connect-sink/src/main/java/com/google/cloud/kafka/connect/bigtable/BigtableSinkTask.java b/google-cloud-bigtable-kafka-connect-sink/src/main/java/com/google/cloud/kafka/connect/bigtable/BigtableSinkTask.java index a7c61d6819..4e436b3898 100644 --- a/google-cloud-bigtable-kafka-connect-sink/src/main/java/com/google/cloud/kafka/connect/bigtable/BigtableSinkTask.java +++ b/google-cloud-bigtable-kafka-connect-sink/src/main/java/com/google/cloud/kafka/connect/bigtable/BigtableSinkTask.java @@ -50,6 +50,7 @@ import java.util.concurrent.ExecutionException; import java.util.concurrent.Future; import java.util.stream.Collectors; +import org.apache.kafka.connect.data.SchemaAndValue; import org.apache.kafka.connect.errors.ConnectException; import org.apache.kafka.connect.errors.DataException; import org.apache.kafka.connect.sink.ErrantRecordReporter; @@ -220,14 +221,17 @@ Map prepareRecords(Collection records) { @VisibleForTesting Optional createRecordMutationData(SinkRecord record) { String recordTableId = getTableName(record); - ByteString rowKey = ByteString.copyFrom(keyMapper.getKey(record.key())); + SchemaAndValue kafkaKey = new SchemaAndValue(record.keySchema(), record.key()); + ByteString rowKey = ByteString.copyFrom(keyMapper.getKey(kafkaKey)); if (rowKey.isEmpty()) { throw new DataException( "The record's key converts into an illegal empty Cloud Bigtable row key."); } + SchemaAndValue kafkaValue = new SchemaAndValue(record.valueSchema(), record.value()); long timestamp = getTimestampMicros(record); MutationDataBuilder mutationDataBuilder = - valueMapper.getRecordMutationDataBuilder(record.value(), record.topic(), timestamp); + valueMapper.getRecordMutationDataBuilder(kafkaValue, record.topic(), timestamp); + return mutationDataBuilder.maybeBuild(recordTableId, rowKey); } @@ -286,7 +290,7 @@ void reportError(SinkRecord record, Throwable throwable) { reporter.report(record, throwable); logger.warn( "Used DLQ for reporting a problem with a record (throwableClass={}).", - throwable.getClass().getSimpleName()); + throwable.getClass().getName()); } else { BigtableErrorMode errorMode = config.getBigtableErrorMode(); switch (errorMode) { diff --git a/google-cloud-bigtable-kafka-connect-sink/src/main/java/com/google/cloud/kafka/connect/bigtable/mapping/KeyMapper.java b/google-cloud-bigtable-kafka-connect-sink/src/main/java/com/google/cloud/kafka/connect/bigtable/mapping/KeyMapper.java index 179dd00c25..219e88468a 100644 --- a/google-cloud-bigtable-kafka-connect-sink/src/main/java/com/google/cloud/kafka/connect/bigtable/mapping/KeyMapper.java +++ b/google-cloud-bigtable-kafka-connect-sink/src/main/java/com/google/cloud/kafka/connect/bigtable/mapping/KeyMapper.java @@ -18,11 +18,15 @@ import com.google.cloud.kafka.connect.bigtable.config.BigtableSinkConfig; import java.io.ByteArrayOutputStream; import java.io.IOException; +import java.math.BigDecimal; import java.nio.ByteBuffer; import java.nio.charset.StandardCharsets; +import java.time.ZoneOffset; +import java.time.format.DateTimeFormatter; import java.util.AbstractMap; import java.util.Arrays; import java.util.Collections; +import java.util.Date; import java.util.Iterator; import java.util.List; import java.util.Map; @@ -30,7 +34,11 @@ import java.util.stream.Collectors; import java.util.stream.Stream; 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.errors.DataException; /** @@ -62,16 +70,22 @@ public KeyMapper(String delimiter, List definition) { * Converts input data into Cloud Bigtable row key bytes as described in {@link * BigtableSinkConfig#getDefinition()}. * - * @param kafkaKey An {@link Object} to be converted into Cloud Bigtable row key. + * @param kafkaKeyAndSchema Value and optional {@link Schema} of a Kafka message's key to be + * converted into a Cloud Bigtable row key. * @return {@link Optional#empty()} if the input doesn't convert into a valid Cloud Bigtable row * key, {@link Optional} containing row Cloud Bigtable row key bytes the input converts into * otherwise. */ - public byte[] getKey(Object kafkaKey) { + public byte[] getKey(SchemaAndValue kafkaKeyAndSchema) { + Object kafkaKey = kafkaKeyAndSchema.value(); + Optional kafkaKeySchema = Optional.ofNullable(kafkaKeyAndSchema.schema()); ensureKeyElementIsNotNull(kafkaKey); Stream keyParts = this.getDefinition(kafkaKey).stream() - .map((d) -> serializeTopLevelKeyElement(extractField(kafkaKey, d.iterator()))); + .map( + (d) -> + serializeTopLevelKeyElement( + extractField(kafkaKey, kafkaKeySchema, d.iterator()))); return concatenateByteArrays(new byte[0], keyParts, delimiter, new byte[0]); } @@ -82,8 +96,8 @@ public byte[] getKey(Object kafkaKey) { * @param kafkaKey {@link org.apache.kafka.connect.sink.SinkRecord SinkRecord's} key. * @return {@link List} containing {@link List Lists} of key fields that need to be retrieved and * concatenated to construct the Cloud Bigtable row key. - *

    See {@link KeyMapper#extractField(Object, Iterator)} for details on semantics of the - * inner list. + *

    See {@link KeyMapper#extractField(Object, Optional, Iterator)} for details on semantics + * of the inner list. */ private List> getDefinition(Object kafkaKey) { if (this.definition.isEmpty()) { @@ -112,10 +126,6 @@ private static Optional> getFieldsOfRootValue(Object kafkaKey) { return Optional.of( ((Struct) kafkaKey) .schema().fields().stream().map(Field::name).collect(Collectors.toList())); - } else if (kafkaKey instanceof Map) { - return Optional.of( - ((Map) kafkaKey) - .keySet().stream().map(Object::toString).collect(Collectors.toList())); } else { return Optional.empty(); } @@ -126,39 +136,39 @@ private static Optional> getFieldsOfRootValue(Object kafkaKey) { * * @param value {@link org.apache.kafka.connect.sink.SinkRecord SinkRecord's} key or some its * child. + * @param schema A schema of {@code value}. * @param fields Fields that need to be accessed before the target value is reached. * @return Extracted nested field. */ - private Object extractField(Object value, Iterator fields) { + private SchemaAndValue extractField( + Object value, Optional schema, Iterator fields) { ensureKeyElementIsNotNull(value); + LogicalTypeUtils.logIfLogicalTypeUnsupported(schema); if (!fields.hasNext()) { - return value; + return new SchemaAndValue(schema.orElse(null), value); } String field = fields.next(); if (value instanceof Struct) { Struct struct = (Struct) value; // Note that getWithoutDefault() throws if such a field does not exist. - return extractField(struct.getWithoutDefault(field), fields); - } else if (value instanceof Map) { - Map map = (Map) value; - if (!map.containsKey(field)) { - throw new DataException("Map contains no value for key `" + field + "`."); - } - return extractField(map.get(field), fields); + return extractField( + struct.getWithoutDefault(field), + SchemaUtils.maybeExtractFieldSchema(schema, field), + fields); } else { throw new DataException( "Unexpected class `" - + value.getClass() - + "` doesn't " - + "support extracting field `" + + value.getClass().getName() + + "` doesn't support extracting field `" + field + "` using a dot."); } } - private static byte[] serializeTopLevelKeyElement(Object keyElement) { + private static byte[] serializeTopLevelKeyElement(SchemaAndValue keyElementAndSchema) { + Object keyElement = keyElementAndSchema.value(); ensureKeyElementIsNotNull(keyElement); - return serializeKeyElement(keyElement); + return serializeKeyElement(keyElement, Optional.ofNullable(keyElementAndSchema.schema())); } /** @@ -168,23 +178,41 @@ private static byte[] serializeTopLevelKeyElement(Object keyElement) { * * @param keyElement {@link org.apache.kafka.connect.sink.SinkRecord SinkRecord's} key to be * serialized. + * @param keyElementSchema An optional schema of {@code keyElement}. * @return Serialization of the input value. */ - private static byte[] serializeKeyElement(Object keyElement) { + private static byte[] serializeKeyElement(Object keyElement, Optional keyElementSchema) { if (keyElement == null) { // Note that it's needed for serializing null-containing Maps and Lists. - return "null".getBytes(StandardCharsets.UTF_8); + return serializeKeyElement("null", Optional.empty()); } else if (keyElement instanceof byte[]) { - // Note that it breaks compatibility with Confluent's sink. return (byte[]) keyElement; + } else if (keyElement instanceof String) { + return ((String) keyElement).getBytes(StandardCharsets.UTF_8); } else if (keyElement instanceof ByteBuffer) { return ((ByteBuffer) keyElement).array(); + } else if (keyElement instanceof Boolean + || keyElement instanceof Byte + || keyElement instanceof Short + || keyElement instanceof Integer + || keyElement instanceof Long + || keyElement instanceof Float + || keyElement instanceof Double + || keyElement instanceof Character) { + return serializeKeyElement(keyElement.toString(), Optional.empty()); } else if (keyElement instanceof List) { + // Note that it breaks compatibility with Confluent's sink when serializing byte array + // elements. List list = (List) keyElement; + Optional elementSchema = SchemaUtils.maybeExtractValueSchema(keyElementSchema); return concatenateByteArrays( - "[", list.stream().map(o -> o.toString().getBytes(StandardCharsets.UTF_8)), ", ", "]"); + "[", list.stream().map(e -> serializeKeyElement(e, elementSchema)), ", ", "]"); } else if (keyElement instanceof Map) { + // Note that it breaks compatibility with Confluent's sink when serializing byte array keys or + // values. Map map = (Map) keyElement; + Optional keySchema = SchemaUtils.maybeExtractKeySchema(keyElementSchema); + Optional valueSchema = SchemaUtils.maybeExtractValueSchema(keyElementSchema); return concatenateByteArrays( "{", // Note that it inherits ordering of entries from the configured converter. @@ -194,14 +222,17 @@ private static byte[] serializeKeyElement(Object keyElement) { concatenateByteArrays( new byte[0], Stream.of( - serializeKeyElement(e.getKey()), serializeKeyElement(e.getValue())), + serializeKeyElement(e.getKey(), keySchema), + serializeKeyElement(e.getValue(), valueSchema)), "=".getBytes(StandardCharsets.UTF_8), new byte[0])), // Note that Map and Struct have different delimiters for compatibility's sake. ", ", "}"); } else if (keyElement instanceof Struct) { + // Note that it breaks compatibility with Confluent's sink when serializing byte array fields. Struct struct = (Struct) keyElement; + Optional fieldNameSchema = Optional.empty(); return concatenateByteArrays( "Struct{", struct.schema().fields().stream() @@ -215,15 +246,41 @@ private static byte[] serializeKeyElement(Object keyElement) { concatenateByteArrays( new byte[0], Stream.of( - serializeKeyElement(e.getKey()), serializeKeyElement(e.getValue())), + serializeKeyElement(e.getKey(), Optional.empty()), + serializeKeyElement( + e.getValue(), + SchemaUtils.maybeExtractFieldSchema( + keyElementSchema, e.getKey()))), "=".getBytes(StandardCharsets.UTF_8), new byte[0])), // Note that Map and Struct have different delimiters for compatibility's sake. ",", "}"); + } else if (keyElement instanceof Date) { + // Note that it breaks compatibility with Confluent's sink, which seems to use toString(). + DateTimeFormatter fmt; + switch (keyElementSchema.map(Schema::name).orElse(Timestamp.LOGICAL_NAME)) { + case org.apache.kafka.connect.data.Date.LOGICAL_NAME: + fmt = DateTimeFormatter.ISO_LOCAL_DATE; + break; + case Time.LOGICAL_NAME: + fmt = DateTimeFormatter.ISO_LOCAL_TIME; + break; + default: + fmt = DateTimeFormatter.ISO_INSTANT; + break; + } + String formatted = fmt.format(((Date) keyElement).toInstant().atZone(ZoneOffset.UTC)); + return serializeKeyElement(formatted, Optional.empty()); + } else if (keyElement instanceof BigDecimal) { + // Note that it breaks compatibility with Confluent's sink when serializing values for which + // toString() returns values using scientific notation. + return serializeKeyElement(((BigDecimal) keyElement).toPlainString(), Optional.empty()); } else { - // TODO: handle logical data types. - return keyElement.toString().getBytes(StandardCharsets.UTF_8); + throw new DataException( + "Unsupported serialization of an unexpected class `" + + keyElement.getClass().getName() + + "` in key."); } } diff --git a/google-cloud-bigtable-kafka-connect-sink/src/main/java/com/google/cloud/kafka/connect/bigtable/mapping/LogicalTypeUtils.java b/google-cloud-bigtable-kafka-connect-sink/src/main/java/com/google/cloud/kafka/connect/bigtable/mapping/LogicalTypeUtils.java new file mode 100644 index 0000000000..8287af5665 --- /dev/null +++ b/google-cloud-bigtable-kafka-connect-sink/src/main/java/com/google/cloud/kafka/connect/bigtable/mapping/LogicalTypeUtils.java @@ -0,0 +1,59 @@ +/* + * Copyright 2025 Google LLC + * + * Licensed under the Apache License, Version 2.0 (the "License"); + * you may not use this file except in compliance with the License. + * You may obtain a copy of the License at + * + * https://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ +package com.google.cloud.kafka.connect.bigtable.mapping; + +import java.util.HashSet; +import java.util.Optional; +import java.util.Set; +import org.apache.kafka.connect.data.Date; +import org.apache.kafka.connect.data.Decimal; +import org.apache.kafka.connect.data.Schema; +import org.apache.kafka.connect.data.Time; +import org.apache.kafka.connect.data.Timestamp; + +/** A class facilitating support for Kafka Connect logical types. */ +public class LogicalTypeUtils { + private static final Set SUPPORTED_LOGICAL_TYPES = + Set.of(Date.LOGICAL_NAME, Time.LOGICAL_NAME, Timestamp.LOGICAL_NAME, Decimal.LOGICAL_NAME); + private static final Set REPORTED_UNSUPPORTED_LOGICAL_TYPES = new HashSet<>(); + + /** + * Logs a warning if the encountered logical type is unsupported. To prevent flooding the log, it + * only reports each unsupported logical type at most once for each {@link + * com.google.cloud.kafka.connect.bigtable.BigtableSinkTask}. + * + * @param schema A schema of a Kafka key or value about which type's lack of support the user + * should be notified. + */ + public static void logIfLogicalTypeUnsupported(Optional schema) { + String logicalTypeName = schema.map(Schema::name).orElse(null); + boolean isStruct = schema.map(s -> s.type() == Schema.Type.STRUCT).orElse(false); + // Note that `Struct`s may have non-null names, but we do support them (ignoring their name). + if (isStruct || logicalTypeName == null) { + return; + } + if (!SUPPORTED_LOGICAL_TYPES.contains(logicalTypeName)) { + synchronized (REPORTED_UNSUPPORTED_LOGICAL_TYPES) { + SchemaErrorReporter.reportProblemAtMostOnce( + REPORTED_UNSUPPORTED_LOGICAL_TYPES, + logicalTypeName, + "Unsupported logical type: `" + + logicalTypeName + + "`. Falling back to handling the value according to its physical type."); + } + } + } +} diff --git a/google-cloud-bigtable-kafka-connect-sink/src/main/java/com/google/cloud/kafka/connect/bigtable/mapping/SchemaErrorReporter.java b/google-cloud-bigtable-kafka-connect-sink/src/main/java/com/google/cloud/kafka/connect/bigtable/mapping/SchemaErrorReporter.java new file mode 100644 index 0000000000..87f0ff360c --- /dev/null +++ b/google-cloud-bigtable-kafka-connect-sink/src/main/java/com/google/cloud/kafka/connect/bigtable/mapping/SchemaErrorReporter.java @@ -0,0 +1,45 @@ +/* + * Copyright 2025 Google LLC + * + * Licensed under the Apache License, Version 2.0 (the "License"); + * you may not use this file except in compliance with the License. + * You may obtain a copy of the License at + * + * https://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ +package com.google.cloud.kafka.connect.bigtable.mapping; + +import java.util.Set; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +/** A helper class allowing to report some problem only once intended for use with */ +public class SchemaErrorReporter { + private static final Logger LOGGER = LoggerFactory.getLogger(SchemaErrorReporter.class); + + /** + * Logs a warning about a problem. + * + *

    Further warnings related to the same {@code problem} problem will not be logged to avoid the + * possibility of flooding the log with warnings for possibly each encountered Kafka message. + * + * @param alreadyReportedProblems A {@link Set} of problems for which error messages have already + * been sent. + * @param problem A problem associated with {@code problemMessage}. + * @param problemMessage A message to be logged. + * @param Identifier of a problem. + */ + public static void reportProblemAtMostOnce( + Set alreadyReportedProblems, T problem, String problemMessage) { + boolean firstSeen = alreadyReportedProblems.add(problem); + if (firstSeen) { + LOGGER.warn(problemMessage); + } + } +} diff --git a/google-cloud-bigtable-kafka-connect-sink/src/main/java/com/google/cloud/kafka/connect/bigtable/mapping/SchemaUtils.java b/google-cloud-bigtable-kafka-connect-sink/src/main/java/com/google/cloud/kafka/connect/bigtable/mapping/SchemaUtils.java new file mode 100644 index 0000000000..83bff2a8af --- /dev/null +++ b/google-cloud-bigtable-kafka-connect-sink/src/main/java/com/google/cloud/kafka/connect/bigtable/mapping/SchemaUtils.java @@ -0,0 +1,110 @@ +/* + * Copyright 2025 Google LLC + * + * Licensed under the Apache License, Version 2.0 (the "License"); + * you may not use this file except in compliance with the License. + * You may obtain a copy of the License at + * + * https://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ +package com.google.cloud.kafka.connect.bigtable.mapping; + +import java.util.HashSet; +import java.util.Objects; +import java.util.Optional; +import java.util.Set; +import org.apache.kafka.connect.data.Field; +import org.apache.kafka.connect.data.Schema; + +/** + * A class containing operations on {@link Schema Schema(s)}. + * + *

    The mappers ({@link KeyMapper}, {@link ValueMapper}) that make use of this class treat {@link + * Schema Schema (s)} as a hint, not as a requirement. Thus, despite the fact that the operations in + * {@code try} blocks should not throw, we do catch possible exceptions in order not to fail the + * records in case the converter the user configured does something unexpected. + * + *

    This class uses {@link SchemaErrorReporter#reportProblemAtMostOnce(Set, Object, String)} to + * avoid flooding the logs since these errors are not fatal and could pessimistically be very + * frequent. + */ +public class SchemaUtils { + private static final Set PROBLEMS_WITH_VALUE_SCHEMA = new HashSet<>(); + private static final Set PROBLEMS_WITH_KEY_SCHEMA = new HashSet<>(); + private static final Set PROBLEMS_WITH_FIELD_SCHEMA = new HashSet<>(); + + public static Optional maybeExtractValueSchema(Optional maybeSchema) { + try { + return maybeSchema.map(Schema::valueSchema); + } catch (Exception ignored) { + synchronized (PROBLEMS_WITH_VALUE_SCHEMA) { + maybeSchema.ifPresent( + s -> + SchemaErrorReporter.reportProblemAtMostOnce( + PROBLEMS_WITH_VALUE_SCHEMA, + s, + String.format( + "A Kafka Connect object with schema `%s` has no schema for its values.", + s))); + } + return Optional.empty(); + } + } + + public static Optional maybeExtractKeySchema(Optional maybeSchema) { + try { + return maybeSchema.map(Schema::keySchema); + } catch (Exception ignored) { + synchronized (PROBLEMS_WITH_KEY_SCHEMA) { + maybeSchema.ifPresent( + s -> + SchemaErrorReporter.reportProblemAtMostOnce( + PROBLEMS_WITH_KEY_SCHEMA, + s, + String.format( + "A Kafka Connect object with schema `%s` has no schema for its keys.", s))); + return Optional.empty(); + } + } + } + + public static Optional maybeExtractFieldSchema( + Optional maybeSchema, String field) { + try { + return maybeSchema.map(s -> s.field(field)).map(Field::schema); + } catch (Exception ignored) { + maybeSchema.ifPresent( + s -> + SchemaErrorReporter.reportProblemAtMostOnce( + PROBLEMS_WITH_FIELD_SCHEMA, + new SchemaAndField(s, field), + String.format( + "A Kafka Connect object with schema `%s` has no schema for its field `%s`.", + s, field))); + return Optional.empty(); + } + } + + private static class SchemaAndField { + private final Schema schema; + private final String field; + + public SchemaAndField(Schema schema, String field) { + this.schema = schema; + this.field = field; + } + + @Override + public boolean equals(Object o) { + if (o == null || getClass() != o.getClass()) return false; + SchemaAndField other = (SchemaAndField) o; + return Objects.equals(schema, other.schema) && Objects.equals(field, other.field); + } + } +} diff --git a/google-cloud-bigtable-kafka-connect-sink/src/main/java/com/google/cloud/kafka/connect/bigtable/mapping/ValueMapper.java b/google-cloud-bigtable-kafka-connect-sink/src/main/java/com/google/cloud/kafka/connect/bigtable/mapping/ValueMapper.java index 5cefd00dde..3255e1c440 100644 --- a/google-cloud-bigtable-kafka-connect-sink/src/main/java/com/google/cloud/kafka/connect/bigtable/mapping/ValueMapper.java +++ b/google-cloud-bigtable-kafka-connect-sink/src/main/java/com/google/cloud/kafka/connect/bigtable/mapping/ValueMapper.java @@ -15,13 +15,14 @@ */ package com.google.cloud.kafka.connect.bigtable.mapping; +import static com.google.cloud.kafka.connect.bigtable.mapping.LogicalTypeUtils.logIfLogicalTypeUnsupported; + import com.fasterxml.jackson.core.JsonGenerator; import com.fasterxml.jackson.core.JsonProcessingException; import com.fasterxml.jackson.databind.ObjectMapper; import com.fasterxml.jackson.databind.SerializerProvider; import com.fasterxml.jackson.databind.module.SimpleModule; import com.fasterxml.jackson.databind.ser.std.StdSerializer; -import com.google.cloud.ByteArray; import com.google.cloud.bigtable.data.v2.models.Range; import com.google.cloud.kafka.connect.bigtable.config.ConfigInterpolation; import com.google.cloud.kafka.connect.bigtable.config.NullValueMode; @@ -29,19 +30,24 @@ import com.google.protobuf.ByteString; import java.io.IOException; import java.math.BigDecimal; +import java.nio.ByteBuffer; import java.nio.charset.StandardCharsets; import java.util.AbstractMap; -import java.util.ArrayList; import java.util.Date; import java.util.List; import java.util.Map; +import java.util.Optional; import java.util.stream.Collectors; import javax.annotation.Nonnull; import org.apache.hadoop.hbase.util.Bytes; import org.apache.kafka.common.utils.Utils; +import org.apache.kafka.connect.data.Decimal; 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.errors.DataException; /** @@ -78,31 +84,42 @@ public ValueMapper( * Creates a {@link MutationDataBuilder} that can be used to create a {@link MutationData} * representing the input Kafka Connect value as Cloud Bigtable mutations that need to be applied. * - * @param rootKafkaValue The value to be converted into Cloud Bigtable {@link - * com.google.cloud.bigtable.data.v2.models.Mutation Mutation(s)}. + * @param kafkaValueAndSchema The value to be converted into Cloud Bigtable {@link + * com.google.cloud.bigtable.data.v2.models.Mutation Mutation(s)} and its optional {@link + * Schema}. * @param topic The name of Kafka topic this value originates from. * @param timestampMicros The timestamp the mutations will be created at in microseconds. */ public MutationDataBuilder getRecordMutationDataBuilder( - Object rootKafkaValue, String topic, long timestampMicros) { + SchemaAndValue kafkaValueAndSchema, String topic, long timestampMicros) { + Object rootKafkaValue = kafkaValueAndSchema.value(); + Optional rootKafkaSchema = Optional.ofNullable(kafkaValueAndSchema.schema()); + logIfLogicalTypeUnsupported(rootKafkaSchema); MutationDataBuilder mutationDataBuilder = createMutationDataBuilder(); if (rootKafkaValue == null && nullMode == NullValueMode.IGNORE) { // Do nothing } else if (rootKafkaValue == null && nullMode == NullValueMode.DELETE) { mutationDataBuilder.deleteRow(); - } else if (rootKafkaValue instanceof Map || rootKafkaValue instanceof Struct) { - for (Map.Entry field : getChildren(rootKafkaValue)) { + } else if (rootKafkaValue instanceof Struct) { + for (Map.Entry field : + getChildren((Struct) rootKafkaValue, rootKafkaSchema)) { String kafkaFieldName = field.getKey().toString(); - Object kafkaFieldValue = field.getValue(); + Object kafkaFieldValue = field.getValue().value(); + Optional kafkaFieldSchema = Optional.ofNullable(field.getValue().schema()); + logIfLogicalTypeUnsupported(kafkaFieldSchema); if (kafkaFieldValue == null && nullMode == NullValueMode.IGNORE) { continue; } else if (kafkaFieldValue == null && nullMode == NullValueMode.DELETE) { mutationDataBuilder.deleteFamily(kafkaFieldName); - } else if (kafkaFieldValue instanceof Map || kafkaFieldValue instanceof Struct) { - for (Map.Entry subfield : getChildren(kafkaFieldValue)) { + } else if (kafkaFieldValue instanceof Struct) { + for (Map.Entry subfield : + getChildren((Struct) kafkaFieldValue, kafkaFieldSchema)) { ByteString kafkaSubfieldName = ByteString.copyFrom(subfield.getKey().toString().getBytes(StandardCharsets.UTF_8)); - Object kafkaSubfieldValue = subfield.getValue(); + Object kafkaSubfieldValue = subfield.getValue().value(); + Optional kafkaSubfieldSchema = + Optional.ofNullable(subfield.getValue().schema()); + logIfLogicalTypeUnsupported(kafkaSubfieldSchema); if (kafkaSubfieldValue == null && nullMode == NullValueMode.IGNORE) { continue; } else if (kafkaSubfieldValue == null && nullMode == NullValueMode.DELETE) { @@ -115,7 +132,7 @@ public MutationDataBuilder getRecordMutationDataBuilder( kafkaFieldName, kafkaSubfieldName, timestampMicros, - ByteString.copyFrom(serialize(kafkaSubfieldValue))); + ByteString.copyFrom(serialize(kafkaSubfieldValue, kafkaSubfieldSchema))); } } } else { @@ -124,7 +141,7 @@ public MutationDataBuilder getRecordMutationDataBuilder( getDefaultColumnFamily(topic), ByteString.copyFrom(kafkaFieldName.getBytes(StandardCharsets.UTF_8)), timestampMicros, - ByteString.copyFrom(serialize(kafkaFieldValue))); + ByteString.copyFrom(serialize(kafkaFieldValue, kafkaFieldSchema))); } } } @@ -134,7 +151,7 @@ public MutationDataBuilder getRecordMutationDataBuilder( getDefaultColumnFamily(topic), defaultColumnQualifier, timestampMicros, - ByteString.copyFrom(serialize(rootKafkaValue))); + ByteString.copyFrom(serialize(rootKafkaValue, rootKafkaSchema))); } } return mutationDataBuilder; @@ -151,40 +168,37 @@ protected String getDefaultColumnFamily(String topic) { } /** - * @param mapOrStruct {@link Map} or {@link Struct} whose children we want to list - * @return {@link List} of names or keys of input value's child entries. + * @param struct {@link Struct} whose children we want to list + * @return {@link List} of pairs of field names and values (with optional schemas) of {@code + * struct}'s fields. */ - private static List> getChildren(Object mapOrStruct) { - if (mapOrStruct instanceof Map) { - @SuppressWarnings("unchecked") - Map kafkaMapValue = (Map) mapOrStruct; - return new ArrayList<>(kafkaMapValue.entrySet()); - } else if (mapOrStruct instanceof Struct) { - Struct kafkaStructValue = (Struct) mapOrStruct; - return kafkaStructValue.schema().fields().stream() - .map( - f -> - new AbstractMap.SimpleImmutableEntry<>( - (Object) f.name(), kafkaStructValue.get(f))) - .collect(Collectors.toList()); - } else { - throw new IllegalStateException(); - } + private static List> getChildren( + Struct struct, Optional schema) { + return struct.schema().fields().stream() + .map(Field::name) + .map( + f -> + new AbstractMap.SimpleImmutableEntry<>( + (Object) f, + new SchemaAndValue( + SchemaUtils.maybeExtractFieldSchema(schema, f).orElse(null), + struct.get(f)))) + .collect(Collectors.toList()); } /** * @param value Input value. + * @param schema An optional schema of {@code value}. * @return Input value's serialization's bytes that will be written to Cloud Bigtable as a cell's * value. */ - private static byte[] serialize(Object value) { + private static byte[] serialize(Object value, Optional schema) { if (value == null) { return new byte[0]; - } - if (value instanceof byte[]) { + } else if (value instanceof byte[]) { return (byte[]) value; - } else if (value instanceof ByteArray) { - return serialize(((ByteArray) value).toByteArray()); + } else if (value instanceof ByteBuffer) { + return serialize(((ByteBuffer) value).array(), Optional.empty()); } else if (value instanceof Integer) { return Bytes.toBytes((Integer) value); } else if (value instanceof Long) { @@ -202,22 +216,23 @@ private static byte[] serialize(Object value) { } else if (value instanceof String) { return Bytes.toBytes((String) value); } else if (value instanceof Character) { - return serialize(Character.toString((Character) value)); + return serialize(Character.toString((Character) value), Optional.empty()); } else if (value instanceof Date) { - // TODO: implement. - throw new DataException("TODO"); + // Note that the value might have different Kafka Connect schema: Date, Time or Timestamp. + return serialize(((Date) value).getTime(), Optional.empty()); } else if (value instanceof BigDecimal) { - // TODO: implement. - throw new DataException("TODO"); + return Bytes.toBytes((BigDecimal) value); } else if (value instanceof Map || value instanceof Struct || value instanceof List) { try { return jsonMapper.writeValueAsBytes(value); } catch (JsonProcessingException e) { - throw new DataException("Failed to deserialize a(n) " + value.getClass(), e); + throw new DataException("Failed to deserialize a(n) " + value.getClass().getName(), e); } } else { throw new DataException( - "Unsupported serialization of an unexpected class `" + value.getClass() + "`."); + "Unsupported serialization of an unexpected class `" + + value.getClass().getName() + + "` in value."); } } @@ -241,12 +256,35 @@ protected StructJsonSerializer(Class t) { public void serialize(Struct value, JsonGenerator gen, SerializerProvider provider) throws IOException { Schema schema = value.schema(); + logIfLogicalTypeUnsupported(Optional.ofNullable(schema)); gen.writeStartObject(); for (Field field : schema.fields()) { String fieldName = field.name(); - gen.writeObjectField(fieldName, value.getWithoutDefault(fieldName)); + Schema fieldSchema = field.schema(); + logIfLogicalTypeUnsupported(Optional.ofNullable(fieldSchema)); + Object logicalFieldValue = value.getWithoutDefault(fieldName); + Object physicalFieldValue = convertToPhysicalValue(logicalFieldValue, fieldSchema); + gen.writeObjectField(fieldName, physicalFieldValue); } gen.writeEndObject(); } + + private Object convertToPhysicalValue(Object logicalFieldValue, Schema fieldSchema) { + if (logicalFieldValue instanceof Date) { + Date date = (Date) logicalFieldValue; + String logicalName = Optional.ofNullable(fieldSchema).map(Schema::name).orElse(""); + switch (logicalName) { + case Timestamp.LOGICAL_NAME: + return Timestamp.fromLogical(fieldSchema, date); + case Time.LOGICAL_NAME: + return Time.fromLogical(fieldSchema, date); + case org.apache.kafka.connect.data.Date.LOGICAL_NAME: + return org.apache.kafka.connect.data.Date.fromLogical(fieldSchema, date); + } + } else if (logicalFieldValue instanceof BigDecimal) { + return Decimal.fromLogical(fieldSchema, (BigDecimal) logicalFieldValue); + } + return logicalFieldValue; + } } } diff --git a/google-cloud-bigtable-kafka-connect-sink/src/test/java/com/google/cloud/kafka/connect/bigtable/mapping/KeyMapperTest.java b/google-cloud-bigtable-kafka-connect-sink/src/test/java/com/google/cloud/kafka/connect/bigtable/mapping/KeyMapperTest.java index 8135ef83b2..39aa05efac 100644 --- a/google-cloud-bigtable-kafka-connect-sink/src/test/java/com/google/cloud/kafka/connect/bigtable/mapping/KeyMapperTest.java +++ b/google-cloud-bigtable-kafka-connect-sink/src/test/java/com/google/cloud/kafka/connect/bigtable/mapping/KeyMapperTest.java @@ -25,7 +25,6 @@ import static org.junit.Assert.assertArrayEquals; import static org.junit.Assert.assertEquals; import static org.junit.Assert.assertThrows; -import static org.junit.Assert.assertTrue; import com.google.cloud.kafka.connect.bigtable.util.JsonConverterFactory; import java.io.ByteArrayOutputStream; @@ -33,6 +32,8 @@ import java.math.BigDecimal; import java.nio.ByteBuffer; import java.nio.charset.StandardCharsets; +import java.time.ZoneOffset; +import java.time.format.DateTimeFormatter; import java.util.Arrays; import java.util.Date; import java.util.HashMap; @@ -52,7 +53,7 @@ // Confluent sink's implementations. @RunWith(JUnit4.class) public class KeyMapperTest { - private static String DELIMITER = "##"; + private static final String DELIMITER = "##"; @Test public void testBoolean() { @@ -63,18 +64,15 @@ public void testBoolean() { Struct kafkaConnectStruct = new Struct(kafkaConnectSchema); kafkaConnectStruct.put(fieldName, fieldValue); - assertTrue( - Arrays.equals( - calculateKey(List.of(), DELIMITER, fieldValue), - fieldValue.toString().getBytes(StandardCharsets.UTF_8))); - assertTrue( - Arrays.equals( - calculateKey(List.of(), DELIMITER, kafkaConnectStruct), - fieldValue.toString().getBytes(StandardCharsets.UTF_8))); - assertTrue( - Arrays.equals( - calculateKey(List.of(fieldName), DELIMITER, kafkaConnectStruct), - fieldValue.toString().getBytes(StandardCharsets.UTF_8))); + assertArrayEquals( + calculateKey(List.of(), DELIMITER, fieldValue), + fieldValue.toString().getBytes(StandardCharsets.UTF_8)); + assertArrayEquals( + calculateKey(List.of(), DELIMITER, kafkaConnectStruct), + fieldValue.toString().getBytes(StandardCharsets.UTF_8)); + assertArrayEquals( + calculateKey(List.of(fieldName), DELIMITER, kafkaConnectStruct), + fieldValue.toString().getBytes(StandardCharsets.UTF_8)); } @Test @@ -84,19 +82,15 @@ public void testByte() { Schema kafkaConnectSchema = SchemaBuilder.struct().field(fieldName, Schema.INT8_SCHEMA).build(); Struct kafkaConnectStruct = new Struct(kafkaConnectSchema); kafkaConnectStruct.put(fieldName, fieldByteValue); - - assertTrue( - Arrays.equals( - calculateKey(List.of(), DELIMITER, fieldByteValue), - fieldByteValue.toString().getBytes(StandardCharsets.UTF_8))); - assertTrue( - Arrays.equals( - calculateKey(List.of(), DELIMITER, kafkaConnectStruct), - fieldByteValue.toString().getBytes(StandardCharsets.UTF_8))); - assertTrue( - Arrays.equals( - calculateKey(List.of(fieldName), DELIMITER, kafkaConnectStruct), - fieldByteValue.toString().getBytes(StandardCharsets.UTF_8))); + assertArrayEquals( + calculateKey(List.of(), DELIMITER, fieldByteValue), + fieldByteValue.toString().getBytes(StandardCharsets.UTF_8)); + assertArrayEquals( + calculateKey(List.of(), DELIMITER, kafkaConnectStruct), + fieldByteValue.toString().getBytes(StandardCharsets.UTF_8)); + assertArrayEquals( + calculateKey(List.of(fieldName), DELIMITER, kafkaConnectStruct), + fieldByteValue.toString().getBytes(StandardCharsets.UTF_8)); } @Test @@ -107,19 +101,15 @@ public void testShort() { SchemaBuilder.struct().field(fieldName, Schema.INT16_SCHEMA).build(); Struct kafkaConnectStruct = new Struct(kafkaConnectSchema); kafkaConnectStruct.put(fieldName, fieldShortValue); - - assertTrue( - Arrays.equals( - calculateKey(List.of(), DELIMITER, fieldShortValue), - fieldShortValue.toString().getBytes(StandardCharsets.UTF_8))); - assertTrue( - Arrays.equals( - calculateKey(List.of(), DELIMITER, kafkaConnectStruct), - fieldShortValue.toString().getBytes(StandardCharsets.UTF_8))); - assertTrue( - Arrays.equals( - calculateKey(List.of(fieldName), DELIMITER, kafkaConnectStruct), - fieldShortValue.toString().getBytes(StandardCharsets.UTF_8))); + assertArrayEquals( + calculateKey(List.of(), DELIMITER, fieldShortValue), + fieldShortValue.toString().getBytes(StandardCharsets.UTF_8)); + assertArrayEquals( + calculateKey(List.of(), DELIMITER, kafkaConnectStruct), + fieldShortValue.toString().getBytes(StandardCharsets.UTF_8)); + assertArrayEquals( + calculateKey(List.of(fieldName), DELIMITER, kafkaConnectStruct), + fieldShortValue.toString().getBytes(StandardCharsets.UTF_8)); } @Test @@ -130,19 +120,15 @@ public void testInteger() { SchemaBuilder.struct().field(fieldName, Schema.INT32_SCHEMA).build(); Struct kafkaConnectStruct = new Struct(kafkaConnectSchema); kafkaConnectStruct.put(fieldName, fieldIntegerValue); - - assertTrue( - Arrays.equals( - calculateKey(List.of(), DELIMITER, fieldIntegerValue), - fieldIntegerValue.toString().getBytes(StandardCharsets.UTF_8))); - assertTrue( - Arrays.equals( - calculateKey(List.of(), DELIMITER, kafkaConnectStruct), - fieldIntegerValue.toString().getBytes(StandardCharsets.UTF_8))); - assertTrue( - Arrays.equals( - calculateKey(List.of(fieldName), DELIMITER, kafkaConnectStruct), - fieldIntegerValue.toString().getBytes(StandardCharsets.UTF_8))); + assertArrayEquals( + calculateKey(List.of(), DELIMITER, fieldIntegerValue), + fieldIntegerValue.toString().getBytes(StandardCharsets.UTF_8)); + assertArrayEquals( + calculateKey(List.of(), DELIMITER, kafkaConnectStruct), + fieldIntegerValue.toString().getBytes(StandardCharsets.UTF_8)); + assertArrayEquals( + calculateKey(List.of(fieldName), DELIMITER, kafkaConnectStruct), + fieldIntegerValue.toString().getBytes(StandardCharsets.UTF_8)); } @Test @@ -153,19 +139,15 @@ public void testLong() { SchemaBuilder.struct().field(fieldName, Schema.INT64_SCHEMA).build(); Struct kafkaConnectStruct = new Struct(kafkaConnectSchema); kafkaConnectStruct.put(fieldName, fieldLongValue); - - assertTrue( - Arrays.equals( - calculateKey(List.of(), DELIMITER, fieldLongValue), - fieldLongValue.toString().getBytes(StandardCharsets.UTF_8))); - assertTrue( - Arrays.equals( - calculateKey(List.of(), DELIMITER, kafkaConnectStruct), - fieldLongValue.toString().getBytes(StandardCharsets.UTF_8))); - assertTrue( - Arrays.equals( - calculateKey(List.of(fieldName), DELIMITER, kafkaConnectStruct), - fieldLongValue.toString().getBytes(StandardCharsets.UTF_8))); + assertArrayEquals( + calculateKey(List.of(), DELIMITER, fieldLongValue), + fieldLongValue.toString().getBytes(StandardCharsets.UTF_8)); + assertArrayEquals( + calculateKey(List.of(), DELIMITER, kafkaConnectStruct), + fieldLongValue.toString().getBytes(StandardCharsets.UTF_8)); + assertArrayEquals( + calculateKey(List.of(fieldName), DELIMITER, kafkaConnectStruct), + fieldLongValue.toString().getBytes(StandardCharsets.UTF_8)); } @Test @@ -176,19 +158,15 @@ public void testFloat() { SchemaBuilder.struct().field(fieldName, Schema.FLOAT32_SCHEMA).build(); Struct kafkaConnectStruct = new Struct(kafkaConnectSchema); kafkaConnectStruct.put(fieldName, fieldFloatValue); - - assertTrue( - Arrays.equals( - calculateKey(List.of(), DELIMITER, fieldFloatValue), - fieldFloatValue.toString().getBytes(StandardCharsets.UTF_8))); - assertTrue( - Arrays.equals( - calculateKey(List.of(), DELIMITER, kafkaConnectStruct), - fieldFloatValue.toString().getBytes(StandardCharsets.UTF_8))); - assertTrue( - Arrays.equals( - calculateKey(List.of(fieldName), DELIMITER, kafkaConnectStruct), - fieldFloatValue.toString().getBytes(StandardCharsets.UTF_8))); + assertArrayEquals( + calculateKey(List.of(), DELIMITER, fieldFloatValue), + fieldFloatValue.toString().getBytes(StandardCharsets.UTF_8)); + assertArrayEquals( + calculateKey(List.of(), DELIMITER, kafkaConnectStruct), + fieldFloatValue.toString().getBytes(StandardCharsets.UTF_8)); + assertArrayEquals( + calculateKey(List.of(fieldName), DELIMITER, kafkaConnectStruct), + fieldFloatValue.toString().getBytes(StandardCharsets.UTF_8)); } @Test @@ -199,19 +177,15 @@ public void testDouble() { SchemaBuilder.struct().field(fieldName, Schema.FLOAT64_SCHEMA).build(); Struct kafkaConnectStruct = new Struct(kafkaConnectSchema); kafkaConnectStruct.put(fieldName, fieldDoubleValue); - - assertTrue( - Arrays.equals( - calculateKey(List.of(), DELIMITER, fieldDoubleValue), - fieldDoubleValue.toString().getBytes(StandardCharsets.UTF_8))); - assertTrue( - Arrays.equals( - calculateKey(List.of(), DELIMITER, kafkaConnectStruct), - fieldDoubleValue.toString().getBytes(StandardCharsets.UTF_8))); - assertTrue( - Arrays.equals( - calculateKey(List.of(fieldName), DELIMITER, kafkaConnectStruct), - fieldDoubleValue.toString().getBytes(StandardCharsets.UTF_8))); + assertArrayEquals( + calculateKey(List.of(), DELIMITER, fieldDoubleValue), + fieldDoubleValue.toString().getBytes(StandardCharsets.UTF_8)); + assertArrayEquals( + calculateKey(List.of(), DELIMITER, kafkaConnectStruct), + fieldDoubleValue.toString().getBytes(StandardCharsets.UTF_8)); + assertArrayEquals( + calculateKey(List.of(fieldName), DELIMITER, kafkaConnectStruct), + fieldDoubleValue.toString().getBytes(StandardCharsets.UTF_8)); } @Test @@ -224,25 +198,21 @@ public void testDoubleSpecial() { Arrays.asList(Double.MAX_VALUE, Double.MIN_VALUE, Double.MIN_VALUE); assertEquals(testValues.size(), expectedValues.size()); - for (int test = 0; test < testValues.size(); ++test) { + for (Double testValue : testValues) { Schema kafkaConnectSchema = SchemaBuilder.struct().field(fieldName, Schema.FLOAT64_SCHEMA).build(); - Double testValue = testValues.get(test); Struct kafkaConnectStruct = new Struct(kafkaConnectSchema); kafkaConnectStruct.put(fieldName, testValue); - assertTrue( - Arrays.equals( - calculateKey(List.of(), DELIMITER, testValue), - testValue.toString().getBytes(StandardCharsets.UTF_8))); - assertTrue( - Arrays.equals( - calculateKey(List.of(), DELIMITER, kafkaConnectStruct), - testValue.toString().getBytes(StandardCharsets.UTF_8))); - assertTrue( - Arrays.equals( - calculateKey(List.of(fieldName), DELIMITER, kafkaConnectStruct), - testValue.toString().getBytes(StandardCharsets.UTF_8))); + assertArrayEquals( + calculateKey(List.of(), DELIMITER, testValue), + testValue.toString().getBytes(StandardCharsets.UTF_8)); + assertArrayEquals( + calculateKey(List.of(), DELIMITER, kafkaConnectStruct), + testValue.toString().getBytes(StandardCharsets.UTF_8)); + assertArrayEquals( + calculateKey(List.of(fieldName), DELIMITER, kafkaConnectStruct), + testValue.toString().getBytes(StandardCharsets.UTF_8)); } } @@ -255,18 +225,15 @@ public void testString() { Struct kafkaConnectStruct = new Struct(kafkaConnectSchema); kafkaConnectStruct.put(fieldName, fieldValue); - assertTrue( - Arrays.equals( - calculateKey(List.of(), DELIMITER, fieldValue), - fieldValue.toString().getBytes(StandardCharsets.UTF_8))); - assertTrue( - Arrays.equals( - calculateKey(List.of(), DELIMITER, kafkaConnectStruct), - fieldValue.toString().getBytes(StandardCharsets.UTF_8))); - assertTrue( - Arrays.equals( - calculateKey(List.of(fieldName), DELIMITER, kafkaConnectStruct), - fieldValue.toString().getBytes(StandardCharsets.UTF_8))); + assertArrayEquals( + calculateKey(List.of(), DELIMITER, fieldValue), + fieldValue.getBytes(StandardCharsets.UTF_8)); + assertArrayEquals( + calculateKey(List.of(), DELIMITER, kafkaConnectStruct), + fieldValue.getBytes(StandardCharsets.UTF_8)); + assertArrayEquals( + calculateKey(List.of(fieldName), DELIMITER, kafkaConnectStruct), + fieldValue.getBytes(StandardCharsets.UTF_8)); } @Test @@ -286,18 +253,15 @@ public void testFlatStruct() { kafkaConnectInnerStruct.put(fieldStringName, stringValue); kafkaConnectInnerStruct.put(fieldIntegerName, integerValue); - assertTrue( - Arrays.equals( - calculateKey(List.of(), DELIMITER, kafkaConnectInnerStruct), - (stringValue + DELIMITER + integerValue).getBytes(StandardCharsets.UTF_8))); - assertTrue( - Arrays.equals( - calculateKey(List.of(fieldStringName), DELIMITER, kafkaConnectInnerStruct), - stringValue.toString().getBytes(StandardCharsets.UTF_8))); - assertTrue( - Arrays.equals( - calculateKey(List.of(fieldIntegerName), DELIMITER, kafkaConnectInnerStruct), - integerValue.toString().getBytes(StandardCharsets.UTF_8))); + assertArrayEquals( + calculateKey(List.of(), DELIMITER, kafkaConnectInnerStruct), + (stringValue + DELIMITER + integerValue).getBytes(StandardCharsets.UTF_8)); + assertArrayEquals( + calculateKey(List.of(fieldStringName), DELIMITER, kafkaConnectInnerStruct), + stringValue.toString().getBytes(StandardCharsets.UTF_8)); + assertArrayEquals( + calculateKey(List.of(fieldIntegerName), DELIMITER, kafkaConnectInnerStruct), + integerValue.toString().getBytes(StandardCharsets.UTF_8)); } @Test @@ -329,33 +293,28 @@ public void testStructNestedOnce() { Struct kafkaConnectStruct = new Struct(kafkaConnectSchema); kafkaConnectStruct.put(innerFieldStructName, kafkaConnectInnerStruct); kafkaConnectStruct.put(fieldArrayName, arrayValue); - assertTrue( - Arrays.equals( - calculateKey(List.of(), DELIMITER, kafkaConnectStruct), - (kafkaConnectInnerStruct.toString() + DELIMITER + arrayValue.toString()) - .getBytes(StandardCharsets.UTF_8))); - assertTrue( - Arrays.equals( - calculateKey(List.of(innerFieldStructName), DELIMITER, kafkaConnectStruct), - kafkaConnectInnerStruct.toString().getBytes(StandardCharsets.UTF_8))); - assertTrue( - Arrays.equals( - calculateKey(List.of(fieldArrayName), DELIMITER, kafkaConnectStruct), - arrayValue.toString().getBytes(StandardCharsets.UTF_8))); - assertTrue( - Arrays.equals( - calculateKey( - List.of(innerFieldStructName + "." + innerFieldStringName), - DELIMITER, - kafkaConnectStruct), - innerStringValue.toString().getBytes(StandardCharsets.UTF_8))); - assertTrue( - Arrays.equals( - calculateKey( - List.of(innerFieldStructName + "." + innerFieldIntegerName), - DELIMITER, - kafkaConnectStruct), - innerIntegerValue.toString().getBytes(StandardCharsets.UTF_8))); + assertArrayEquals( + calculateKey(List.of(), DELIMITER, kafkaConnectStruct), + (kafkaConnectInnerStruct.toString() + DELIMITER + arrayValue.toString()) + .getBytes(StandardCharsets.UTF_8)); + assertArrayEquals( + calculateKey(List.of(innerFieldStructName), DELIMITER, kafkaConnectStruct), + kafkaConnectInnerStruct.toString().getBytes(StandardCharsets.UTF_8)); + assertArrayEquals( + calculateKey(List.of(fieldArrayName), DELIMITER, kafkaConnectStruct), + arrayValue.toString().getBytes(StandardCharsets.UTF_8)); + assertArrayEquals( + calculateKey( + List.of(innerFieldStructName + "." + innerFieldStringName), + DELIMITER, + kafkaConnectStruct), + innerStringValue.toString().getBytes(StandardCharsets.UTF_8)); + assertArrayEquals( + calculateKey( + List.of(innerFieldStructName + "." + innerFieldIntegerName), + DELIMITER, + kafkaConnectStruct), + innerIntegerValue.toString().getBytes(StandardCharsets.UTF_8)); } @Test @@ -399,38 +358,29 @@ public void testStructNestedTwice() { kafkaConnectOuterStruct.put(innerFieldStructName, kafkaConnectInnerStruct); kafkaConnectOuterStruct.put(middleFieldStructName, kafkaConnectMiddleStruct); - assertTrue( - Arrays.equals( - calculateKey(List.of(), DELIMITER, kafkaConnectOuterStruct), - (kafkaConnectInnerStruct.toString() + DELIMITER + kafkaConnectMiddleStruct.toString()) - .getBytes(StandardCharsets.UTF_8))); - assertTrue( - Arrays.equals( - calculateKey(List.of(innerFieldStructName), DELIMITER, kafkaConnectOuterStruct), - kafkaConnectInnerStruct.toString().getBytes(StandardCharsets.UTF_8))); - assertTrue( - Arrays.equals( - calculateKey(List.of(middleFieldStructName), DELIMITER, kafkaConnectOuterStruct), - kafkaConnectMiddleStruct.toString().getBytes(StandardCharsets.UTF_8))); - assertTrue( - Arrays.equals( - calculateKey( - List.of(innerFieldStructName + "." + innerFieldStringName), - DELIMITER, - kafkaConnectOuterStruct), - innerStringValue.toString().getBytes(StandardCharsets.UTF_8))); - assertTrue( - Arrays.equals( - calculateKey( - List.of( - middleFieldStructName - + "." - + innerFieldStructName - + "." - + innerFieldIntegerName), - DELIMITER, - kafkaConnectOuterStruct), - innerIntegerValue.toString().getBytes(StandardCharsets.UTF_8))); + assertArrayEquals( + calculateKey(List.of(), DELIMITER, kafkaConnectOuterStruct), + (kafkaConnectInnerStruct.toString() + DELIMITER + kafkaConnectMiddleStruct.toString()) + .getBytes(StandardCharsets.UTF_8)); + assertArrayEquals( + calculateKey(List.of(innerFieldStructName), DELIMITER, kafkaConnectOuterStruct), + kafkaConnectInnerStruct.toString().getBytes(StandardCharsets.UTF_8)); + assertArrayEquals( + calculateKey(List.of(middleFieldStructName), DELIMITER, kafkaConnectOuterStruct), + kafkaConnectMiddleStruct.toString().getBytes(StandardCharsets.UTF_8)); + assertArrayEquals( + calculateKey( + List.of(innerFieldStructName + "." + innerFieldStringName), + DELIMITER, + kafkaConnectOuterStruct), + innerStringValue.toString().getBytes(StandardCharsets.UTF_8)); + assertArrayEquals( + calculateKey( + List.of( + middleFieldStructName + "." + innerFieldStructName + "." + innerFieldIntegerName), + DELIMITER, + kafkaConnectOuterStruct), + innerIntegerValue.toString().getBytes(StandardCharsets.UTF_8)); } @Test @@ -452,19 +402,15 @@ public void testEmptyInnerStruct() { Struct kafkaConnectOuterStruct = new Struct(kafkaConnectOuterSchema); kafkaConnectOuterStruct.put(innerFieldStructName, kafkaConnectInnerStruct); kafkaConnectOuterStruct.put(innerFieldStringName, innerStringValue); - assertTrue( - Arrays.equals( - calculateKey(List.of(), DELIMITER, kafkaConnectOuterStruct), - (kafkaConnectInnerStruct + DELIMITER + innerStringValue) - .getBytes(StandardCharsets.UTF_8))); - assertTrue( - Arrays.equals( - calculateKey(List.of(innerFieldStructName), DELIMITER, kafkaConnectOuterStruct), - kafkaConnectInnerStruct.toString().getBytes(StandardCharsets.UTF_8))); - assertTrue( - Arrays.equals( - calculateKey(List.of(innerFieldStructName), DELIMITER, kafkaConnectOuterStruct), - "Struct{}".getBytes(StandardCharsets.UTF_8))); + assertArrayEquals( + calculateKey(List.of(), DELIMITER, kafkaConnectOuterStruct), + (kafkaConnectInnerStruct + DELIMITER + innerStringValue).getBytes(StandardCharsets.UTF_8)); + assertArrayEquals( + calculateKey(List.of(innerFieldStructName), DELIMITER, kafkaConnectOuterStruct), + kafkaConnectInnerStruct.toString().getBytes(StandardCharsets.UTF_8)); + assertArrayEquals( + calculateKey(List.of(innerFieldStructName), DELIMITER, kafkaConnectOuterStruct), + "Struct{}".getBytes(StandardCharsets.UTF_8)); } @Test @@ -488,7 +434,6 @@ public void testUnmappableValues() { byte[] expected = new byte[0]; assertArrayEquals(expected, calculateKey(List.of(), DELIMITER, "")); assertArrayEquals(expected, calculateKey(List.of(), DELIMITER, new byte[0])); - assertArrayEquals(expected, calculateKey(List.of(), DELIMITER, new HashMap<>())); assertArrayEquals(expected, calculateKey(List.of(), DELIMITER, structNoFields)); } @@ -509,20 +454,17 @@ public void testDifferentStructMappings() { struct.put(fieldIntegerName, integerValue); // Note that it preserves field order from the Schema. - assertTrue( - Arrays.equals( - calculateKey(List.of(), DELIMITER, struct), - (stringValue + DELIMITER + integerValue).getBytes(StandardCharsets.UTF_8))); + assertArrayEquals( + calculateKey(List.of(), DELIMITER, struct), + (stringValue + DELIMITER + integerValue).getBytes(StandardCharsets.UTF_8)); // Force another order. - assertTrue( - Arrays.equals( - calculateKey(List.of(fieldIntegerName, fieldStringName), DELIMITER, struct), - (integerValue + DELIMITER + stringValue).getBytes(StandardCharsets.UTF_8))); + assertArrayEquals( + calculateKey(List.of(fieldIntegerName, fieldStringName), DELIMITER, struct), + (integerValue + DELIMITER + stringValue).getBytes(StandardCharsets.UTF_8)); // Use the same field twice. - assertTrue( - Arrays.equals( - calculateKey(List.of(fieldIntegerName, fieldIntegerName), DELIMITER, struct), - (integerValue + DELIMITER + integerValue).getBytes(StandardCharsets.UTF_8))); + assertArrayEquals( + calculateKey(List.of(fieldIntegerName, fieldIntegerName), DELIMITER, struct), + (integerValue + DELIMITER + integerValue).getBytes(StandardCharsets.UTF_8)); // Try accessing nonexistent key. assertThrows(DataException.class, () -> calculateKey(List.of("invalid"), DELIMITER, struct)); } @@ -566,27 +508,23 @@ public void testMap() { Struct kafkaConnectStruct = new Struct(kafkaConnectSchema); kafkaConnectStruct.put(fieldNameIntegerMap, integerMap); kafkaConnectStruct.put(fieldNameStringMap, stringMap); - assertTrue( - Arrays.equals( - calculateKey(List.of(), DELIMITER, kafkaConnectStruct), - (integerMap.toString() + DELIMITER + stringMap.toString()) - .getBytes(StandardCharsets.UTF_8))); - assertTrue( - Arrays.equals( - calculateKey(List.of(fieldNameIntegerMap), DELIMITER, kafkaConnectStruct), - integerMap.toString().getBytes(StandardCharsets.UTF_8))); - assertTrue( - Arrays.equals( - calculateKey(List.of(fieldNameStringMap), DELIMITER, kafkaConnectStruct), - stringMap.toString().getBytes(StandardCharsets.UTF_8))); - // The key is Integer, not String - we don't support it + assertArrayEquals( + calculateKey(List.of(), DELIMITER, kafkaConnectStruct), + (integerMap.toString() + DELIMITER + stringMap.toString()) + .getBytes(StandardCharsets.UTF_8)); + assertArrayEquals( + calculateKey(List.of(fieldNameIntegerMap), DELIMITER, kafkaConnectStruct), + integerMap.toString().getBytes(StandardCharsets.UTF_8)); + assertArrayEquals( + calculateKey(List.of(fieldNameStringMap), DELIMITER, kafkaConnectStruct), + stringMap.toString().getBytes(StandardCharsets.UTF_8)); + // Accessing map keys is not supported. assertThrows( DataException.class, () -> calculateKey(List.of(fieldNameIntegerMap + ".3"), DELIMITER, kafkaConnectStruct)); - assertTrue( - Arrays.equals( - calculateKey(List.of(fieldNameStringMap + ".3"), DELIMITER, kafkaConnectStruct), - "true".getBytes(StandardCharsets.UTF_8))); + assertThrows( + DataException.class, + () -> calculateKey(List.of(fieldNameStringMap + ".3"), DELIMITER, kafkaConnectStruct)); } @Test @@ -601,14 +539,12 @@ public void testIntegerArray() { Struct kafkaConnectStruct = new Struct(kafkaConnectSchema); kafkaConnectStruct.put(fieldName, fieldValue); - assertTrue( - Arrays.equals( - calculateKey(List.of(), DELIMITER, kafkaConnectStruct), - fieldValue.toString().getBytes(StandardCharsets.UTF_8))); - assertTrue( - Arrays.equals( - calculateKey(List.of(fieldName), DELIMITER, kafkaConnectStruct), - fieldValue.toString().getBytes(StandardCharsets.UTF_8))); + assertArrayEquals( + calculateKey(List.of(), DELIMITER, kafkaConnectStruct), + fieldValue.toString().getBytes(StandardCharsets.UTF_8)); + assertArrayEquals( + calculateKey(List.of(fieldName), DELIMITER, kafkaConnectStruct), + fieldValue.toString().getBytes(StandardCharsets.UTF_8)); } @Test @@ -636,14 +572,12 @@ public void testStructArray() { List innerStructList = List.of(kafkaConnectInnerStruct); kafkaConnectStruct.put(middleFieldArrayName, innerStructList); - assertTrue( - Arrays.equals( - calculateKey(List.of(), DELIMITER, kafkaConnectStruct), - innerStructList.toString().getBytes(StandardCharsets.UTF_8))); - assertTrue( - Arrays.equals( - calculateKey(List.of(middleFieldArrayName), DELIMITER, kafkaConnectStruct), - innerStructList.toString().getBytes(StandardCharsets.UTF_8))); + assertArrayEquals( + calculateKey(List.of(), DELIMITER, kafkaConnectStruct), + innerStructList.toString().getBytes(StandardCharsets.UTF_8)); + assertArrayEquals( + calculateKey(List.of(middleFieldArrayName), DELIMITER, kafkaConnectStruct), + innerStructList.toString().getBytes(StandardCharsets.UTF_8)); } @Test @@ -660,14 +594,12 @@ public void testStringArray() { Struct kafkaConnectStruct = new Struct(kafkaConnectSchema); kafkaConnectStruct.put(fieldName, fieldValue); - assertTrue( - Arrays.equals( - calculateKey(List.of(), DELIMITER, kafkaConnectStruct), - fieldValue.toString().getBytes(StandardCharsets.UTF_8))); - assertTrue( - Arrays.equals( - calculateKey(List.of(fieldName), DELIMITER, kafkaConnectStruct), - fieldValue.toString().getBytes(StandardCharsets.UTF_8))); + assertArrayEquals( + calculateKey(List.of(), DELIMITER, kafkaConnectStruct), + fieldValue.toString().getBytes(StandardCharsets.UTF_8)); + assertArrayEquals( + calculateKey(List.of(fieldName), DELIMITER, kafkaConnectStruct), + fieldValue.toString().getBytes(StandardCharsets.UTF_8)); } @Test @@ -680,9 +612,8 @@ public void testBytes() { Struct kafkaConnectStruct = new Struct(kafkaConnectSchema); kafkaConnectStruct.put(fieldName, fieldValueKafkaConnect); - assertTrue(Arrays.equals(calculateKey(List.of(), DELIMITER, kafkaConnectStruct), fieldBytes)); - assertTrue( - Arrays.equals(calculateKey(List.of(fieldName), DELIMITER, kafkaConnectStruct), fieldBytes)); + assertArrayEquals(calculateKey(List.of(), DELIMITER, kafkaConnectStruct), fieldBytes); + assertArrayEquals(calculateKey(List.of(fieldName), DELIMITER, kafkaConnectStruct), fieldBytes); } @Test @@ -712,21 +643,17 @@ public void testBytesInStruct() throws IOException { expectedBuilder.write("}".getBytes(StandardCharsets.UTF_8)); byte[] expectedStructSerialization = expectedBuilder.toByteArray(); - assertTrue( - Arrays.equals( - calculateKey(List.of(), DELIMITER, kafkaConnectOuterStruct), - expectedStructSerialization)); - assertTrue( - Arrays.equals( - calculateKey(List.of(innerFieldStructName), DELIMITER, kafkaConnectOuterStruct), - expectedStructSerialization)); - assertTrue( - Arrays.equals( - calculateKey( - List.of(innerFieldStructName + "." + innerFieldBytesName), - DELIMITER, - kafkaConnectOuterStruct), - innerBytesValue)); + assertArrayEquals( + calculateKey(List.of(), DELIMITER, kafkaConnectOuterStruct), expectedStructSerialization); + assertArrayEquals( + calculateKey(List.of(innerFieldStructName), DELIMITER, kafkaConnectOuterStruct), + expectedStructSerialization); + assertArrayEquals( + calculateKey( + List.of(innerFieldStructName + "." + innerFieldBytesName), + DELIMITER, + kafkaConnectOuterStruct), + innerBytesValue); } @Test @@ -735,18 +662,24 @@ public void testKafkaLogicalTypes() { final String timestampFieldName = "KafkaTimestamp"; final String timeFieldName = "KafkaTime"; final String decimalFieldName = "KafkaDecimal"; - final Long dateLong = 1488406838808L; - final Date date = new Date(dateLong); - final String decimalString = "0.30000000000000004"; - final Integer decimalScale = 0; + final Date date = new Date(1488406838808L); + final String decimalString = "-1.23E-12"; final BigDecimal decimal = new BigDecimal(decimalString); + final String formattedDate = + DateTimeFormatter.ISO_LOCAL_DATE.format(date.toInstant().atZone(ZoneOffset.UTC)); + final String formattedTimestamp = + DateTimeFormatter.ISO_INSTANT.format(date.toInstant().atZone(ZoneOffset.UTC)); + final String formattedTime = + DateTimeFormatter.ISO_LOCAL_TIME.format(date.toInstant().atZone(ZoneOffset.UTC)); + final String formattedDecimal = decimal.toPlainString(); + Schema kafkaConnectSchema = SchemaBuilder.struct() .field(dateFieldName, org.apache.kafka.connect.data.Date.SCHEMA) .field(timestampFieldName, org.apache.kafka.connect.data.Timestamp.SCHEMA) - .field(timeFieldName, org.apache.kafka.connect.data.Timestamp.SCHEMA) - .field(decimalFieldName, org.apache.kafka.connect.data.Decimal.schema(decimalScale)) + .field(timeFieldName, org.apache.kafka.connect.data.Time.SCHEMA) + .field(decimalFieldName, org.apache.kafka.connect.data.Decimal.schema(decimal.scale())) .build(); Struct kafkaConnectStruct = new Struct(kafkaConnectSchema); @@ -754,23 +687,43 @@ public void testKafkaLogicalTypes() { kafkaConnectStruct.put(timestampFieldName, date); kafkaConnectStruct.put(timeFieldName, date); kafkaConnectStruct.put(decimalFieldName, decimal); - // TODO: test in practice whether the Confluent sink works exactly like this. - assertTrue( - Arrays.equals( - calculateKey(List.of(dateFieldName), DELIMITER, kafkaConnectStruct), - date.toString().getBytes(StandardCharsets.UTF_8))); - assertTrue( - Arrays.equals( - calculateKey(List.of(timestampFieldName), DELIMITER, kafkaConnectStruct), - date.toString().getBytes(StandardCharsets.UTF_8))); - assertTrue( - Arrays.equals( - calculateKey(List.of(timeFieldName), DELIMITER, kafkaConnectStruct), - date.toString().getBytes(StandardCharsets.UTF_8))); - assertTrue( - Arrays.equals( - calculateKey(List.of(decimalFieldName), DELIMITER, kafkaConnectStruct), - decimalString.getBytes(StandardCharsets.UTF_8))); + assertArrayEquals( + calculateKey(List.of(dateFieldName), DELIMITER, kafkaConnectStruct, kafkaConnectSchema), + formattedDate.getBytes(StandardCharsets.UTF_8)); + assertArrayEquals( + calculateKey( + List.of(timestampFieldName), DELIMITER, kafkaConnectStruct, kafkaConnectSchema), + formattedTimestamp.getBytes(StandardCharsets.UTF_8)); + assertArrayEquals( + calculateKey(List.of(timeFieldName), DELIMITER, kafkaConnectStruct, kafkaConnectSchema), + formattedTime.getBytes(StandardCharsets.UTF_8)); + assertArrayEquals( + calculateKey(List.of(decimalFieldName), DELIMITER, kafkaConnectStruct, kafkaConnectSchema), + formattedDecimal.getBytes(StandardCharsets.UTF_8)); + } + + @Test + public void testKafkaLogicalTypeInStructField() { + Date epochDate = new Date(0L); + + String dateFieldName = "date"; + String structFieldName = "struct"; + Schema innerStructSchema = + SchemaBuilder.struct() + .field(dateFieldName, org.apache.kafka.connect.data.Date.SCHEMA) + .build(); + Schema outerStructSchema = + SchemaBuilder.struct().field(structFieldName, innerStructSchema).build(); + + Struct innerStruct = new Struct(innerStructSchema); + innerStruct.put(dateFieldName, epochDate); + + Struct outerStruct = new Struct(outerStructSchema); + outerStruct.put(structFieldName, innerStruct); + + byte[] keyBytes = + calculateKey(List.of(structFieldName), DELIMITER, outerStruct, outerStructSchema); + assertEquals("Struct{date=1970-01-01}", new String(keyBytes, StandardCharsets.UTF_8)); } @Test @@ -795,10 +748,9 @@ public void testNullable() { assertThrows( DataException.class, () -> calculateKey(List.of(nullableFieldName), DELIMITER, kafkaConnectStruct)); - assertTrue( - Arrays.equals( - calculateKey(List.of(requiredFieldName), DELIMITER, kafkaConnectStruct), - requiredFieldValue.toString().getBytes(StandardCharsets.UTF_8))); + assertArrayEquals( + calculateKey(List.of(requiredFieldName), DELIMITER, kafkaConnectStruct), + requiredFieldValue.toString().getBytes(StandardCharsets.UTF_8)); } @Test @@ -844,58 +796,58 @@ public void testSchemalessRecordSuccesses() { for (Object[] testCase : List.of( // Default key definition and all kinds of types. - // I know of no way to pass unserialized bytes or logical types here. I think it's only - // possible using some kind of schema. + // Note that logical types cannot be used without schema. new Object[] {List.of(), "2.130", "2.13"}, new Object[] {List.of(), "7", "7"}, new Object[] {List.of(), "\"x\"", "x"}, new Object[] {List.of(), "true", "true"}, new Object[] {List.of(), "[]", "[]"}, new Object[] {List.of(), "[1,\"s\",true]", "[1, s, true]"}, - // Default key definition when using on a map (schemaless data is converted into Map not - // Struct!). - new Object[] {List.of(), "{\"a\":1,\"b\":true,\"c\":\"str\"}", "1##true##str"}, - new Object[] { - List.of(), "{\"b\":1,\"a\":3}", "3##1" - }, // Note it doesn't keep key ordering. + // Default key definition when using on a map (schemaless data is converted into a Map + // rather than a Struct, so its fields are not accessed when serializing!). + new Object[] {List.of(), "{\"a\":1,\"b\":true,\"c\":\"str\"}", "{a=1, b=true, c=str}"}, + new Object[] {List.of(), "{\"b\":1,\"a\":3}", "{a=3, b=1}"}, new Object[] { List.of(), "{\"b\":[1,2],\"a\":3,\"c\":{\"x\":\"D\",\"y\":2137}}", - "3##[1, 2]##{x=D, y=2137}" - }, - // Key extraction and serialization of nested beings. - new Object[] {List.of("f"), "{\"f\":{}}", "{}"}, - new Object[] {List.of("f"), "{\"f\":1}", "1"}, - new Object[] {List.of("f"), "{\"f\":true}", "true"}, - new Object[] {List.of("f"), "{\"f\":\"s\"}", "s"}, - new Object[] {List.of("f"), "{\"f\":[]}", "[]"}, - new Object[] {List.of("f"), "{\"f\":[1,\"a\"]}", "[1, a]"}, - new Object[] {List.of("f"), "{\"f\":{\"b\":1,\"a\":3}}", "{a=3, b=1}"}, - new Object[] {List.of("f"), "{\"f\":{\"a\":{\"b\": true}}}", "{a={b=true}}"}, - new Object[] { - List.of("f"), "{\"f\":{\"a\":{\"b\": true,\"c\":2}}}", "{a={b=true, c=2}}" - }, - new Object[] {List.of("f.a"), "{\"f\":{\"b\":1,\"a\":3}}", "3"})) { + "{a=3, b=[1, 2], c={x=D, y=2137}}" + } + // Note that there are no tests for accessing fields of the values. Maps, to which + // schemaless JSON data deserializes when using JsonConverter, does not support + // accessing its values by keys. + )) { KeyMapper mapper = new KeyMapper(delimiter, (List) testCase[0]); SchemaAndValue connectData = jsonConverter.toConnectData( topic, ((String) testCase[1]).getBytes(StandardCharsets.UTF_8)); byte[] expectedResult = ((String) testCase[2]).getBytes(StandardCharsets.UTF_8); - byte[] result = mapper.getKey(connectData.value()); - assertTrue(Arrays.equals(expectedResult, result)); + byte[] result = mapper.getKey(new SchemaAndValue(null, connectData.value())); + List expectedAndReal = + List.of((String) testCase[2], new String(result, StandardCharsets.UTF_8)); + assertArrayEquals(expectedResult, result); } - ; } @Test public void testAccessingSchemalessPrimitiveField() { KeyMapper mapper = new KeyMapper("#", List.of("fieldName")); - assertThrows(DataException.class, () -> mapper.getKey("primitiveString")); + assertThrows( + DataException.class, () -> mapper.getKey(new SchemaAndValue(null, "primitiveString"))); } private static byte[] calculateKey( List mapperDefinition, String mapperDelimiter, Object kafkaKey) { + // We use `null` in this test since our code for now uses Schema only to warn the user when an + // unsupported logical type is encountered. + return calculateKey(mapperDefinition, mapperDelimiter, kafkaKey, null); + } + + private static byte[] calculateKey( + List mapperDefinition, + String mapperDelimiter, + Object kafkaKey, + Schema kafkaKeySchema) { KeyMapper mapper = new KeyMapper(mapperDelimiter, mapperDefinition); - return mapper.getKey(kafkaKey); + return mapper.getKey(new SchemaAndValue(kafkaKeySchema, kafkaKey)); } } diff --git a/google-cloud-bigtable-kafka-connect-sink/src/test/java/com/google/cloud/kafka/connect/bigtable/mapping/LogicalTypeUtilsTest.java b/google-cloud-bigtable-kafka-connect-sink/src/test/java/com/google/cloud/kafka/connect/bigtable/mapping/LogicalTypeUtilsTest.java new file mode 100644 index 0000000000..37a82ba645 --- /dev/null +++ b/google-cloud-bigtable-kafka-connect-sink/src/test/java/com/google/cloud/kafka/connect/bigtable/mapping/LogicalTypeUtilsTest.java @@ -0,0 +1,51 @@ +/* + * Copyright 2025 Google LLC + * + * Licensed under the Apache License, Version 2.0 (the "License"); + * you may not use this file except in compliance with the License. + * You may obtain a copy of the License at + * + * https://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ +package com.google.cloud.kafka.connect.bigtable.mapping; + +import java.util.Optional; +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.junit.Test; +import org.junit.runner.RunWith; +import org.junit.runners.JUnit4; + +@RunWith(JUnit4.class) +public class LogicalTypeUtilsTest { + @Test + public void testSupportedType() { + LogicalTypeUtils.logIfLogicalTypeUnsupported(Optional.of(Date.SCHEMA)); + } + + @Test + public void testStruct() { + LogicalTypeUtils.logIfLogicalTypeUnsupported(Optional.of(SchemaBuilder.struct().name("name"))); + } + + @Test + public void testUnsupportedType() { + Optional schema = + Optional.of(new ConnectSchema(Schema.Type.INT64, false, null, "name", 1, "doc")); + LogicalTypeUtils.logIfLogicalTypeUnsupported(schema); + LogicalTypeUtils.logIfLogicalTypeUnsupported(schema); + } + + @Test + public void testNoSchema() { + LogicalTypeUtils.logIfLogicalTypeUnsupported(Optional.empty()); + } +} diff --git a/google-cloud-bigtable-kafka-connect-sink/src/test/java/com/google/cloud/kafka/connect/bigtable/mapping/ValueMapperTest.java b/google-cloud-bigtable-kafka-connect-sink/src/test/java/com/google/cloud/kafka/connect/bigtable/mapping/ValueMapperTest.java index 49c7d4f876..accd38dedb 100644 --- a/google-cloud-bigtable-kafka-connect-sink/src/test/java/com/google/cloud/kafka/connect/bigtable/mapping/ValueMapperTest.java +++ b/google-cloud-bigtable-kafka-connect-sink/src/test/java/com/google/cloud/kafka/connect/bigtable/mapping/ValueMapperTest.java @@ -16,6 +16,9 @@ package com.google.cloud.kafka.connect.bigtable.mapping; import static com.google.cloud.kafka.connect.bigtable.util.MockUtil.assertTotalNumberOfInvocations; +import static com.google.cloud.kafka.connect.bigtable.util.NestedNullStructFactory.NESTED_NULL_STRUCT_FIELD_NAME; +import static com.google.cloud.kafka.connect.bigtable.util.NestedNullStructFactory.NESTED_NULL_STRUCT_FIELD_NAME_BYTES; +import static com.google.cloud.kafka.connect.bigtable.util.NestedNullStructFactory.getStructhWithNullOnNthNestingLevel; import static org.junit.Assert.assertTrue; import static org.mockito.Mockito.spy; import static org.mockito.Mockito.times; @@ -24,22 +27,20 @@ import com.google.cloud.bigtable.data.v2.models.Range; import com.google.cloud.kafka.connect.bigtable.config.ConfigInterpolation; import com.google.cloud.kafka.connect.bigtable.config.NullValueMode; -import com.google.cloud.kafka.connect.bigtable.util.JsonConverterFactory; import com.google.protobuf.ByteString; import java.math.BigDecimal; import java.nio.charset.StandardCharsets; -import java.util.AbstractMap; +import java.time.Instant; +import java.time.temporal.ChronoUnit; import java.util.Date; import java.util.HashMap; import java.util.List; import java.util.Map; -import java.util.TreeMap; import org.apache.hadoop.hbase.util.Bytes; import org.apache.kafka.connect.data.Schema; +import org.apache.kafka.connect.data.SchemaAndValue; import org.apache.kafka.connect.data.SchemaBuilder; import org.apache.kafka.connect.data.Struct; -import org.apache.kafka.connect.json.JsonConverter; -import org.junit.Ignore; import org.junit.Test; import org.junit.runner.RunWith; import org.junit.runners.JUnit4; @@ -58,19 +59,13 @@ public class ValueMapperTest { Range.TimestampRange.create(0, TIMESTAMP); private static final String DEFAULT_TOPIC = "topic"; - private static final String NESTED_NULL_STRUCT_FIELD_NAME = "struct"; - private static final ByteString NESTED_NULL_STRUCT_FIELD_NAME_BYTES = - ByteString.copyFrom(NESTED_NULL_STRUCT_FIELD_NAME.getBytes(StandardCharsets.UTF_8)); - - private static final JsonConverter jsonConverter = JsonConverterFactory.create(false, false); - @Test public void testBoolean() { Boolean value = true; ByteString expected = ByteString.copyFrom(Bytes.toBytes(value)); ValueMapper mapper = new TestValueMapper(DEFAULT_COLUMN_FAMILY, DEFAULT_COLUMN, NullValueMode.IGNORE); - MutationDataBuilder mutationDataBuilder = mapper.getRecordMutationDataBuilder(value, DEFAULT_TOPIC, TIMESTAMP); + MutationDataBuilder mutationDataBuilder = getRecordMutationDataBuilder(mapper, value); verify(mutationDataBuilder, times(1)) .setCell(DEFAULT_COLUMN_FAMILY, DEFAULT_COLUMN_BYTES, TIMESTAMP, expected); assertTotalNumberOfInvocations(mutationDataBuilder, 1); @@ -82,7 +77,7 @@ public void testString() { ByteString expected = ByteString.copyFrom(Bytes.toBytes(value)); ValueMapper mapper = new TestValueMapper(DEFAULT_COLUMN_FAMILY, DEFAULT_COLUMN, NullValueMode.IGNORE); - MutationDataBuilder mutationDataBuilder = mapper.getRecordMutationDataBuilder(value, DEFAULT_TOPIC, TIMESTAMP); + MutationDataBuilder mutationDataBuilder = getRecordMutationDataBuilder(mapper, value); verify(mutationDataBuilder, times(1)) .setCell(DEFAULT_COLUMN_FAMILY, DEFAULT_COLUMN_BYTES, TIMESTAMP, expected); assertTotalNumberOfInvocations(mutationDataBuilder, 1); @@ -94,7 +89,7 @@ public void testLong() { ByteString expected = ByteString.copyFrom(Bytes.toBytes(value)); ValueMapper mapper = new TestValueMapper(DEFAULT_COLUMN_FAMILY, DEFAULT_COLUMN, NullValueMode.IGNORE); - MutationDataBuilder mutationDataBuilder = mapper.getRecordMutationDataBuilder(value, DEFAULT_TOPIC, TIMESTAMP); + MutationDataBuilder mutationDataBuilder = getRecordMutationDataBuilder(mapper, value); verify(mutationDataBuilder, times(1)) .setCell(DEFAULT_COLUMN_FAMILY, DEFAULT_COLUMN_BYTES, TIMESTAMP, expected); assertTotalNumberOfInvocations(mutationDataBuilder, 1); @@ -106,7 +101,7 @@ public void testInteger() { ByteString expected = ByteString.copyFrom(Bytes.toBytes(value)); ValueMapper mapper = new TestValueMapper(DEFAULT_COLUMN_FAMILY, DEFAULT_COLUMN, NullValueMode.IGNORE); - MutationDataBuilder mutationDataBuilder = mapper.getRecordMutationDataBuilder(value, DEFAULT_TOPIC, TIMESTAMP); + MutationDataBuilder mutationDataBuilder = getRecordMutationDataBuilder(mapper, value); verify(mutationDataBuilder, times(1)) .setCell(DEFAULT_COLUMN_FAMILY, DEFAULT_COLUMN_BYTES, TIMESTAMP, expected); assertTotalNumberOfInvocations(mutationDataBuilder, 1); @@ -118,7 +113,7 @@ public void testShort() { ByteString expected = ByteString.copyFrom(Bytes.toBytes(value)); ValueMapper mapper = new TestValueMapper(DEFAULT_COLUMN_FAMILY, DEFAULT_COLUMN, NullValueMode.IGNORE); - MutationDataBuilder mutationDataBuilder = mapper.getRecordMutationDataBuilder(value, DEFAULT_TOPIC, TIMESTAMP); + MutationDataBuilder mutationDataBuilder = getRecordMutationDataBuilder(mapper, value); verify(mutationDataBuilder, times(1)) .setCell(DEFAULT_COLUMN_FAMILY, DEFAULT_COLUMN_BYTES, TIMESTAMP, expected); assertTotalNumberOfInvocations(mutationDataBuilder, 1); @@ -130,7 +125,7 @@ public void testByte() { ByteString expected = ByteString.copyFrom(Bytes.toBytes(value)); ValueMapper mapper = new TestValueMapper(DEFAULT_COLUMN_FAMILY, DEFAULT_COLUMN, NullValueMode.IGNORE); - MutationDataBuilder mutationDataBuilder = mapper.getRecordMutationDataBuilder(value, DEFAULT_TOPIC, TIMESTAMP); + MutationDataBuilder mutationDataBuilder = getRecordMutationDataBuilder(mapper, value); verify(mutationDataBuilder, times(1)) .setCell(DEFAULT_COLUMN_FAMILY, DEFAULT_COLUMN_BYTES, TIMESTAMP, expected); assertTotalNumberOfInvocations(mutationDataBuilder, 1); @@ -142,7 +137,7 @@ public void testBytes() { ByteString expected = ByteString.copyFrom(value); ValueMapper mapper = new TestValueMapper(DEFAULT_COLUMN_FAMILY, DEFAULT_COLUMN, NullValueMode.IGNORE); - MutationDataBuilder mutationDataBuilder = mapper.getRecordMutationDataBuilder(value, DEFAULT_TOPIC, TIMESTAMP); + MutationDataBuilder mutationDataBuilder = getRecordMutationDataBuilder(mapper, value); verify(mutationDataBuilder, times(1)) .setCell(DEFAULT_COLUMN_FAMILY, DEFAULT_COLUMN_BYTES, TIMESTAMP, expected); assertTotalNumberOfInvocations(mutationDataBuilder, 1); @@ -154,7 +149,7 @@ public void testFloat() { ByteString expected = ByteString.copyFrom(Bytes.toBytes(value)); ValueMapper mapper = new TestValueMapper(DEFAULT_COLUMN_FAMILY, DEFAULT_COLUMN, NullValueMode.IGNORE); - MutationDataBuilder mutationDataBuilder = mapper.getRecordMutationDataBuilder(value, DEFAULT_TOPIC, TIMESTAMP); + MutationDataBuilder mutationDataBuilder = getRecordMutationDataBuilder(mapper, value); verify(mutationDataBuilder, times(1)) .setCell(DEFAULT_COLUMN_FAMILY, DEFAULT_COLUMN_BYTES, TIMESTAMP, expected); assertTotalNumberOfInvocations(mutationDataBuilder, 1); @@ -166,7 +161,7 @@ public void testDouble() { ByteString expected = ByteString.copyFrom(Bytes.toBytes(value)); ValueMapper mapper = new TestValueMapper(DEFAULT_COLUMN_FAMILY, DEFAULT_COLUMN, NullValueMode.IGNORE); - MutationDataBuilder mutationDataBuilder = mapper.getRecordMutationDataBuilder(value, DEFAULT_TOPIC, TIMESTAMP); + MutationDataBuilder mutationDataBuilder = getRecordMutationDataBuilder(mapper, value); verify(mutationDataBuilder, times(1)) .setCell(DEFAULT_COLUMN_FAMILY, DEFAULT_COLUMN_BYTES, TIMESTAMP, expected); assertTotalNumberOfInvocations(mutationDataBuilder, 1); @@ -178,36 +173,31 @@ public void testDoubleSpecial() { ByteString expected = ByteString.copyFrom(Bytes.toBytes(value)); ValueMapper mapper = new TestValueMapper(DEFAULT_COLUMN_FAMILY, DEFAULT_COLUMN, NullValueMode.IGNORE); - MutationDataBuilder mutationDataBuilder = mapper.getRecordMutationDataBuilder(value, DEFAULT_TOPIC, TIMESTAMP); + MutationDataBuilder mutationDataBuilder = getRecordMutationDataBuilder(mapper, value); verify(mutationDataBuilder, times(1)) .setCell(DEFAULT_COLUMN_FAMILY, DEFAULT_COLUMN_BYTES, TIMESTAMP, expected); assertTotalNumberOfInvocations(mutationDataBuilder, 1); } - @Ignore // TODO: fix it. @Test public void testDate() { - // TODO: is it correct? Or maybe should the implementation first convert it into logical value? - Long value = 1732822801000L; - ByteString expected = ByteString.copyFrom(Bytes.toBytes(value)); + Date date = new Date(1732822801000L); + ByteString expected = ByteString.copyFrom(Bytes.toBytes(date.getTime())); ValueMapper mapper = new TestValueMapper(DEFAULT_COLUMN_FAMILY, DEFAULT_COLUMN, NullValueMode.IGNORE); - MutationDataBuilder mutationDataBuilder = - mapper.getRecordMutationDataBuilder(new Date(value), DEFAULT_TOPIC, TIMESTAMP); + MutationDataBuilder mutationDataBuilder = getRecordMutationDataBuilder(mapper, date); verify(mutationDataBuilder, times(1)) .setCell(DEFAULT_COLUMN_FAMILY, DEFAULT_COLUMN_BYTES, TIMESTAMP, expected); assertTotalNumberOfInvocations(mutationDataBuilder, 1); } - @Ignore // TODO: fix it. @Test public void testDecimal() { - // TODO: is it correct? Or maybe should the implementation first convert it into logical value? - BigDecimal value = new BigDecimal("0.30000000000000000004"); + BigDecimal value = new BigDecimal("0.300000000000000000000000000000001"); ByteString expected = ByteString.copyFrom(Bytes.toBytes(value)); ValueMapper mapper = new TestValueMapper(DEFAULT_COLUMN_FAMILY, DEFAULT_COLUMN, NullValueMode.IGNORE); - MutationDataBuilder mutationDataBuilder = mapper.getRecordMutationDataBuilder(value, DEFAULT_TOPIC, TIMESTAMP); + MutationDataBuilder mutationDataBuilder = getRecordMutationDataBuilder(mapper, value); verify(mutationDataBuilder, times(1)) .setCell(DEFAULT_COLUMN_FAMILY, DEFAULT_COLUMN_BYTES, TIMESTAMP, expected); assertTotalNumberOfInvocations(mutationDataBuilder, 1); @@ -218,7 +208,7 @@ public void testArray() { List value = List.of("1", 2, true); ValueMapper mapper = new TestValueMapper(DEFAULT_COLUMN_FAMILY, DEFAULT_COLUMN, NullValueMode.IGNORE); - MutationDataBuilder mutationDataBuilder = mapper.getRecordMutationDataBuilder(value, DEFAULT_TOPIC, TIMESTAMP); + MutationDataBuilder mutationDataBuilder = getRecordMutationDataBuilder(mapper, value); verify(mutationDataBuilder, times(1)) .setCell( DEFAULT_COLUMN_FAMILY, @@ -236,8 +226,7 @@ public void testRootValueNeedsBothDefaultColumns() { new TestValueMapper(null, null, NullValueMode.WRITE), new TestValueMapper(DEFAULT_COLUMN_FAMILY, null, NullValueMode.WRITE), new TestValueMapper(null, DEFAULT_COLUMN, NullValueMode.WRITE))) { - MutationDataBuilder mutationDataBuilder = - mapper.getRecordMutationDataBuilder(value, DEFAULT_TOPIC, TIMESTAMP); + MutationDataBuilder mutationDataBuilder = getRecordMutationDataBuilder(mapper, value); verify(mutationDataBuilder, times(0)) .setCell( DEFAULT_COLUMN_FAMILY, @@ -247,7 +236,7 @@ public void testRootValueNeedsBothDefaultColumns() { } ValueMapper mapper = new TestValueMapper(DEFAULT_COLUMN_FAMILY, DEFAULT_COLUMN, NullValueMode.WRITE); - MutationDataBuilder mutationDataBuilder = mapper.getRecordMutationDataBuilder(value, DEFAULT_TOPIC, TIMESTAMP); + MutationDataBuilder mutationDataBuilder = getRecordMutationDataBuilder(mapper, value); verify(mutationDataBuilder, times(1)) .setCell( DEFAULT_COLUMN_FAMILY, @@ -259,15 +248,19 @@ public void testRootValueNeedsBothDefaultColumns() { @Test public void testValueNestedOnceNeedsOnlyDefaultColumnFamily() { - Object value = fromJson("{\"key\": 2}"); + String key = "key"; + Long value = 2L; + Struct struct = new Struct(SchemaBuilder.struct().field(key, Schema.INT64_SCHEMA)); + struct.put(key, value); + ValueMapper mapper = new TestValueMapper(DEFAULT_COLUMN_FAMILY, null, NullValueMode.WRITE); - MutationDataBuilder mutationDataBuilder = mapper.getRecordMutationDataBuilder(value, DEFAULT_TOPIC, TIMESTAMP); + MutationDataBuilder mutationDataBuilder = getRecordMutationDataBuilder(mapper, struct); verify(mutationDataBuilder, times(1)) .setCell( DEFAULT_COLUMN_FAMILY, - ByteString.copyFrom("key".getBytes(StandardCharsets.UTF_8)), + ByteString.copyFrom(key.getBytes(StandardCharsets.UTF_8)), TIMESTAMP, - ByteString.copyFrom(Bytes.toBytes(2L))); + ByteString.copyFrom(Bytes.toBytes(value))); assertTotalNumberOfInvocations(mutationDataBuilder, 1); } @@ -285,7 +278,7 @@ public void testDefaultColumnFamilyInterpolation() { String topic = "topic"; String value = "value"; ValueMapper mapper = new TestValueMapper(test.getKey(), DEFAULT_COLUMN, NullValueMode.WRITE); - MutationDataBuilder mutationDataBuilder = mapper.getRecordMutationDataBuilder(value, topic, TIMESTAMP); + MutationDataBuilder mutationDataBuilder = getRecordMutationDataBuilder(mapper, value, topic); verify(mutationDataBuilder, times(1)) .setCell( test.getValue(), @@ -298,28 +291,55 @@ public void testDefaultColumnFamilyInterpolation() { @Test public void testMultipleOperationsAtOnce() { - Object value = fromJson("{\"a\":{\"b\":789},\"c\":true,\"x\":{\"y\":null},\"z\":null}"); + String setColumnFamily = "setColumnFamily"; + String setColumn = "setColumn"; + String setRoot = "setRoot"; + String deleteColumnFamily = "deleteColumnFamily"; + String deleteColumn = "deleteColumn"; + String deleteRoot = "deleteRoot"; + + Integer value = 789; + Boolean rootValue = true; + + Struct createStruct = + new Struct(SchemaBuilder.struct().field(setColumn, Schema.INT32_SCHEMA)) + .put(setColumn, value); + Struct deleteStruct = + new Struct(SchemaBuilder.struct().field(deleteColumn, Schema.OPTIONAL_INT8_SCHEMA)) + .put(deleteColumn, null); + Struct struct = + new Struct( + SchemaBuilder.struct() + .field(setColumnFamily, createStruct.schema()) + .field(setRoot, Schema.BOOLEAN_SCHEMA) + .field(deleteColumnFamily, deleteStruct.schema()) + .field(deleteRoot, Schema.OPTIONAL_INT8_SCHEMA)) + .put(setColumnFamily, createStruct) + .put(setRoot, rootValue) + .put(deleteColumnFamily, deleteStruct) + .put(deleteRoot, null); + ValueMapper mapper = new TestValueMapper(DEFAULT_COLUMN_FAMILY, DEFAULT_COLUMN, NullValueMode.DELETE); - MutationDataBuilder mutationDataBuilder = mapper.getRecordMutationDataBuilder(value, DEFAULT_TOPIC, TIMESTAMP); + MutationDataBuilder mutationDataBuilder = getRecordMutationDataBuilder(mapper, struct); verify(mutationDataBuilder, times(1)) .setCell( - "a", - ByteString.copyFrom("b".getBytes(StandardCharsets.UTF_8)), + setColumnFamily, + ByteString.copyFrom(setColumn.getBytes(StandardCharsets.UTF_8)), TIMESTAMP, - ByteString.copyFrom(Bytes.toBytes(789L))); + ByteString.copyFrom(Bytes.toBytes(value))); verify(mutationDataBuilder, times(1)) .setCell( DEFAULT_COLUMN_FAMILY, - ByteString.copyFrom("c".getBytes(StandardCharsets.UTF_8)), + ByteString.copyFrom(setRoot.getBytes(StandardCharsets.UTF_8)), TIMESTAMP, - ByteString.copyFrom(Bytes.toBytes(true))); + ByteString.copyFrom(Bytes.toBytes(rootValue))); verify(mutationDataBuilder, times(1)) .deleteCells( - "x", - ByteString.copyFrom("y".getBytes(StandardCharsets.UTF_8)), + deleteColumnFamily, + ByteString.copyFrom(deleteColumn.getBytes(StandardCharsets.UTF_8)), Range.TimestampRange.create(0, TIMESTAMP)); - verify(mutationDataBuilder, times(1)).deleteFamily("z"); + verify(mutationDataBuilder, times(1)).deleteFamily(deleteRoot); assertTotalNumberOfInvocations(mutationDataBuilder, 4); } @@ -329,15 +349,10 @@ public void testMap() { Object innerMapKey = "innerMapKey"; String familyToBeDeleted = "familyToBeDeleted"; String columnToBeDeleted = "columnToBeDeleted"; - ByteString columnToBeDeletedBytes = - ByteString.copyFrom(columnToBeDeleted.getBytes(StandardCharsets.UTF_8)); Object innermostNullKey = "innermostNullKey"; Object value = "value"; - ByteString valueBytes = ByteString.copyFrom(((String) value).getBytes(StandardCharsets.UTF_8)); Object valueKey = "valueKey"; - ByteString valueKeyBytes = - ByteString.copyFrom(((String) valueKey).getBytes(StandardCharsets.UTF_8)); Map innermostMap = new HashMap<>(); Map innerMap = new HashMap<>(); @@ -368,31 +383,18 @@ public void testMap() { familyToBeDeleted: null, } */ - String expectedJsonification = "{\"innermostNullKey\":null,\"valueKey\":\"value\"}"; + String expectedJsonification = + "{\"123456\":{\"columnToBeDeleted\":null,\"innerMapKey\":{\"innermostNullKey\":null,\"valueKey\":\"value\"},\"valueKey\":\"value\"},\"familyToBeDeleted\":null,\"valueKey\":\"value\"}"; ByteString expectedJsonificationBytes = ByteString.copyFrom(expectedJsonification.getBytes(StandardCharsets.UTF_8)); ValueMapper mapper = new TestValueMapper(DEFAULT_COLUMN_FAMILY, DEFAULT_COLUMN, NullValueMode.DELETE); - MutationDataBuilder mutationDataBuilder = - mapper.getRecordMutationDataBuilder(outerMap, DEFAULT_TOPIC, TIMESTAMP); - verify(mutationDataBuilder, times(1)).deleteFamily(familyToBeDeleted); - verify(mutationDataBuilder, times(1)) - .setCell(DEFAULT_COLUMN_FAMILY, valueKeyBytes, TIMESTAMP, valueBytes); - verify(mutationDataBuilder, times(1)) - .deleteCells( - outerMapKey.toString(), - columnToBeDeletedBytes, - Range.TimestampRange.create(0, TIMESTAMP)); - verify(mutationDataBuilder, times(1)) - .setCell(outerMapKey.toString(), valueKeyBytes, TIMESTAMP, valueBytes); + MutationDataBuilder mutationDataBuilder = getRecordMutationDataBuilder(mapper, outerMap); verify(mutationDataBuilder, times(1)) .setCell( - outerMapKey.toString(), - ByteString.copyFrom(innerMapKey.toString().getBytes(StandardCharsets.UTF_8)), - TIMESTAMP, - expectedJsonificationBytes); - assertTotalNumberOfInvocations(mutationDataBuilder, 5); + DEFAULT_COLUMN_FAMILY, DEFAULT_COLUMN_BYTES, TIMESTAMP, expectedJsonificationBytes); + assertTotalNumberOfInvocations(mutationDataBuilder, 1); } @Test @@ -402,74 +404,54 @@ public void testJsonificationOfNonJsonNativeTypes() { final String timeFieldName = "KafkaTime"; final String decimalFieldName = "KafkaDecimal"; final String bytesFieldName = "KafkaBytes"; - final Long dateLong = 1488406838808L; - final Date date = new Date(dateLong); + final Date timestamp = new Date(1488406838808L); + final Date time = Date.from(Instant.EPOCH.plus(1234567890, ChronoUnit.MILLIS)); + final Date date = Date.from(Instant.EPOCH.plus(363, ChronoUnit.DAYS)); final String decimalString = "0.30000000000000004"; - final Integer decimalScale = 0; + final Integer decimalScale = 17; final BigDecimal decimal = new BigDecimal(decimalString); final byte[] bytes = "bytes\0".getBytes(StandardCharsets.UTF_8); - final String schemaStructFieldName = "schema"; - final ByteString schemaStructFieldNameBytes = - ByteString.copyFrom(schemaStructFieldName.getBytes(StandardCharsets.UTF_8)); - final String schemalessMapFieldName = "schemaless"; - final ByteString schemalessMapFieldNameBytes = - ByteString.copyFrom(schemalessMapFieldName.getBytes(StandardCharsets.UTF_8)); - - Schema structSchema = - SchemaBuilder.struct() - .field(dateFieldName, org.apache.kafka.connect.data.Date.SCHEMA) - .field(timestampFieldName, org.apache.kafka.connect.data.Timestamp.SCHEMA) - .field(timeFieldName, org.apache.kafka.connect.data.Timestamp.SCHEMA) - .field(decimalFieldName, org.apache.kafka.connect.data.Decimal.schema(decimalScale)) - .field(bytesFieldName, Schema.BYTES_SCHEMA) - .build(); - Struct struct = new Struct(structSchema); - Map map = new TreeMap<>(); // Note we need this map to be ordered! - - Map outerMap = new HashMap<>(); - Map innerMap = new HashMap<>(); - outerMap.put(DEFAULT_COLUMN_FAMILY, innerMap); - innerMap.put(schemaStructFieldName, struct); - innerMap.put(schemalessMapFieldName, map); - struct.put(dateFieldName, date); - map.put(dateFieldName, date); - struct.put(timestampFieldName, date); - map.put(timestampFieldName, date); - struct.put(timeFieldName, date); - map.put(timeFieldName, date); - struct.put(decimalFieldName, decimal); - map.put(decimalFieldName, decimal); - struct.put(bytesFieldName, bytes); - map.put(bytesFieldName, bytes); - - String expectedStringificationWithoutSchema = - "{\"KafkaBytes\":\"Ynl0ZXMA\",\"KafkaDate\":1488406838808,\"KafkaDecimal\":0.30000000000000004,\"KafkaTime\":1488406838808,\"KafkaTimestamp\":1488406838808}"; - ByteString expectedStringificationWithoutSchemaBytes = - ByteString.copyFrom(expectedStringificationWithoutSchema.getBytes(StandardCharsets.UTF_8)); - // TODO: shouldn't it be different than schemaless serialization? (e.g., count 'time' modulo - // 24h) - String expectedStringificationWithSchema = - "{\"KafkaDate\":1488406838808,\"KafkaTimestamp\":1488406838808,\"KafkaTime\":1488406838808,\"KafkaDecimal\":0.30000000000000004,\"KafkaBytes\":\"Ynl0ZXMA\"}"; - ByteString expectedStringificationWithSchemaBytes = - ByteString.copyFrom(expectedStringificationWithSchema.getBytes(StandardCharsets.UTF_8)); + Struct structToBeJsonified = + new Struct( + SchemaBuilder.struct() + .field(dateFieldName, org.apache.kafka.connect.data.Date.SCHEMA) + .field(timestampFieldName, org.apache.kafka.connect.data.Timestamp.SCHEMA) + .field(timeFieldName, org.apache.kafka.connect.data.Timestamp.SCHEMA) + .field( + decimalFieldName, + org.apache.kafka.connect.data.Decimal.schema(decimalScale)) + .field(bytesFieldName, Schema.BYTES_SCHEMA) + .build()) + .put(dateFieldName, date) + .put(timestampFieldName, timestamp) + .put(timeFieldName, time) + .put(decimalFieldName, decimal) + .put(bytesFieldName, bytes); + + String innerField = "innerField"; + String outerField = "outerField"; + Struct innerStruct = + new Struct(SchemaBuilder.struct().field(innerField, structToBeJsonified.schema())) + .put(innerField, structToBeJsonified); + Struct outerStruct = + new Struct(SchemaBuilder.struct().field(outerField, innerStruct.schema())) + .put(outerField, innerStruct); + + String expectedStringification = + "{\"KafkaDate\":363,\"KafkaTimestamp\":1488406838808,\"KafkaTime\":1234567890,\"KafkaDecimal\":\"apTXT0MABA==\",\"KafkaBytes\":\"Ynl0ZXMA\"}"; + ByteString expectedStringificationBytes = + ByteString.copyFrom(expectedStringification.getBytes(StandardCharsets.UTF_8)); ValueMapper mapper = new TestValueMapper(null, null, NullValueMode.DELETE); - MutationDataBuilder mutationDataBuilder = - mapper.getRecordMutationDataBuilder(outerMap, DEFAULT_TOPIC, TIMESTAMP); + MutationDataBuilder mutationDataBuilder = getRecordMutationDataBuilder(mapper, outerStruct); verify(mutationDataBuilder, times(1)) .setCell( - DEFAULT_COLUMN_FAMILY, - schemalessMapFieldNameBytes, - TIMESTAMP, - expectedStringificationWithoutSchemaBytes); - verify(mutationDataBuilder, times(1)) - .setCell( - DEFAULT_COLUMN_FAMILY, - schemaStructFieldNameBytes, + outerField, + ByteString.copyFrom(innerField.getBytes(StandardCharsets.UTF_8)), TIMESTAMP, - expectedStringificationWithSchemaBytes); - assertTotalNumberOfInvocations(mutationDataBuilder, 2); + expectedStringificationBytes); + assertTotalNumberOfInvocations(mutationDataBuilder, 1); } @Test @@ -534,8 +516,7 @@ public void testStruct() { */ ValueMapper mapper = new TestValueMapper(DEFAULT_COLUMN_FAMILY, DEFAULT_COLUMN, NullValueMode.DELETE); - MutationDataBuilder mutationDataBuilder = - mapper.getRecordMutationDataBuilder(struct, DEFAULT_TOPIC, TIMESTAMP); + MutationDataBuilder mutationDataBuilder = getRecordMutationDataBuilder(mapper, struct); verify(mutationDataBuilder, times(1)) .setCell(DEFAULT_COLUMN_FAMILY, valueFieldNameBytes, TIMESTAMP, ByteString.copyFrom(value)); verify(mutationDataBuilder, times(1)).deleteFamily(optionalFieldName); @@ -555,69 +536,175 @@ public void testStruct() { } @Test - public void testEmpty() { + public void testEmptyStruct() { Schema emptyStructSchema = SchemaBuilder.struct().build(); Struct emptyStruct = new Struct(emptyStructSchema); - Map emptyMap = new HashMap<>(); ValueMapper mapper = new TestValueMapper(DEFAULT_COLUMN_FAMILY, DEFAULT_COLUMN, NullValueMode.WRITE); - for (Object value : List.of(emptyMap, emptyStruct)) { - MutationDataBuilder mutationDataBuilder = - mapper.getRecordMutationDataBuilder(value, DEFAULT_TOPIC, TIMESTAMP); - assertTotalNumberOfInvocations(mutationDataBuilder, 0); - assertTrue(mutationDataBuilder.maybeBuild(TARGET_TABLE_NAME, ROW_KEY).isEmpty()); - } + MutationDataBuilder mutationDataBuilder = getRecordMutationDataBuilder(mapper, emptyStruct); + assertTotalNumberOfInvocations(mutationDataBuilder, 0); + assertTrue(mutationDataBuilder.maybeBuild(TARGET_TABLE_NAME, ROW_KEY).isEmpty()); } @Test public void testSimpleCase1() { - Object value = fromJson("{\"foo\": {\"bar\": 1}}"); + Integer value = 1; + String innerField = "bar"; + String outerField = "foo"; + + Struct innerStruct = + new Struct(SchemaBuilder.struct().field(innerField, Schema.INT32_SCHEMA)) + .put(innerField, value); + Struct outerStruct = + new Struct(SchemaBuilder.struct().field(outerField, innerStruct.schema())) + .put(outerField, innerStruct); + ValueMapper mapper = new TestValueMapper(null, null, NullValueMode.IGNORE); - MutationDataBuilder mutationDataBuilder = mapper.getRecordMutationDataBuilder(value, DEFAULT_TOPIC, TIMESTAMP); + MutationDataBuilder mutationDataBuilder = getRecordMutationDataBuilder(mapper, outerStruct); verify(mutationDataBuilder, times(1)) .setCell( - "foo", - ByteString.copyFrom("bar".getBytes(StandardCharsets.UTF_8)), + outerField, + ByteString.copyFrom(innerField.getBytes(StandardCharsets.UTF_8)), TIMESTAMP, - ByteString.copyFrom(Bytes.toBytes(1L))); + ByteString.copyFrom(Bytes.toBytes(value))); assertTotalNumberOfInvocations(mutationDataBuilder, 1); assertTrue(mutationDataBuilder.maybeBuild(TARGET_TABLE_NAME, ROW_KEY).isPresent()); } @Test public void testSimpleCase2() { - Object value = fromJson("{\"foo\": {\"bar\": {\"fizz\": 1}}}"); + Integer value = 1; + String innerField = "fizz"; + String middleField = "bar"; + String outerField = "foo"; + + Struct innerStruct = + new Struct(SchemaBuilder.struct().field(innerField, Schema.INT32_SCHEMA)) + .put(innerField, value); + Struct middleStruct = + new Struct(SchemaBuilder.struct().field(middleField, innerStruct.schema())) + .put(middleField, innerStruct); + Struct outerStruct = + new Struct(SchemaBuilder.struct().field(outerField, middleStruct.schema())) + .put(outerField, middleStruct); + ValueMapper mapper = new TestValueMapper(null, null, NullValueMode.IGNORE); - MutationDataBuilder mutationDataBuilder = mapper.getRecordMutationDataBuilder(value, DEFAULT_TOPIC, TIMESTAMP); + MutationDataBuilder mutationDataBuilder = getRecordMutationDataBuilder(mapper, outerStruct); verify(mutationDataBuilder, times(1)) .setCell( - "foo", - ByteString.copyFrom("bar".getBytes(StandardCharsets.UTF_8)), + outerField, + ByteString.copyFrom(middleField.getBytes(StandardCharsets.UTF_8)), TIMESTAMP, - ByteString.copyFrom("{\"fizz\":1}".getBytes(StandardCharsets.UTF_8))); + ByteString.copyFrom( + ("{\"" + innerField + "\":" + value + "}").getBytes(StandardCharsets.UTF_8))); assertTrue(mutationDataBuilder.maybeBuild(TARGET_TABLE_NAME, ROW_KEY).isPresent()); } @Test public void testSimpleCase3() { - Object value = fromJson("{\"foo\": 1}"); + Integer value = 1; + String field = "foo"; + Struct struct = + new Struct(SchemaBuilder.struct().field(field, Schema.INT32_SCHEMA)).put(field, value); + ValueMapper mapper = new TestValueMapper(DEFAULT_COLUMN_FAMILY, null, NullValueMode.IGNORE); - MutationDataBuilder mutationDataBuilder = mapper.getRecordMutationDataBuilder(value, DEFAULT_TOPIC, TIMESTAMP); + MutationDataBuilder mutationDataBuilder = getRecordMutationDataBuilder(mapper, struct); verify(mutationDataBuilder, times(1)) .setCell( DEFAULT_COLUMN_FAMILY, - ByteString.copyFrom("foo".getBytes(StandardCharsets.UTF_8)), + ByteString.copyFrom(field.getBytes(StandardCharsets.UTF_8)), TIMESTAMP, - ByteString.copyFrom(Bytes.toBytes(1L))); + ByteString.copyFrom(Bytes.toBytes(value))); assertTotalNumberOfInvocations(mutationDataBuilder, 1); assertTrue(mutationDataBuilder.maybeBuild(TARGET_TABLE_NAME, ROW_KEY).isPresent()); } + @Test + public void testComplicatedCase() { + String innerStructKey = "innerStructKey"; + String familyToBeDeleted = "familyToBeDeleted"; + String columnToBeDeleted = "columnToBeDeleted"; + ByteString columnToBeDeletedBytes = + ByteString.copyFrom(columnToBeDeleted.getBytes(StandardCharsets.UTF_8)); + String innermostNullKey = "innermostNullKey"; + String outerStructKey = "outerStructKey"; + + String value = "value"; + ByteString valueBytes = ByteString.copyFrom((value).getBytes(StandardCharsets.UTF_8)); + String valueKey = "valueKey"; + ByteString valueKeyBytes = ByteString.copyFrom((valueKey).getBytes(StandardCharsets.UTF_8)); + + Struct innermostStruct = + new Struct( + SchemaBuilder.struct() + .field(valueKey, Schema.STRING_SCHEMA) + .field(innermostNullKey, Schema.OPTIONAL_INT8_SCHEMA)) + .put(valueKey, value) + .put(innermostNullKey, null); + Struct innerStruct = + new Struct( + SchemaBuilder.struct() + .field(innerStructKey, innermostStruct.schema()) + .field(valueKey, Schema.STRING_SCHEMA) + .field(columnToBeDeleted, Schema.OPTIONAL_INT8_SCHEMA)) + .put(innerStructKey, innermostStruct) + .put(valueKey, value) + .put(columnToBeDeleted, null); + Struct outerStruct = + new Struct( + SchemaBuilder.struct() + .field(outerStructKey, innerStruct.schema()) + .field(valueKey, Schema.STRING_SCHEMA) + .field(familyToBeDeleted, Schema.OPTIONAL_INT8_SCHEMA)) + .put(outerStructKey, innerStruct) + .put(valueKey, value) + .put(familyToBeDeleted, null); + + /* + { + outerStructKey: { + innerStructKey: { + valueKey: value, + innermostNullKey: null, + } + valueKey: value, + columnToBeDeleted: null, + } + valueKey: value, + familyToBeDeleted: null, + } + */ + String expectedJsonification = "{\"valueKey\":\"value\",\"innermostNullKey\":null}"; + ByteString expectedJsonificationBytes = + ByteString.copyFrom(expectedJsonification.getBytes(StandardCharsets.UTF_8)); + + ValueMapper mapper = + new TestValueMapper(DEFAULT_COLUMN_FAMILY, DEFAULT_COLUMN, NullValueMode.DELETE); + MutationDataBuilder mutationDataBuilder = getRecordMutationDataBuilder(mapper, outerStruct); + verify(mutationDataBuilder, times(1)).deleteFamily(familyToBeDeleted); + verify(mutationDataBuilder, times(1)) + .setCell(DEFAULT_COLUMN_FAMILY, valueKeyBytes, TIMESTAMP, valueBytes); + verify(mutationDataBuilder, times(1)) + .deleteCells( + outerStructKey.toString(), + columnToBeDeletedBytes, + Range.TimestampRange.create(0, TIMESTAMP)); + verify(mutationDataBuilder, times(1)) + .setCell(outerStructKey.toString(), valueKeyBytes, TIMESTAMP, valueBytes); + verify(mutationDataBuilder, times(1)) + .setCell( + outerStructKey.toString(), + ByteString.copyFrom(innerStructKey.toString().getBytes(StandardCharsets.UTF_8)), + TIMESTAMP, + expectedJsonificationBytes); + assertTotalNumberOfInvocations(mutationDataBuilder, 5); + } + @Test public void testNullModeIgnoreRoot() { ValueMapper mapper = new TestValueMapper(DEFAULT_COLUMN_FAMILY, DEFAULT_COLUMN, NullValueMode.IGNORE); - MutationDataBuilder mutationDataBuilder = mapper.getRecordMutationDataBuilder(null, DEFAULT_TOPIC, TIMESTAMP); + MutationDataBuilder mutationDataBuilder = getRecordMutationDataBuilder(mapper, null); assertTotalNumberOfInvocations(mutationDataBuilder, 0); assertTrue(mutationDataBuilder.maybeBuild(TARGET_TABLE_NAME, ROW_KEY).isEmpty()); } @@ -627,7 +714,7 @@ public void testNullModeIgnoreNestedOnce() { ValueMapper mapper = new TestValueMapper(DEFAULT_COLUMN_FAMILY, DEFAULT_COLUMN, NullValueMode.IGNORE); MutationDataBuilder mutationDataBuilder = - mapper.getRecordMutationDataBuilder(getStructhWithNullOnNthNestingLevel(1), DEFAULT_TOPIC, TIMESTAMP); + getRecordMutationDataBuilder(mapper, getStructhWithNullOnNthNestingLevel(1)); assertTotalNumberOfInvocations(mutationDataBuilder, 0); assertTrue(mutationDataBuilder.maybeBuild(TARGET_TABLE_NAME, ROW_KEY).isEmpty()); } @@ -637,7 +724,7 @@ public void testNullModeIgnoreNestedTwice() { ValueMapper mapper = new TestValueMapper(DEFAULT_COLUMN_FAMILY, DEFAULT_COLUMN, NullValueMode.IGNORE); MutationDataBuilder mutationDataBuilder = - mapper.getRecordMutationDataBuilder(getStructhWithNullOnNthNestingLevel(2), DEFAULT_TOPIC, TIMESTAMP); + getRecordMutationDataBuilder(mapper, getStructhWithNullOnNthNestingLevel(2)); assertTotalNumberOfInvocations(mutationDataBuilder, 0); assertTrue(mutationDataBuilder.maybeBuild(TARGET_TABLE_NAME, ROW_KEY).isEmpty()); } @@ -646,7 +733,7 @@ public void testNullModeIgnoreNestedTwice() { public void testNullModeWriteRoot() { ValueMapper mapper = new TestValueMapper(DEFAULT_COLUMN_FAMILY, DEFAULT_COLUMN, NullValueMode.WRITE); - MutationDataBuilder mutationDataBuilder = mapper.getRecordMutationDataBuilder(null, DEFAULT_TOPIC, TIMESTAMP); + MutationDataBuilder mutationDataBuilder = getRecordMutationDataBuilder(mapper, null); verify(mutationDataBuilder, times(1)) .setCell(DEFAULT_COLUMN_FAMILY, DEFAULT_COLUMN_BYTES, TIMESTAMP, ByteString.empty()); assertTotalNumberOfInvocations(mutationDataBuilder, 1); @@ -658,7 +745,7 @@ public void testNullModeWriteNestedOnce() { ValueMapper mapper = new TestValueMapper(DEFAULT_COLUMN_FAMILY, DEFAULT_COLUMN, NullValueMode.WRITE); MutationDataBuilder mutationDataBuilder = - mapper.getRecordMutationDataBuilder(getStructhWithNullOnNthNestingLevel(1), DEFAULT_TOPIC, TIMESTAMP); + getRecordMutationDataBuilder(mapper, getStructhWithNullOnNthNestingLevel(1)); verify(mutationDataBuilder, times(1)) .setCell( DEFAULT_COLUMN_FAMILY, @@ -674,7 +761,7 @@ public void testNullModeWriteNestedTwice() { ValueMapper mapper = new TestValueMapper(DEFAULT_COLUMN_FAMILY, DEFAULT_COLUMN, NullValueMode.WRITE); MutationDataBuilder mutationDataBuilder = - mapper.getRecordMutationDataBuilder(getStructhWithNullOnNthNestingLevel(2), DEFAULT_TOPIC, TIMESTAMP); + getRecordMutationDataBuilder(mapper, getStructhWithNullOnNthNestingLevel(2)); verify(mutationDataBuilder, times(1)) .setCell( NESTED_NULL_STRUCT_FIELD_NAME, @@ -689,7 +776,7 @@ public void testNullModeWriteNestedTwice() { public void testNullModeDeleteRoot() { ValueMapper mapper = new TestValueMapper(DEFAULT_COLUMN_FAMILY, DEFAULT_COLUMN, NullValueMode.DELETE); - MutationDataBuilder mutationDataBuilder = mapper.getRecordMutationDataBuilder(null, DEFAULT_TOPIC, TIMESTAMP); + MutationDataBuilder mutationDataBuilder = getRecordMutationDataBuilder(mapper, null); verify(mutationDataBuilder, times(1)).deleteRow(); assertTotalNumberOfInvocations(mutationDataBuilder, 1); assertTrue(mutationDataBuilder.maybeBuild(TARGET_TABLE_NAME, ROW_KEY).isPresent()); @@ -700,7 +787,7 @@ public void testNullModeDeleteNestedOnce() { ValueMapper mapper = new TestValueMapper(DEFAULT_COLUMN_FAMILY, DEFAULT_COLUMN, NullValueMode.DELETE); MutationDataBuilder mutationDataBuilder = - mapper.getRecordMutationDataBuilder(getStructhWithNullOnNthNestingLevel(1), DEFAULT_TOPIC, TIMESTAMP); + getRecordMutationDataBuilder(mapper, getStructhWithNullOnNthNestingLevel(1)); verify(mutationDataBuilder, times(1)).deleteFamily(NESTED_NULL_STRUCT_FIELD_NAME); assertTotalNumberOfInvocations(mutationDataBuilder, 1); assertTrue(mutationDataBuilder.maybeBuild(TARGET_TABLE_NAME, ROW_KEY).isPresent()); @@ -711,7 +798,7 @@ public void testNullModeDeleteNestedTwice() { ValueMapper mapper = new TestValueMapper(DEFAULT_COLUMN_FAMILY, DEFAULT_COLUMN, NullValueMode.DELETE); MutationDataBuilder mutationDataBuilder = - mapper.getRecordMutationDataBuilder(getStructhWithNullOnNthNestingLevel(2), DEFAULT_TOPIC, TIMESTAMP); + getRecordMutationDataBuilder(mapper, getStructhWithNullOnNthNestingLevel(2)); verify(mutationDataBuilder, times(1)) .deleteCells( NESTED_NULL_STRUCT_FIELD_NAME, NESTED_NULL_STRUCT_FIELD_NAME_BYTES, TIMESTAMP_RANGE); @@ -727,7 +814,7 @@ public void testNullModeNestedThrice() { ByteString expectedJsonificationBytes = ByteString.copyFrom(expectedJsonification.getBytes(StandardCharsets.UTF_8)); MutationDataBuilder mutationDataBuilder = - mapper.getRecordMutationDataBuilder(getStructhWithNullOnNthNestingLevel(3), DEFAULT_TOPIC, TIMESTAMP); + getRecordMutationDataBuilder(mapper, getStructhWithNullOnNthNestingLevel(3)); verify(mutationDataBuilder, times(1)) .setCell( NESTED_NULL_STRUCT_FIELD_NAME, @@ -738,37 +825,28 @@ public void testNullModeNestedThrice() { assertTrue(mutationDataBuilder.maybeBuild(TARGET_TABLE_NAME, ROW_KEY).isPresent()); } - private static Struct getStructhWithNullOnNthNestingLevel(int n) { - assert n > 0; - - Schema schema = SchemaBuilder.struct().field(NESTED_NULL_STRUCT_FIELD_NAME, SchemaBuilder.struct().optional()).build(); - // We consider a Struct with a null child to be a level 1 nested struct. - Struct struct = new Struct(schema); - - while (n > 1) { - n -= 1; - schema = SchemaBuilder.struct().field(NESTED_NULL_STRUCT_FIELD_NAME, schema).optional().build(); - final Struct outerStruct = new Struct(schema); - outerStruct.put(NESTED_NULL_STRUCT_FIELD_NAME, struct); - struct = outerStruct; - } - return struct; - } - @Test public void testDefaultColumnFamilySubstitution() { ValueMapper mapper = new TestValueMapper( ConfigInterpolation.TOPIC_PLACEHOLDER, DEFAULT_COLUMN, NullValueMode.WRITE); - MutationDataBuilder mutationDataBuilder = mapper.getRecordMutationDataBuilder(null, DEFAULT_TOPIC, TIMESTAMP); + MutationDataBuilder mutationDataBuilder = getRecordMutationDataBuilder(mapper, null); verify(mutationDataBuilder, times(1)) .setCell(DEFAULT_TOPIC, DEFAULT_COLUMN_BYTES, TIMESTAMP, ByteString.empty()); assertTotalNumberOfInvocations(mutationDataBuilder, 1); assertTrue(mutationDataBuilder.maybeBuild(TARGET_TABLE_NAME, ROW_KEY).isPresent()); } - private static Object fromJson(String s) { - return jsonConverter.toConnectData(DEFAULT_TOPIC, s.getBytes(StandardCharsets.UTF_8)).value(); + private MutationDataBuilder getRecordMutationDataBuilder(ValueMapper mapper, Object kafkaValue) { + return getRecordMutationDataBuilder(mapper, kafkaValue, DEFAULT_TOPIC); + } + + private MutationDataBuilder getRecordMutationDataBuilder( + ValueMapper mapper, Object kafkaValue, String topic) { + // We use `null` in this test since our code for now uses Schema only to warn the user when an + // unsupported logical type is encountered. + SchemaAndValue schemaAndValue = new SchemaAndValue(null, kafkaValue); + return mapper.getRecordMutationDataBuilder(schemaAndValue, topic, TIMESTAMP); } private static class TestValueMapper extends ValueMapper { diff --git a/google-cloud-bigtable-kafka-connect-sink/src/test/java/com/google/cloud/kafka/connect/bigtable/util/NestedNullStructFactory.java b/google-cloud-bigtable-kafka-connect-sink/src/test/java/com/google/cloud/kafka/connect/bigtable/util/NestedNullStructFactory.java new file mode 100644 index 0000000000..1280ddb5a5 --- /dev/null +++ b/google-cloud-bigtable-kafka-connect-sink/src/test/java/com/google/cloud/kafka/connect/bigtable/util/NestedNullStructFactory.java @@ -0,0 +1,49 @@ +/* + * Copyright 2025 Google LLC + * + * Licensed under the Apache License, Version 2.0 (the "License"); + * you may not use this file except in compliance with the License. + * You may obtain a copy of the License at + * + * https://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ +package com.google.cloud.kafka.connect.bigtable.util; + +import com.google.protobuf.ByteString; +import java.nio.charset.StandardCharsets; +import org.apache.kafka.connect.data.Schema; +import org.apache.kafka.connect.data.SchemaBuilder; +import org.apache.kafka.connect.data.Struct; + +public class NestedNullStructFactory { + public static final String NESTED_NULL_STRUCT_FIELD_NAME = "struct"; + public static final ByteString NESTED_NULL_STRUCT_FIELD_NAME_BYTES = + ByteString.copyFrom(NESTED_NULL_STRUCT_FIELD_NAME.getBytes(StandardCharsets.UTF_8)); + + public static Struct getStructhWithNullOnNthNestingLevel(int n) { + assert n > 0; + + Schema schema = + SchemaBuilder.struct() + .field(NESTED_NULL_STRUCT_FIELD_NAME, SchemaBuilder.struct().optional()) + .build(); + // We consider a Struct with a null child to be a level 1 nested struct. + Struct struct = new Struct(schema); + + while (n > 1) { + n -= 1; + schema = + SchemaBuilder.struct().field(NESTED_NULL_STRUCT_FIELD_NAME, schema).optional().build(); + final Struct outerStruct = new Struct(schema); + outerStruct.put(NESTED_NULL_STRUCT_FIELD_NAME, struct); + struct = outerStruct; + } + return struct; + } +} From 0754eda626f12d6a44ce5db024c339c2d1d4657f Mon Sep 17 00:00:00 2001 From: Adam Czajkowski Date: Thu, 30 Jan 2025 16:12:30 +0100 Subject: [PATCH 22/76] More integration tests --- .../connect/bigtable/integration/BaseIT.java | 44 ++- .../BaseKafkaConnectBigtableIT.java | 55 +++ ...eKafkaConnectBigtableSchemaRegistryIT.java | 37 ++ .../integration/BaseKafkaConnectIT.java | 84 +++++ .../connect/bigtable/integration/BasicIT.java | 52 +-- .../integration/BigtableClientIT.java | 45 +++ .../bigtable/integration/ConfigIT.java | 2 +- .../integration/ConfluentCompatibilityIT.java | 263 +++++++++++++ .../integration/DifferentConvertersIT.java | 245 ++++++++++++ .../bigtable/integration/ErrorHandlingIT.java | 37 ++ .../integration/ErrorReportingIT.java | 107 ++++++ .../bigtable/integration/InsertUpsertIT.java | 86 +++++ .../bigtable/integration/NullHandlingIT.java | 354 ++++++++++++++++++ .../integration/ResourceAutoCreationIT.java | 159 ++++++++ .../bigtable/integration/VersionIT.java | 5 +- .../bigtable/mapping/ValueMapperTest.java | 31 +- .../bigtable/util/SchemaRegistryTestUtil.java | 84 +++++ .../confluent_compat_topic/KAFKA_VALUE | 1 + .../key_containers/data.json | 1 + .../key_containers/key-schema.json | 33 ++ .../key_containers/value-schema.json | 1 + .../confluent_compat_topic/KAFKA_VALUE | 1 + .../confluent_compat_topic/KAFKA_VALUE | 1 + .../confluent_compat_topic/KAFKA_VALUE | 1 + .../key_logicals/data.json | 3 + .../key_logicals/key-schema.json | 37 ++ .../key_logicals/value-schema.json | 1 + .../confluent_compat_topic/KAFKA_VALUE | 1 + .../key_matryoshkas/data.json | 1 + .../key_matryoshkas/key-schema.json | 47 +++ .../key_matryoshkas/value-schema.json | 1 + .../confluent_compat_topic/KAFKA_VALUE | 1 + .../key_nestedlogicals/data.json | 1 + .../key_nestedlogicals/key-schema.json | 55 +++ .../key_nestedlogicals/value-schema.json | 1 + .../confluent_compat_topic/KAFKA_VALUE | 1 + .../key_primitives/data.json | 1 + .../key_primitives/key-schema.json | 34 ++ .../key_primitives/value-schema.json | 1 + .../containers1/confluent_compat_topic/array | 1 + .../containers1/confluent_compat_topic/map | 1 + .../containers1/struct/integer | Bin 0 -> 4 bytes .../value_containers/data.json | 1 + .../value_containers/key-schema.json | 1 + .../value_containers/value-schema.json | 33 ++ .../confluent_compat_topic/row | Bin 0 -> 4 bytes .../matryoshka-dolls1/inner/inner_int | 1 + .../matryoshka-dolls1/inner/inner_string | 1 + .../matryoshka-dolls1/middle/inner | 1 + .../matryoshka-dolls1/middle/middle_array | 1 + .../value_matryoshkas/data.json | 1 + .../value_matryoshkas/key-schema.json | 1 + .../value_matryoshkas/value-schema.json | 47 +++ .../nestedlogicals1/outer/inner | 1 + .../value_nestedlogicals/data.json | 1 + .../value_nestedlogicals/key-schema.json | 1 + .../value_nestedlogicals/value-schema.json | 55 +++ .../nulls1/confluent_compat_topic/f1 | 1 + .../nulls1/confluent_compat_topic/f2 | 0 .../nulls1/confluent_compat_topic/f3 | Bin 0 -> 4 bytes .../nulls1/confluent_compat_topic/f4 | Bin 0 -> 1 bytes .../nulls1/confluent_compat_topic/row | Bin 0 -> 4 bytes .../nulls2/confluent_compat_topic/f1 | 1 + .../nulls2/confluent_compat_topic/f2 | 1 + .../nulls2/confluent_compat_topic/f3 | Bin 0 -> 4 bytes .../nulls2/confluent_compat_topic/f4 | 0 .../nulls2/confluent_compat_topic/row | Bin 0 -> 4 bytes .../nulls3/confluent_compat_topic/f1 | 1 + .../nulls3/confluent_compat_topic/f2 | 0 .../nulls3/confluent_compat_topic/f3 | 0 .../nulls3/confluent_compat_topic/f4 | 1 + .../nulls3/confluent_compat_topic/row | Bin 0 -> 4 bytes .../nulls4/confluent_compat_topic/f1 | 1 + .../nulls4/confluent_compat_topic/f2 | 1 + .../nulls4/confluent_compat_topic/f3 | 0 .../nulls4/confluent_compat_topic/f4 | 0 .../nulls4/confluent_compat_topic/row | Bin 0 -> 4 bytes .../value_nulls/data.json | 4 + .../value_nulls/key-schema.json | 1 + .../value_nulls/value-schema.json | 35 ++ .../confluent_compat_topic/boolean_prim | Bin 0 -> 1 bytes .../confluent_compat_topic/double_prim | 1 + .../confluent_compat_topic/float_prim | 1 + .../confluent_compat_topic/int_prim | Bin 0 -> 4 bytes .../confluent_compat_topic/long_prim | Bin 0 -> 8 bytes .../confluent_compat_topic/null_prim | 0 .../primitives1/confluent_compat_topic/row | Bin 0 -> 4 bytes .../confluent_compat_topic/string_prim | 1 + .../value_primitives/data.json | 1 + .../value_primitives/key-schema.json | 1 + .../value_primitives/value-schema.json | 41 ++ 91 files changed, 2076 insertions(+), 80 deletions(-) create mode 100644 google-cloud-bigtable-kafka-connect-sink/src/test/java/com/google/cloud/kafka/connect/bigtable/integration/BaseKafkaConnectBigtableIT.java create mode 100644 google-cloud-bigtable-kafka-connect-sink/src/test/java/com/google/cloud/kafka/connect/bigtable/integration/BaseKafkaConnectBigtableSchemaRegistryIT.java create mode 100644 google-cloud-bigtable-kafka-connect-sink/src/test/java/com/google/cloud/kafka/connect/bigtable/integration/BaseKafkaConnectIT.java create mode 100644 google-cloud-bigtable-kafka-connect-sink/src/test/java/com/google/cloud/kafka/connect/bigtable/integration/BigtableClientIT.java create mode 100644 google-cloud-bigtable-kafka-connect-sink/src/test/java/com/google/cloud/kafka/connect/bigtable/integration/ConfluentCompatibilityIT.java create mode 100644 google-cloud-bigtable-kafka-connect-sink/src/test/java/com/google/cloud/kafka/connect/bigtable/integration/DifferentConvertersIT.java create mode 100644 google-cloud-bigtable-kafka-connect-sink/src/test/java/com/google/cloud/kafka/connect/bigtable/integration/ErrorHandlingIT.java create mode 100644 google-cloud-bigtable-kafka-connect-sink/src/test/java/com/google/cloud/kafka/connect/bigtable/integration/ErrorReportingIT.java create mode 100644 google-cloud-bigtable-kafka-connect-sink/src/test/java/com/google/cloud/kafka/connect/bigtable/integration/InsertUpsertIT.java create mode 100644 google-cloud-bigtable-kafka-connect-sink/src/test/java/com/google/cloud/kafka/connect/bigtable/integration/NullHandlingIT.java create mode 100644 google-cloud-bigtable-kafka-connect-sink/src/test/java/com/google/cloud/kafka/connect/bigtable/integration/ResourceAutoCreationIT.java create mode 100644 google-cloud-bigtable-kafka-connect-sink/src/test/java/com/google/cloud/kafka/connect/bigtable/util/SchemaRegistryTestUtil.java create mode 100644 google-cloud-bigtable-kafka-connect-sink/src/test/resources/compatibility_test_cases/key_containers/confluent_sink_output/{a=1, b=4, c=3, d=2}#[1, 2, 3, 4]#Struct{integer=1}/confluent_compat_topic/KAFKA_VALUE create mode 100644 google-cloud-bigtable-kafka-connect-sink/src/test/resources/compatibility_test_cases/key_containers/data.json create mode 100644 google-cloud-bigtable-kafka-connect-sink/src/test/resources/compatibility_test_cases/key_containers/key-schema.json create mode 100644 google-cloud-bigtable-kafka-connect-sink/src/test/resources/compatibility_test_cases/key_containers/value-schema.json create mode 100644 google-cloud-bigtable-kafka-connect-sink/src/test/resources/compatibility_test_cases/key_logicals/confluent_sink_output/Mon Jul 11 22:11:42 UTC 2016#Thu Jan 01 00:00:16 UTC 1970#Mon Jul 11 00:00:00 UTC 2016#1.089357896855742840E+68/confluent_compat_topic/KAFKA_VALUE create mode 100644 google-cloud-bigtable-kafka-connect-sink/src/test/resources/compatibility_test_cases/key_logicals/confluent_sink_output/Thu Jan 01 00:00:00 UTC 1970#Thu Jan 01 00:00:00 UTC 1970#Thu Jan 01 00:00:00 UTC 1970#2.53635900E+58/confluent_compat_topic/KAFKA_VALUE create mode 100644 google-cloud-bigtable-kafka-connect-sink/src/test/resources/compatibility_test_cases/key_logicals/confluent_sink_output/Thu Jan 01 11:40:00 UTC 1970#Thu Jan 01 00:00:04 UTC 1970#Thu Jul 02 00:00:00 UTC 1981#1.6622282361690E+63/confluent_compat_topic/KAFKA_VALUE create mode 100644 google-cloud-bigtable-kafka-connect-sink/src/test/resources/compatibility_test_cases/key_logicals/data.json create mode 100644 google-cloud-bigtable-kafka-connect-sink/src/test/resources/compatibility_test_cases/key_logicals/key-schema.json create mode 100644 google-cloud-bigtable-kafka-connect-sink/src/test/resources/compatibility_test_cases/key_logicals/value-schema.json create mode 100644 google-cloud-bigtable-kafka-connect-sink/src/test/resources/compatibility_test_cases/key_matryoshkas/confluent_sink_output/1#Struct{middle_array=[42.0, 42.42, 42.4242],inner=Struct{inner_int=42,inner_string=42}}#Struct{inner_int=-42,inner_string=-42}/confluent_compat_topic/KAFKA_VALUE create mode 100644 google-cloud-bigtable-kafka-connect-sink/src/test/resources/compatibility_test_cases/key_matryoshkas/data.json create mode 100644 google-cloud-bigtable-kafka-connect-sink/src/test/resources/compatibility_test_cases/key_matryoshkas/key-schema.json create mode 100644 google-cloud-bigtable-kafka-connect-sink/src/test/resources/compatibility_test_cases/key_matryoshkas/value-schema.json create mode 100644 google-cloud-bigtable-kafka-connect-sink/src/test/resources/compatibility_test_cases/key_nestedlogicals/confluent_sink_output/Struct{inner=Struct{logical_timestamp=Thu Jan 01 00:00:04 UTC 1970,logical_time=Thu Jan 01 00:00:04 UTC 1970,logical_date=Thu Feb 12 00:00:00 UTC 1970,logical_decimal=1.089357896855742840E+68}}/confluent_compat_topic/KAFKA_VALUE create mode 100644 google-cloud-bigtable-kafka-connect-sink/src/test/resources/compatibility_test_cases/key_nestedlogicals/data.json create mode 100644 google-cloud-bigtable-kafka-connect-sink/src/test/resources/compatibility_test_cases/key_nestedlogicals/key-schema.json create mode 100644 google-cloud-bigtable-kafka-connect-sink/src/test/resources/compatibility_test_cases/key_nestedlogicals/value-schema.json create mode 100644 google-cloud-bigtable-kafka-connect-sink/src/test/resources/compatibility_test_cases/key_primitives/confluent_sink_output/1#false#4242#42424242424242#42.42#4.242424242424242E7#forty-two/confluent_compat_topic/KAFKA_VALUE create mode 100644 google-cloud-bigtable-kafka-connect-sink/src/test/resources/compatibility_test_cases/key_primitives/data.json create mode 100644 google-cloud-bigtable-kafka-connect-sink/src/test/resources/compatibility_test_cases/key_primitives/key-schema.json create mode 100644 google-cloud-bigtable-kafka-connect-sink/src/test/resources/compatibility_test_cases/key_primitives/value-schema.json create mode 100644 google-cloud-bigtable-kafka-connect-sink/src/test/resources/compatibility_test_cases/value_containers/confluent_sink_output/containers1/confluent_compat_topic/array create mode 100644 google-cloud-bigtable-kafka-connect-sink/src/test/resources/compatibility_test_cases/value_containers/confluent_sink_output/containers1/confluent_compat_topic/map create mode 100644 google-cloud-bigtable-kafka-connect-sink/src/test/resources/compatibility_test_cases/value_containers/confluent_sink_output/containers1/struct/integer create mode 100644 google-cloud-bigtable-kafka-connect-sink/src/test/resources/compatibility_test_cases/value_containers/data.json create mode 100644 google-cloud-bigtable-kafka-connect-sink/src/test/resources/compatibility_test_cases/value_containers/key-schema.json create mode 100644 google-cloud-bigtable-kafka-connect-sink/src/test/resources/compatibility_test_cases/value_containers/value-schema.json create mode 100644 google-cloud-bigtable-kafka-connect-sink/src/test/resources/compatibility_test_cases/value_matryoshkas/confluent_sink_output/matryoshka-dolls1/confluent_compat_topic/row create mode 100644 google-cloud-bigtable-kafka-connect-sink/src/test/resources/compatibility_test_cases/value_matryoshkas/confluent_sink_output/matryoshka-dolls1/inner/inner_int create mode 100644 google-cloud-bigtable-kafka-connect-sink/src/test/resources/compatibility_test_cases/value_matryoshkas/confluent_sink_output/matryoshka-dolls1/inner/inner_string create mode 100644 google-cloud-bigtable-kafka-connect-sink/src/test/resources/compatibility_test_cases/value_matryoshkas/confluent_sink_output/matryoshka-dolls1/middle/inner create mode 100644 google-cloud-bigtable-kafka-connect-sink/src/test/resources/compatibility_test_cases/value_matryoshkas/confluent_sink_output/matryoshka-dolls1/middle/middle_array create mode 100644 google-cloud-bigtable-kafka-connect-sink/src/test/resources/compatibility_test_cases/value_matryoshkas/data.json create mode 100644 google-cloud-bigtable-kafka-connect-sink/src/test/resources/compatibility_test_cases/value_matryoshkas/key-schema.json create mode 100644 google-cloud-bigtable-kafka-connect-sink/src/test/resources/compatibility_test_cases/value_matryoshkas/value-schema.json create mode 100644 google-cloud-bigtable-kafka-connect-sink/src/test/resources/compatibility_test_cases/value_nestedlogicals/confluent_sink_output/nestedlogicals1/outer/inner create mode 100644 google-cloud-bigtable-kafka-connect-sink/src/test/resources/compatibility_test_cases/value_nestedlogicals/data.json create mode 100644 google-cloud-bigtable-kafka-connect-sink/src/test/resources/compatibility_test_cases/value_nestedlogicals/key-schema.json create mode 100644 google-cloud-bigtable-kafka-connect-sink/src/test/resources/compatibility_test_cases/value_nestedlogicals/value-schema.json create mode 100644 google-cloud-bigtable-kafka-connect-sink/src/test/resources/compatibility_test_cases/value_nulls/confluent_sink_output/nulls1/confluent_compat_topic/f1 create mode 100644 google-cloud-bigtable-kafka-connect-sink/src/test/resources/compatibility_test_cases/value_nulls/confluent_sink_output/nulls1/confluent_compat_topic/f2 create mode 100644 google-cloud-bigtable-kafka-connect-sink/src/test/resources/compatibility_test_cases/value_nulls/confluent_sink_output/nulls1/confluent_compat_topic/f3 create mode 100644 google-cloud-bigtable-kafka-connect-sink/src/test/resources/compatibility_test_cases/value_nulls/confluent_sink_output/nulls1/confluent_compat_topic/f4 create mode 100644 google-cloud-bigtable-kafka-connect-sink/src/test/resources/compatibility_test_cases/value_nulls/confluent_sink_output/nulls1/confluent_compat_topic/row create mode 100644 google-cloud-bigtable-kafka-connect-sink/src/test/resources/compatibility_test_cases/value_nulls/confluent_sink_output/nulls2/confluent_compat_topic/f1 create mode 100644 google-cloud-bigtable-kafka-connect-sink/src/test/resources/compatibility_test_cases/value_nulls/confluent_sink_output/nulls2/confluent_compat_topic/f2 create mode 100644 google-cloud-bigtable-kafka-connect-sink/src/test/resources/compatibility_test_cases/value_nulls/confluent_sink_output/nulls2/confluent_compat_topic/f3 create mode 100644 google-cloud-bigtable-kafka-connect-sink/src/test/resources/compatibility_test_cases/value_nulls/confluent_sink_output/nulls2/confluent_compat_topic/f4 create mode 100644 google-cloud-bigtable-kafka-connect-sink/src/test/resources/compatibility_test_cases/value_nulls/confluent_sink_output/nulls2/confluent_compat_topic/row create mode 100644 google-cloud-bigtable-kafka-connect-sink/src/test/resources/compatibility_test_cases/value_nulls/confluent_sink_output/nulls3/confluent_compat_topic/f1 create mode 100644 google-cloud-bigtable-kafka-connect-sink/src/test/resources/compatibility_test_cases/value_nulls/confluent_sink_output/nulls3/confluent_compat_topic/f2 create mode 100644 google-cloud-bigtable-kafka-connect-sink/src/test/resources/compatibility_test_cases/value_nulls/confluent_sink_output/nulls3/confluent_compat_topic/f3 create mode 100644 google-cloud-bigtable-kafka-connect-sink/src/test/resources/compatibility_test_cases/value_nulls/confluent_sink_output/nulls3/confluent_compat_topic/f4 create mode 100644 google-cloud-bigtable-kafka-connect-sink/src/test/resources/compatibility_test_cases/value_nulls/confluent_sink_output/nulls3/confluent_compat_topic/row create mode 100644 google-cloud-bigtable-kafka-connect-sink/src/test/resources/compatibility_test_cases/value_nulls/confluent_sink_output/nulls4/confluent_compat_topic/f1 create mode 100644 google-cloud-bigtable-kafka-connect-sink/src/test/resources/compatibility_test_cases/value_nulls/confluent_sink_output/nulls4/confluent_compat_topic/f2 create mode 100644 google-cloud-bigtable-kafka-connect-sink/src/test/resources/compatibility_test_cases/value_nulls/confluent_sink_output/nulls4/confluent_compat_topic/f3 create mode 100644 google-cloud-bigtable-kafka-connect-sink/src/test/resources/compatibility_test_cases/value_nulls/confluent_sink_output/nulls4/confluent_compat_topic/f4 create mode 100644 google-cloud-bigtable-kafka-connect-sink/src/test/resources/compatibility_test_cases/value_nulls/confluent_sink_output/nulls4/confluent_compat_topic/row create mode 100644 google-cloud-bigtable-kafka-connect-sink/src/test/resources/compatibility_test_cases/value_nulls/data.json create mode 100644 google-cloud-bigtable-kafka-connect-sink/src/test/resources/compatibility_test_cases/value_nulls/key-schema.json create mode 100644 google-cloud-bigtable-kafka-connect-sink/src/test/resources/compatibility_test_cases/value_nulls/value-schema.json create mode 100644 google-cloud-bigtable-kafka-connect-sink/src/test/resources/compatibility_test_cases/value_primitives/confluent_sink_output/primitives1/confluent_compat_topic/boolean_prim create mode 100644 google-cloud-bigtable-kafka-connect-sink/src/test/resources/compatibility_test_cases/value_primitives/confluent_sink_output/primitives1/confluent_compat_topic/double_prim create mode 100644 google-cloud-bigtable-kafka-connect-sink/src/test/resources/compatibility_test_cases/value_primitives/confluent_sink_output/primitives1/confluent_compat_topic/float_prim create mode 100644 google-cloud-bigtable-kafka-connect-sink/src/test/resources/compatibility_test_cases/value_primitives/confluent_sink_output/primitives1/confluent_compat_topic/int_prim create mode 100644 google-cloud-bigtable-kafka-connect-sink/src/test/resources/compatibility_test_cases/value_primitives/confluent_sink_output/primitives1/confluent_compat_topic/long_prim create mode 100644 google-cloud-bigtable-kafka-connect-sink/src/test/resources/compatibility_test_cases/value_primitives/confluent_sink_output/primitives1/confluent_compat_topic/null_prim create mode 100644 google-cloud-bigtable-kafka-connect-sink/src/test/resources/compatibility_test_cases/value_primitives/confluent_sink_output/primitives1/confluent_compat_topic/row create mode 100644 google-cloud-bigtable-kafka-connect-sink/src/test/resources/compatibility_test_cases/value_primitives/confluent_sink_output/primitives1/confluent_compat_topic/string_prim create mode 100644 google-cloud-bigtable-kafka-connect-sink/src/test/resources/compatibility_test_cases/value_primitives/data.json create mode 100644 google-cloud-bigtable-kafka-connect-sink/src/test/resources/compatibility_test_cases/value_primitives/key-schema.json create mode 100644 google-cloud-bigtable-kafka-connect-sink/src/test/resources/compatibility_test_cases/value_primitives/value-schema.json diff --git a/google-cloud-bigtable-kafka-connect-sink/src/test/java/com/google/cloud/kafka/connect/bigtable/integration/BaseIT.java b/google-cloud-bigtable-kafka-connect-sink/src/test/java/com/google/cloud/kafka/connect/bigtable/integration/BaseIT.java index aeec5a6503..6e5481ade5 100644 --- a/google-cloud-bigtable-kafka-connect-sink/src/test/java/com/google/cloud/kafka/connect/bigtable/integration/BaseIT.java +++ b/google-cloud-bigtable-kafka-connect-sink/src/test/java/com/google/cloud/kafka/connect/bigtable/integration/BaseIT.java @@ -22,25 +22,25 @@ import static org.apache.kafka.connect.runtime.WorkerConfig.KEY_CONVERTER_CLASS_CONFIG; import static org.apache.kafka.connect.runtime.WorkerConfig.VALUE_CONVERTER_CLASS_CONFIG; +import com.google.cloud.bigtable.admin.v2.BigtableTableAdminClient; import com.google.cloud.bigtable.data.v2.BigtableDataClient; import com.google.cloud.bigtable.data.v2.models.Query; import com.google.cloud.bigtable.data.v2.models.Row; import com.google.cloud.kafka.connect.bigtable.BigtableSinkConnector; import com.google.cloud.kafka.connect.bigtable.config.BigtableSinkConfig; import com.google.cloud.kafka.connect.bigtable.util.TestId; +import com.google.protobuf.ByteString; import java.util.HashMap; -import java.util.List; import java.util.Map; import java.util.Properties; import java.util.stream.Collectors; import org.apache.kafka.clients.admin.Admin; import org.apache.kafka.common.utils.Utils; +import org.apache.kafka.connect.runtime.SinkConnectorConfig; import org.apache.kafka.connect.runtime.WorkerConfig; import org.apache.kafka.connect.runtime.isolation.PluginDiscoveryMode; import org.apache.kafka.connect.storage.StringConverter; import org.apache.kafka.connect.util.clusters.EmbeddedConnectCluster; -import org.junit.After; -import org.junit.Before; import org.slf4j.Logger; import org.slf4j.LoggerFactory; @@ -51,16 +51,7 @@ public abstract class BaseIT { protected int numWorkers = 1; protected int numBrokers = 1; protected int numTasks = 1; - - @Before - public void setUp() { - startConnect(); - } - - @After - public void tearDown() { - stopConnect(); - } + protected int maxKafkaMessageSizeBytes = 10 * 1024 * 1024; protected void startConnect() { logger.info("Starting embedded Kafka Connect cluster..."); @@ -69,7 +60,7 @@ protected void startConnect() { workerProps.put(WorkerConfig.PLUGIN_DISCOVERY_CONFIG, PluginDiscoveryMode.HYBRID_WARN.name()); Properties brokerProps = new Properties(); - brokerProps.put("message.max.bytes", 10 * 1024 * 1024); + brokerProps.put("message.max.bytes", maxKafkaMessageSizeBytes); brokerProps.put("auto.create.topics.enable", "false"); brokerProps.put("delete.topic.enable", "true"); connect = @@ -122,14 +113,17 @@ protected Map baseConnectorProps() { return result; } - protected BigtableDataClient getBigtableDataClient() { - BigtableSinkConfig config = new BigtableSinkConfig(baseConnectorProps()); - return config.getBigtableDataClient(); + protected BigtableDataClient getBigtableDataClient(Map configProps) { + return new BigtableSinkConfig(configProps).getBigtableDataClient(); } - protected List readAllRows(BigtableDataClient bigtable, String table) { + protected BigtableTableAdminClient getBigtableAdminClient(Map configProps) { + return new BigtableSinkConfig(configProps).getBigtableAdminClient(); + } + + protected Map readAllRows(BigtableDataClient bigtable, String table) { Query query = Query.create(table); - return bigtable.readRows(query).stream().collect(Collectors.toList()); + return bigtable.readRows(query).stream().collect(Collectors.toMap(Row::getKey, r -> r)); } protected String getTestClassId() { @@ -139,4 +133,16 @@ protected String getTestClassId() { protected String getTestCaseId() { return TestId.getTestCaseId(this.getClass()); } + + protected String startSingleTopicConnector(Map configProps) + throws InterruptedException { + String id = getTestCaseId() + System.currentTimeMillis(); + configProps.put(SinkConnectorConfig.TOPICS_CONFIG, id); + connect.kafka().createTopic(id, numBrokers); + connect.configureConnector(id, configProps); + connect + .assertions() + .assertConnectorAndAtLeastNumTasksAreRunning(id, numTasks, "Connector start timeout"); + return id; + } } diff --git a/google-cloud-bigtable-kafka-connect-sink/src/test/java/com/google/cloud/kafka/connect/bigtable/integration/BaseKafkaConnectBigtableIT.java b/google-cloud-bigtable-kafka-connect-sink/src/test/java/com/google/cloud/kafka/connect/bigtable/integration/BaseKafkaConnectBigtableIT.java new file mode 100644 index 0000000000..2a64e0b8ff --- /dev/null +++ b/google-cloud-bigtable-kafka-connect-sink/src/test/java/com/google/cloud/kafka/connect/bigtable/integration/BaseKafkaConnectBigtableIT.java @@ -0,0 +1,55 @@ +/* + * Copyright 2025 Google LLC + * + * Licensed under the Apache License, Version 2.0 (the "License"); + * you may not use this file except in compliance with the License. + * You may obtain a copy of the License at + * + * https://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ +package com.google.cloud.kafka.connect.bigtable.integration; + +import static org.apache.kafka.test.TestUtils.waitForCondition; + +import com.google.cloud.bigtable.admin.v2.BigtableTableAdminClient; +import com.google.cloud.bigtable.data.v2.BigtableDataClient; +import java.util.Map; +import org.junit.After; +import org.junit.Before; + +public abstract class BaseKafkaConnectBigtableIT extends BaseKafkaConnectIT { + public static long CONSUME_TIMEOUT_MILLIS = 30000; + protected BigtableDataClient bigtableData; + protected BigtableTableAdminClient bigtableAdmin; + + @Before + public void setUpBigtable() { + Map props = baseConnectorProps(); + bigtableData = getBigtableDataClient(props); + bigtableAdmin = getBigtableAdminClient(props); + } + + @After + public void tearDownBigtable() { + if (bigtableData != null) { + bigtableData.close(); + } + if (bigtableAdmin != null) { + bigtableAdmin.close(); + } + } + + public void waitUntilBigtableContainsNumberOfRows(String tableId, long numberOfRows) + throws InterruptedException { + waitForCondition( + () -> readAllRows(bigtableData, tableId).size() == numberOfRows, + CONSUME_TIMEOUT_MILLIS, + "Records not consumed in time."); + } +} diff --git a/google-cloud-bigtable-kafka-connect-sink/src/test/java/com/google/cloud/kafka/connect/bigtable/integration/BaseKafkaConnectBigtableSchemaRegistryIT.java b/google-cloud-bigtable-kafka-connect-sink/src/test/java/com/google/cloud/kafka/connect/bigtable/integration/BaseKafkaConnectBigtableSchemaRegistryIT.java new file mode 100644 index 0000000000..b688c3e512 --- /dev/null +++ b/google-cloud-bigtable-kafka-connect-sink/src/test/java/com/google/cloud/kafka/connect/bigtable/integration/BaseKafkaConnectBigtableSchemaRegistryIT.java @@ -0,0 +1,37 @@ +/* + * Copyright 2025 Google LLC + * + * Licensed under the Apache License, Version 2.0 (the "License"); + * you may not use this file except in compliance with the License. + * You may obtain a copy of the License at + * + * https://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ +package com.google.cloud.kafka.connect.bigtable.integration; + +import com.google.cloud.kafka.connect.bigtable.util.SchemaRegistryTestUtil; +import org.junit.After; +import org.junit.Before; + +public abstract class BaseKafkaConnectBigtableSchemaRegistryIT extends BaseKafkaConnectBigtableIT { + protected SchemaRegistryTestUtil schemaRegistry; + + @Before + public void setUpSchemaRegistry() throws Exception { + schemaRegistry = new SchemaRegistryTestUtil(connect.kafka().bootstrapServers()); + schemaRegistry.start(); + } + + @After + public void tearDownSchemaRegistry() throws Exception { + if (schemaRegistry != null) { + schemaRegistry.stop(); + } + } +} diff --git a/google-cloud-bigtable-kafka-connect-sink/src/test/java/com/google/cloud/kafka/connect/bigtable/integration/BaseKafkaConnectIT.java b/google-cloud-bigtable-kafka-connect-sink/src/test/java/com/google/cloud/kafka/connect/bigtable/integration/BaseKafkaConnectIT.java new file mode 100644 index 0000000000..533a686f8a --- /dev/null +++ b/google-cloud-bigtable-kafka-connect-sink/src/test/java/com/google/cloud/kafka/connect/bigtable/integration/BaseKafkaConnectIT.java @@ -0,0 +1,84 @@ +/* + * Copyright 2025 Google LLC + * + * Licensed under the Apache License, Version 2.0 (the "License"); + * you may not use this file except in compliance with the License. + * You may obtain a copy of the License at + * + * https://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ +package com.google.cloud.kafka.connect.bigtable.integration; + +import java.util.ArrayList; +import java.util.Collection; +import java.util.HashMap; +import java.util.List; +import java.util.Map; +import java.util.concurrent.Future; +import java.util.concurrent.TimeUnit; +import org.apache.kafka.clients.producer.KafkaProducer; +import org.apache.kafka.clients.producer.ProducerConfig; +import org.apache.kafka.clients.producer.ProducerRecord; +import org.apache.kafka.clients.producer.RecordMetadata; +import org.apache.kafka.common.serialization.ByteArraySerializer; +import org.apache.kafka.connect.data.SchemaAndValue; +import org.apache.kafka.connect.storage.Converter; +import org.junit.After; +import org.junit.Before; + +public abstract class BaseKafkaConnectIT extends BaseIT { + private static final long PRODUCE_TIMEOUT_MILLIS = 15000L; + + @Before + public void setUpConnect() { + startConnect(); + } + + @After + public void tearDownConnect() { + stopConnect(); + } + + public KafkaProducer getKafkaProducer() { + Map producerProps = new HashMap<>(); + producerProps.put(ProducerConfig.BOOTSTRAP_SERVERS_CONFIG, connect.kafka().bootstrapServers()); + producerProps.put(ProducerConfig.MAX_REQUEST_SIZE_CONFIG, maxKafkaMessageSizeBytes); + return new KafkaProducer<>(producerProps, new ByteArraySerializer(), new ByteArraySerializer()); + } + + public void sendRecords( + String topic, + Collection> keysAndValues, + Converter keyConverter, + Converter valueConverter) { + try (KafkaProducer producer = getKafkaProducer()) { + List> produceFutures = new ArrayList<>(); + for (Map.Entry keyAndValue : keysAndValues) { + SchemaAndValue key = keyAndValue.getKey(); + SchemaAndValue value = keyAndValue.getValue(); + byte[] serializedKey = keyConverter.fromConnectData(topic, key.schema(), key.value()); + byte[] serializedValue = + valueConverter.fromConnectData(topic, value.schema(), value.value()); + ProducerRecord msg = + new ProducerRecord<>(topic, serializedKey, serializedValue); + Future produceFuture = producer.send(msg); + produceFutures.add(produceFuture); + } + produceFutures.parallelStream() + .forEach( + f -> { + try { + f.get(PRODUCE_TIMEOUT_MILLIS, TimeUnit.MILLISECONDS); + } catch (Exception e) { + throw new RuntimeException(e); + } + }); + } + } +} diff --git a/google-cloud-bigtable-kafka-connect-sink/src/test/java/com/google/cloud/kafka/connect/bigtable/integration/BasicIT.java b/google-cloud-bigtable-kafka-connect-sink/src/test/java/com/google/cloud/kafka/connect/bigtable/integration/BasicIT.java index 0fd454f5d6..fbf77c6bd3 100644 --- a/google-cloud-bigtable-kafka-connect-sink/src/test/java/com/google/cloud/kafka/connect/bigtable/integration/BasicIT.java +++ b/google-cloud-bigtable-kafka-connect-sink/src/test/java/com/google/cloud/kafka/connect/bigtable/integration/BasicIT.java @@ -15,67 +15,41 @@ */ package com.google.cloud.kafka.connect.bigtable.integration; -import static org.apache.kafka.test.TestUtils.waitForCondition; +import static org.junit.Assert.assertArrayEquals; +import static org.junit.Assert.assertEquals; -import com.google.cloud.bigtable.data.v2.BigtableDataClient; import com.google.cloud.bigtable.data.v2.models.Row; import com.google.cloud.bigtable.data.v2.models.RowCell; import com.google.cloud.kafka.connect.bigtable.config.BigtableSinkConfig; -import com.google.cloud.kafka.connect.bigtable.config.InsertMode; import com.google.protobuf.ByteString; import java.nio.charset.StandardCharsets; import java.util.List; import java.util.Map; -import org.apache.kafka.connect.runtime.SinkConnectorConfig; -import org.apache.kafka.test.TestCondition; import org.junit.Test; import org.junit.runner.RunWith; import org.junit.runners.JUnit4; @RunWith(JUnit4.class) -public class BasicIT extends BaseIT { - private static final int TASK_CONSUME_TIMEOUT_MS = 10000; - +public class BasicIT extends BaseKafkaConnectBigtableIT { @Test public void testSimpleWrite() throws InterruptedException { - BigtableDataClient bigtable = getBigtableDataClient(); - String topic = getTestCaseId(); - String connectorName = "connector-" + topic; - connect.kafka().createTopic(topic, numTasks); Map props = baseConnectorProps(); - props.put(SinkConnectorConfig.TOPICS_CONFIG, topic); props.put(BigtableSinkConfig.CONFIG_AUTO_CREATE_TABLES, "true"); props.put(BigtableSinkConfig.CONFIG_AUTO_CREATE_COLUMN_FAMILIES, "true"); - props.put(BigtableSinkConfig.CONFIG_INSERT_MODE, InsertMode.UPSERT.name()); - connect.configureConnector(connectorName, props); - connect - .assertions() - .assertConnectorAndAtLeastNumTasksAreRunning( - connectorName, numTasks, "Connector start timeout"); + String topic = startSingleTopicConnector(props); - int numberOfRecords = 1; String key = "key"; + ByteString keyBytes = ByteString.copyFrom(key.getBytes(StandardCharsets.UTF_8)); String value = "value"; connect.kafka().produce(topic, key, value); + waitUntilBigtableContainsNumberOfRows(topic, 1); - TestCondition testCondition = - () -> { - List allRows = readAllRows(bigtable, topic); - if (numberOfRecords != allRows.size()) { - return false; - } - Row row = allRows.get(0); - if (!ByteString.copyFrom(key.getBytes(StandardCharsets.UTF_8)).equals(row.getKey())) { - return false; - } - List rowCells = row.getCells("default", "KAFKA_VALUE"); - if (numberOfRecords != rowCells.size()) { - return false; - } - return ByteString.copyFrom(value.getBytes(StandardCharsets.UTF_8)) - .equals(rowCells.get(0).getValue()); - }; - waitForCondition( - testCondition, TASK_CONSUME_TIMEOUT_MS, "Correct results not received in time"); + Map allRows = readAllRows(bigtableData, topic); + assertEquals(1, allRows.size()); + Row row = allRows.get(keyBytes); + List rowCells = row.getCells(topic, "KAFKA_VALUE"); + assertEquals(1, rowCells.size()); + assertArrayEquals( + value.getBytes(StandardCharsets.UTF_8), rowCells.get(0).getValue().toByteArray()); } } diff --git a/google-cloud-bigtable-kafka-connect-sink/src/test/java/com/google/cloud/kafka/connect/bigtable/integration/BigtableClientIT.java b/google-cloud-bigtable-kafka-connect-sink/src/test/java/com/google/cloud/kafka/connect/bigtable/integration/BigtableClientIT.java new file mode 100644 index 0000000000..9b0a8a2669 --- /dev/null +++ b/google-cloud-bigtable-kafka-connect-sink/src/test/java/com/google/cloud/kafka/connect/bigtable/integration/BigtableClientIT.java @@ -0,0 +1,45 @@ +/* + * Copyright 2025 Google LLC + * + * Licensed under the Apache License, Version 2.0 (the "License"); + * you may not use this file except in compliance with the License. + * You may obtain a copy of the License at + * + * https://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ +package com.google.cloud.kafka.connect.bigtable.integration; + +import static org.junit.Assert.assertFalse; +import static org.junit.Assert.assertTrue; + +import com.google.cloud.bigtable.admin.v2.BigtableTableAdminClient; +import com.google.cloud.bigtable.admin.v2.models.CreateTableRequest; +import com.google.cloud.kafka.connect.bigtable.config.BigtableSinkConfig; +import java.util.Map; +import org.junit.Test; +import org.junit.runner.RunWith; +import org.junit.runners.JUnit4; + +@RunWith(JUnit4.class) +public class BigtableClientIT extends BaseIT { + @Test + public void testClient() { + Map props = baseConnectorProps(); + BigtableSinkConfig config = new BigtableSinkConfig(props); + + BigtableTableAdminClient admin = config.getBigtableAdminClient(); + String tableId = getTestCaseId() + System.currentTimeMillis(); + String columnFamily = "columnFamily"; + + CreateTableRequest createTableRequest = CreateTableRequest.of(tableId).addFamily(columnFamily); + assertFalse(admin.listTables().contains(tableId)); + admin.createTable(createTableRequest); + assertTrue(admin.listTables().contains(tableId)); + } +} diff --git a/google-cloud-bigtable-kafka-connect-sink/src/test/java/com/google/cloud/kafka/connect/bigtable/integration/ConfigIT.java b/google-cloud-bigtable-kafka-connect-sink/src/test/java/com/google/cloud/kafka/connect/bigtable/integration/ConfigIT.java index ca616fa490..45306295f5 100644 --- a/google-cloud-bigtable-kafka-connect-sink/src/test/java/com/google/cloud/kafka/connect/bigtable/integration/ConfigIT.java +++ b/google-cloud-bigtable-kafka-connect-sink/src/test/java/com/google/cloud/kafka/connect/bigtable/integration/ConfigIT.java @@ -23,7 +23,7 @@ import org.junit.runners.JUnit4; @RunWith(JUnit4.class) -public class ConfigIT extends BaseIT { +public class ConfigIT extends BaseKafkaConnectIT { @Test public void testBaseSuccess() throws InterruptedException { String topic = getTestCaseId(); diff --git a/google-cloud-bigtable-kafka-connect-sink/src/test/java/com/google/cloud/kafka/connect/bigtable/integration/ConfluentCompatibilityIT.java b/google-cloud-bigtable-kafka-connect-sink/src/test/java/com/google/cloud/kafka/connect/bigtable/integration/ConfluentCompatibilityIT.java new file mode 100644 index 0000000000..8c5f87ff0b --- /dev/null +++ b/google-cloud-bigtable-kafka-connect-sink/src/test/java/com/google/cloud/kafka/connect/bigtable/integration/ConfluentCompatibilityIT.java @@ -0,0 +1,263 @@ +/* + * Copyright 2025 Google LLC + * + * Licensed under the Apache License, Version 2.0 (the "License"); + * you may not use this file except in compliance with the License. + * You may obtain a copy of the License at + * + * https://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ +package com.google.cloud.kafka.connect.bigtable.integration; + +/* + * This software contains code derived from the BigQuery Connector for Apache Kafka, + * Copyright Aiven Oy, which in turn contains code derived from the Confluent BigQuery + * Kafka Connector, Copyright Confluent, Inc, which in turn contains code derived from + * the WePay BigQuery Kafka Connector, Copyright WePay, Inc. + */ + +import static org.junit.Assert.assertEquals; +import static org.junit.Assert.assertFalse; +import static org.junit.Assert.assertTrue; + +import com.google.cloud.bigtable.data.v2.models.Row; +import com.google.cloud.bigtable.data.v2.models.RowCell; +import com.google.cloud.kafka.connect.bigtable.config.BigtableSinkConfig; +import com.google.protobuf.ByteString; +import io.confluent.connect.avro.AvroConverter; +import io.confluent.kafka.formatter.AvroMessageReader; +import io.confluent.kafka.serializers.AbstractKafkaSchemaSerDeConfig; +import java.io.IOException; +import java.io.InputStream; +import java.net.URISyntaxException; +import java.nio.charset.StandardCharsets; +import java.nio.file.Files; +import java.nio.file.Path; +import java.nio.file.Paths; +import java.util.AbstractMap; +import java.util.Arrays; +import java.util.Collection; +import java.util.HashMap; +import java.util.List; +import java.util.Map; +import java.util.Properties; +import java.util.Set; +import java.util.concurrent.ExecutionException; +import java.util.concurrent.TimeUnit; +import java.util.concurrent.TimeoutException; +import java.util.stream.Collectors; +import kafka.common.MessageReader; +import org.apache.kafka.clients.producer.Producer; +import org.apache.kafka.clients.producer.ProducerRecord; +import org.apache.kafka.connect.runtime.ConnectorConfig; +import org.junit.Test; +import org.junit.jupiter.api.AfterEach; +import org.junit.jupiter.api.BeforeEach; +import org.junit.runner.RunWith; +import org.junit.runners.Parameterized; + +@RunWith(Parameterized.class) +public class ConfluentCompatibilityIT extends BaseKafkaConnectBigtableSchemaRegistryIT { + public static String TEST_CASES_DIR = "compatibility_test_cases"; + // Needed since the default column family is the topic's name in the default configuration. + public static String COMPATIBILITY_TEST_TOPIC = "confluent_compat_topic"; + + private String testCase; + private Compatibility compatibility; + + @BeforeEach + public void setUp() throws Exception { + setUpConnect(); + setUpBigtable(); + setUpSchemaRegistry(); + } + + @AfterEach + public void tearDown() throws Exception { + tearDownSchemaRegistry(); + tearDownBigtable(); + tearDownConnect(); + } + + public ConfluentCompatibilityIT(String testCase, Compatibility compatibility) { + this.testCase = testCase; + this.compatibility = compatibility; + } + + @Parameterized.Parameters + public static Collection testCases() { + return Arrays.asList( + new Object[][] { + {"key_containers", Compatibility.FULL}, + // We serialize `Date`-based logical types differently. For details, see the comments + // in KeyMapper, especially the ones in serializeKeyElement(). + {"key_logicals", Compatibility.KEY_MISMATCH}, + {"key_matryoshkas", Compatibility.FULL}, + // We serialize `Date`-based logical types differently. For details, see the comments + // in KeyMapper, especially the ones in serializeKeyElement(). + {"key_nestedlogicals", Compatibility.KEY_MISMATCH}, + {"key_primitives", Compatibility.FULL}, + {"value_containers", Compatibility.FULL}, + {"value_matryoshkas", Compatibility.FULL}, + {"value_nestedlogicals", Compatibility.FULL}, + {"value_nulls", Compatibility.FULL}, + {"value_primitives", Compatibility.FULL}, + }); + } + + @Test + public void testCasesUsingSchemaRegistry() + throws InterruptedException, URISyntaxException, IOException { + String testId = startConnector(); + populateTopic(testId); + Map, ByteString>> expected = getExpectedOutput(); + waitUntilBigtableContainsNumberOfRows(testId, expected.size()); + Map allRows = readAllRows(bigtableData, testId); + switch (compatibility) { + case FULL: + assertRowsMatch(expected, allRows.values()); + break; + case KEY_MISMATCH: + assertEquals(expected.size(), allRows.size()); + assertTrue(allRows.values().stream().allMatch(r -> r.getCells().size() == 1)); + Set allValues = + allRows.values().stream() + .map(r -> r.getCells().get(0).getValue()) + .collect(Collectors.toSet()); + Set allExpectedValues = + expected.values().stream() + .flatMap(m -> m.values().stream()) + .collect(Collectors.toSet()); + assertEquals(allExpectedValues, allValues); + break; + } + connect + .assertions() + .assertConnectorAndExactlyNumTasksAreRunning(testId, numTasks, "Some task failed."); + } + + public String startConnector() throws InterruptedException { + Map connectorProps = baseConnectorProps(); + connectorProps.put(ConnectorConfig.KEY_CONVERTER_CLASS_CONFIG, AvroConverter.class.getName()); + connectorProps.put( + ConnectorConfig.KEY_CONVERTER_CLASS_CONFIG + + "." + + AbstractKafkaSchemaSerDeConfig.SCHEMA_REGISTRY_URL_CONFIG, + schemaRegistry.schemaRegistryUrl()); + connectorProps.put(ConnectorConfig.VALUE_CONVERTER_CLASS_CONFIG, AvroConverter.class.getName()); + connectorProps.put( + ConnectorConfig.VALUE_CONVERTER_CLASS_CONFIG + + "." + + AbstractKafkaSchemaSerDeConfig.SCHEMA_REGISTRY_URL_CONFIG, + schemaRegistry.schemaRegistryUrl()); + connectorProps.put(BigtableSinkConfig.CONFIG_AUTO_CREATE_TABLES, "true"); + connectorProps.put(BigtableSinkConfig.CONFIG_AUTO_CREATE_COLUMN_FAMILIES, "true"); + connectorProps.put(BigtableSinkConfig.CONFIG_DEFAULT_COLUMN_FAMILY, COMPATIBILITY_TEST_TOPIC); + connectorProps.put(BigtableSinkConfig.CONFIG_ROW_KEY_DELIMITER, "#"); + String topic = startSingleTopicConnector(connectorProps); + connect + .assertions() + .assertConnectorAndAtLeastNumTasksAreRunning(topic, numTasks, "Connector start timeout"); + return topic; + } + + public void populateTopic(String topic) throws IOException { + String keySchema = readStringResource(getTestCaseDir() + "/key-schema.json"); + String valueSchema = readStringResource(getTestCaseDir() + "/value-schema.json"); + Properties messageReaderProps = new Properties(); + messageReaderProps.put( + AbstractKafkaSchemaSerDeConfig.SCHEMA_REGISTRY_URL_CONFIG, + schemaRegistry.schemaRegistryUrl()); + messageReaderProps.put("topic", topic); + messageReaderProps.put("parse.key", "true"); + messageReaderProps.put("key.schema", keySchema); + messageReaderProps.put("value.schema", valueSchema); + InputStream dataStream = getClassLoader().getResourceAsStream(getTestCaseDir() + "/data.json"); + MessageReader messageReader = new AvroMessageReader(); + messageReader.init(dataStream, messageReaderProps); + + Producer kafkaProducer = getKafkaProducer(); + ProducerRecord message = messageReader.readMessage(); + while (message != null) { + try { + kafkaProducer.send(message).get(1, TimeUnit.SECONDS); + } catch (InterruptedException | ExecutionException | TimeoutException e) { + throw new RuntimeException(e); + } + message = messageReader.readMessage(); + } + } + + private static String readStringResource(String resourceName) throws IOException { + byte[] resourceBytes = getClassLoader().getResourceAsStream(resourceName).readAllBytes(); + return new String(resourceBytes, StandardCharsets.UTF_8); + } + + private static ClassLoader getClassLoader() { + return ConfluentCompatibilityIT.class.getClassLoader(); + } + + private String getTestCaseDir() { + return String.format("%s/%s", TEST_CASES_DIR, testCase); + } + + private Map, ByteString>> getExpectedOutput() + throws URISyntaxException, IOException { + Map, ByteString>> result = new HashMap<>(); + + String expectedOutputDir = getTestCaseDir() + "/confluent_sink_output"; + List outputPaths = + Files.find( + Paths.get(getClassLoader().getResource(expectedOutputDir).toURI()), + 3, + (path, attr) -> attr.isRegularFile()) + .collect(Collectors.toList()); + + for (Path outputPath : outputPaths) { + String[] outputPathParts = outputPath.toString().split("/"); + int outputPathPartsLength = outputPathParts.length; + String row = outputPathParts[outputPathPartsLength - 3]; + String columnFamily = outputPathParts[outputPathPartsLength - 2]; + String columnQualifier = outputPathParts[outputPathPartsLength - 1]; + byte[] value = Files.readAllBytes(outputPath); + + result.putIfAbsent(row, new HashMap<>()); + Map.Entry familyAndQualifier = + new AbstractMap.SimpleImmutableEntry<>(columnFamily, columnQualifier); + assertFalse(result.get(row).containsKey(familyAndQualifier)); + result.get(row).put(familyAndQualifier, ByteString.copyFrom(value)); + } + + return result; + } + + private void assertRowsMatch( + Map, ByteString>> expected, + Collection bigtableRows) { + assertEquals(expected.size(), bigtableRows.size()); + for (Row row : bigtableRows) { + String key = new String(row.getKey().toByteArray(), StandardCharsets.UTF_8); + Map, ByteString> bigtableRow = new HashMap<>(); + for (RowCell cell : row.getCells()) { + Map.Entry familyAndQualifier = + new AbstractMap.SimpleImmutableEntry<>( + cell.getFamily(), + new String(cell.getQualifier().toByteArray(), StandardCharsets.UTF_8)); + assertFalse(bigtableRow.containsKey(familyAndQualifier)); + bigtableRow.put(familyAndQualifier, cell.getValue()); + } + assertEquals(expected.get(key), bigtableRow); + } + } + + public enum Compatibility { + FULL, + KEY_MISMATCH, + } +} diff --git a/google-cloud-bigtable-kafka-connect-sink/src/test/java/com/google/cloud/kafka/connect/bigtable/integration/DifferentConvertersIT.java b/google-cloud-bigtable-kafka-connect-sink/src/test/java/com/google/cloud/kafka/connect/bigtable/integration/DifferentConvertersIT.java new file mode 100644 index 0000000000..3233c4acf3 --- /dev/null +++ b/google-cloud-bigtable-kafka-connect-sink/src/test/java/com/google/cloud/kafka/connect/bigtable/integration/DifferentConvertersIT.java @@ -0,0 +1,245 @@ +/* + * Copyright 2025 Google LLC + * + * Licensed under the Apache License, Version 2.0 (the "License"); + * you may not use this file except in compliance with the License. + * You may obtain a copy of the License at + * + * https://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ +package com.google.cloud.kafka.connect.bigtable.integration; + +/* + * This software contains code derived from the BigQuery Connector for Apache Kafka, + * Copyright Aiven Oy, which in turn contains code derived from the Confluent BigQuery + * Kafka Connector, Copyright Confluent, Inc, which in turn contains code derived from + * the WePay BigQuery Kafka Connector, Copyright WePay, Inc. + */ + +import com.google.cloud.kafka.connect.bigtable.config.BigtableSinkConfig; +import io.confluent.connect.json.JsonSchemaConverter; +import io.confluent.kafka.serializers.AbstractKafkaSchemaSerDeConfig; +import java.math.BigDecimal; +import java.util.AbstractMap; +import java.util.ArrayList; +import java.util.List; +import java.util.Map; +import java.util.function.Supplier; +import java.util.stream.Collectors; +import java.util.stream.LongStream; +import org.apache.kafka.connect.data.Date; +import org.apache.kafka.connect.data.Decimal; +import org.apache.kafka.connect.data.Schema; +import org.apache.kafka.connect.data.SchemaAndValue; +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.json.JsonConverter; +import org.apache.kafka.connect.json.JsonConverterConfig; +import org.apache.kafka.connect.runtime.ConnectorConfig; +import org.apache.kafka.connect.storage.Converter; +import org.junit.Test; +import org.junit.runner.RunWith; +import org.junit.runners.JUnit4; + +@RunWith(JUnit4.class) +public class DifferentConvertersIT extends BaseKafkaConnectBigtableSchemaRegistryIT { + private static final Schema SUB_STRUCT_SCHEMA = + SchemaBuilder.struct() + .field("ssf1", Schema.INT64_SCHEMA) + .field("ssf2", Schema.BOOLEAN_SCHEMA) + .build(); + private static final Schema NESTED_STRUCT_SCHEMA = + SchemaBuilder.struct() + .field("sf1", Schema.STRING_SCHEMA) + .field("sf2", SUB_STRUCT_SCHEMA) + .field("sf3", Schema.FLOAT64_SCHEMA) + .build(); + private static final Schema PRIMITIVES_SCHEMA = + SchemaBuilder.struct() + .field("boolean_field", Schema.BOOLEAN_SCHEMA) + .field("float32_field", Schema.FLOAT32_SCHEMA) + .field("float64_field", Schema.FLOAT64_SCHEMA) + .field("int8_field", Schema.INT8_SCHEMA) + .field("int16_field", Schema.INT16_SCHEMA) + .field("int32_field", Schema.INT32_SCHEMA) + .field("int64_field", Schema.INT64_SCHEMA) + .field("string_field", Schema.STRING_SCHEMA); + private static final Schema LOGICALS_SCHEMA = + SchemaBuilder.struct() + // klf = "Kafka logical field" + .field("klf1", Timestamp.builder().optional().build()) + .field("klf2", Time.builder().optional().build()) + .field("klf3", Date.builder().optional().build()) + .field("klf4", Decimal.builder(5).optional().build()) + .build(); + private static final Schema ARRAY_SCHEMA = SchemaBuilder.array(Schema.STRING_SCHEMA); + private static final Schema KEY_SCHEMA = SchemaBuilder.INT64_SCHEMA; + private static final Schema VALUE_SCHEMA = + SchemaBuilder.struct() + .optional() + .field("f1", Schema.STRING_SCHEMA) + .field("f2", Schema.BOOLEAN_SCHEMA) + .field("f3", Schema.FLOAT64_SCHEMA) + .field("bytes_field", Schema.OPTIONAL_BYTES_SCHEMA) + .field("nested_field", NESTED_STRUCT_SCHEMA) + .field("primitives_field", PRIMITIVES_SCHEMA) + .field("logicals_field", LOGICALS_SCHEMA) + .field("array_field", ARRAY_SCHEMA) + .build(); + private static final Long NUM_RECORDS = 100L; + + @Test + public void testAvro() throws InterruptedException { + testConverter( + JsonSchemaConverter::new, + Map.of( + AbstractKafkaSchemaSerDeConfig.SCHEMA_REGISTRY_URL_CONFIG, + schemaRegistry.schemaRegistryUrl())); + } + + @Test + public void testProtobuf() throws InterruptedException { + testConverter( + JsonSchemaConverter::new, + Map.of( + AbstractKafkaSchemaSerDeConfig.SCHEMA_REGISTRY_URL_CONFIG, + schemaRegistry.schemaRegistryUrl())); + } + + @Test + public void testSchemaJson() throws InterruptedException { + testConverter( + JsonSchemaConverter::new, + Map.of( + AbstractKafkaSchemaSerDeConfig.SCHEMA_REGISTRY_URL_CONFIG, + schemaRegistry.schemaRegistryUrl())); + } + + @Test + public void testJsonWithInBandSchema() throws InterruptedException { + testConverter( + JsonConverter::new, + Map.of(JsonConverterConfig.SCHEMAS_ENABLE_CONFIG, String.valueOf(true))); + } + + @Test + public void testJsonWithoutSchema() throws InterruptedException { + testConverter( + JsonConverter::new, + Map.of(JsonConverterConfig.SCHEMAS_ENABLE_CONFIG, String.valueOf(false))); + } + + private void testConverter( + Supplier converterConstructor, Map converterProps) + throws InterruptedException { + Converter keyConverter = converterConstructor.get(); + keyConverter.configure(converterProps, true); + Converter valueConverter = converterConstructor.get(); + valueConverter.configure(converterProps, false); + + Map connectorProps = connectorProps(); + for (Map.Entry prop : converterProps.entrySet()) { + connectorProps.put( + ConnectorConfig.KEY_CONVERTER_CLASS_CONFIG + "." + prop.getKey(), prop.getValue()); + connectorProps.put( + ConnectorConfig.VALUE_CONVERTER_CLASS_CONFIG + "." + prop.getKey(), prop.getValue()); + } + connectorProps.put( + ConnectorConfig.KEY_CONVERTER_CLASS_CONFIG, keyConverter.getClass().getName()); + connectorProps.put( + ConnectorConfig.VALUE_CONVERTER_CLASS_CONFIG, valueConverter.getClass().getName()); + String topic = startSingleTopicConnector(connectorProps); + connect + .assertions() + .assertConnectorAndAtLeastNumTasksAreRunning(topic, numTasks, "Connector start timeout"); + populateKafkaTopic(topic, NUM_RECORDS, keyConverter, valueConverter); + + waitUntilBigtableContainsNumberOfRows(topic, NUM_RECORDS); + } + + protected Map connectorProps() { + Map props = baseConnectorProps(); + props.put(BigtableSinkConfig.CONFIG_AUTO_CREATE_TABLES, "true"); + props.put(BigtableSinkConfig.CONFIG_AUTO_CREATE_COLUMN_FAMILIES, "true"); + return props; + } + + private void populateKafkaTopic( + String topic, long numRecords, Converter keyConverter, Converter valueConverter) { + List> records = new ArrayList<>(); + for (long i = 0; i < numRecords; i++) { + Object key = i; + Object value = getValue(i); + records.add( + new AbstractMap.SimpleImmutableEntry<>( + new SchemaAndValue(KEY_SCHEMA, key), new SchemaAndValue(VALUE_SCHEMA, value))); + } + sendRecords(topic, records, keyConverter, valueConverter); + } + + private Struct getValue(long iteration) { + Struct primitivesStruct = new Struct(VALUE_SCHEMA.field("primitives_field").schema()); + primitivesStruct.put("boolean_field", iteration % 3 == 1); + primitivesStruct.put("float32_field", iteration * 1.5f); + primitivesStruct.put("float64_field", iteration * 0.5); + primitivesStruct.put("int8_field", (byte) (iteration % 10)); + primitivesStruct.put("int16_field", (short) (iteration % 30 + 1)); + primitivesStruct.put("int32_field", (int) (-1 * (iteration % 100))); + primitivesStruct.put("int64_field", iteration * 10); + primitivesStruct.put("string_field", Long.toString(iteration * 123)); + + Struct logicalsStruct = new Struct(VALUE_SCHEMA.field("logicals_field").schema()); + long timestampMs = 1707835187396L; + int msPerDay = 86400000; + int time = (int) (timestampMs % msPerDay); + int date = (int) (timestampMs / msPerDay); + Schema klf1Schema = logicalsStruct.schema().field("klf1").schema(); + java.util.Date klf1Value = + org.apache.kafka.connect.data.Timestamp.toLogical(klf1Schema, timestampMs); + Schema klf2Schema = logicalsStruct.schema().field("klf2").schema(); + java.util.Date klf2Value = org.apache.kafka.connect.data.Time.toLogical(klf2Schema, time); + Schema klf3Schema = logicalsStruct.schema().field("klf3").schema(); + java.util.Date klf3Value = org.apache.kafka.connect.data.Date.toLogical(klf3Schema, date); + logicalsStruct + .put("klf1", klf1Value) + .put("klf2", klf2Value) + .put("klf3", klf3Value) + .put("klf4", BigDecimal.valueOf(6543).setScale(5)); + + Struct subStruct = + new Struct(VALUE_SCHEMA.field("nested_field").schema().field("sf2").schema()); + subStruct.put("ssf1", iteration / 2); + subStruct.put("ssf2", false); + + Struct nestedStruct = new Struct(VALUE_SCHEMA.field("nested_field").schema()); + nestedStruct.put("sf1", "sv1"); + nestedStruct.put("sf2", subStruct); + nestedStruct.put("sf3", iteration * 1.0); + + List arrayValue = + LongStream.of(iteration % 10) + .mapToObj(l -> "array element " + l) + .collect(Collectors.toList()); + + byte[] bytesValue = new byte[(int) iteration % 4]; + for (int i = 0; i < bytesValue.length; i++) bytesValue[i] = (byte) i; + + return new Struct(VALUE_SCHEMA) + .put("f1", "api" + iteration) + .put("f2", iteration % 2 == 0) + .put("f3", iteration * 0.01) + .put("bytes_field", bytesValue) + .put("nested_field", nestedStruct) + .put("primitives_field", primitivesStruct) + .put("logicals_field", logicalsStruct) + .put("array_field", arrayValue); + } +} diff --git a/google-cloud-bigtable-kafka-connect-sink/src/test/java/com/google/cloud/kafka/connect/bigtable/integration/ErrorHandlingIT.java b/google-cloud-bigtable-kafka-connect-sink/src/test/java/com/google/cloud/kafka/connect/bigtable/integration/ErrorHandlingIT.java new file mode 100644 index 0000000000..67c9f1afa7 --- /dev/null +++ b/google-cloud-bigtable-kafka-connect-sink/src/test/java/com/google/cloud/kafka/connect/bigtable/integration/ErrorHandlingIT.java @@ -0,0 +1,37 @@ +/* + * Copyright 2025 Google LLC + * + * Licensed under the Apache License, Version 2.0 (the "License"); + * you may not use this file except in compliance with the License. + * You may obtain a copy of the License at + * + * https://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ +package com.google.cloud.kafka.connect.bigtable.integration; + +import static org.junit.Assert.assertThrows; + +import com.google.cloud.kafka.connect.bigtable.config.BigtableSinkConfig; +import java.util.Map; +import org.junit.Test; +import org.junit.runner.RunWith; +import org.junit.runners.JUnit4; + +@RunWith(JUnit4.class) +public class ErrorHandlingIT extends BaseKafkaConnectBigtableIT { + @Test + public void testBigtableCredentialsAreCheckedOnStartup() throws InterruptedException { + Map props = baseConnectorProps(); + props.put(BigtableSinkConfig.CONFIG_GCP_CREDENTIALS_JSON, "{}"); + + String testId = getTestCaseId(); + assertThrows(Throwable.class, () -> connect.configureConnector(testId, props)); + assertThrows(Throwable.class, () -> connect.connectorStatus(testId)); + } +} diff --git a/google-cloud-bigtable-kafka-connect-sink/src/test/java/com/google/cloud/kafka/connect/bigtable/integration/ErrorReportingIT.java b/google-cloud-bigtable-kafka-connect-sink/src/test/java/com/google/cloud/kafka/connect/bigtable/integration/ErrorReportingIT.java new file mode 100644 index 0000000000..4e1838ae98 --- /dev/null +++ b/google-cloud-bigtable-kafka-connect-sink/src/test/java/com/google/cloud/kafka/connect/bigtable/integration/ErrorReportingIT.java @@ -0,0 +1,107 @@ +/* + * Copyright 2025 Google LLC + * + * Licensed under the Apache License, Version 2.0 (the "License"); + * you may not use this file except in compliance with the License. + * You may obtain a copy of the License at + * + * https://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ +package com.google.cloud.kafka.connect.bigtable.integration; + +import static org.junit.Assert.assertArrayEquals; +import static org.junit.Assert.assertEquals; +import static org.junit.Assert.assertTrue; + +import com.google.cloud.kafka.connect.bigtable.config.BigtableErrorMode; +import com.google.cloud.kafka.connect.bigtable.config.BigtableSinkConfig; +import java.nio.charset.StandardCharsets; +import java.time.Duration; +import java.util.Arrays; +import java.util.Map; +import org.apache.kafka.clients.consumer.ConsumerRecord; +import org.apache.kafka.clients.consumer.ConsumerRecords; +import org.apache.kafka.connect.runtime.SinkConnectorConfig; +import org.apache.kafka.connect.runtime.errors.ToleranceType; +import org.junit.Test; +import org.junit.runner.RunWith; +import org.junit.runners.JUnit4; + +@RunWith(JUnit4.class) +public class ErrorReportingIT extends BaseKafkaConnectIT { + @Test + public void testErrorModeFail() throws InterruptedException { + Map props = baseConnectorProps(); + props.put(BigtableSinkConfig.CONFIG_ERROR_MODE, BigtableErrorMode.FAIL.name()); + + String testId = startSingleTopicConnector(props); + connect.kafka().produce(testId, "key", "value"); + connect + .assertions() + .assertConnectorIsRunningAndTasksHaveFailed( + testId, 1, "Task didn't fail despite configured error mode."); + } + + @Test + public void testErrorModeWarn() throws InterruptedException { + Map props = baseConnectorProps(); + props.put(BigtableSinkConfig.CONFIG_ERROR_MODE, BigtableErrorMode.WARN.name()); + + String testId = startSingleTopicConnector(props); + connect.kafka().produce(testId, "key", "value"); + connect + .assertions() + .assertConnectorAndExactlyNumTasksAreRunning( + testId, numTasks, "Task failed despite configured error mode."); + } + + @Test + public void testErrorModeIgnore() throws InterruptedException { + Map props = baseConnectorProps(); + props.put(BigtableSinkConfig.CONFIG_ERROR_MODE, BigtableErrorMode.IGNORE.name()); + + String testId = startSingleTopicConnector(props); + connect.kafka().produce(testId, "key", "value"); + connect + .assertions() + .assertConnectorAndExactlyNumTasksAreRunning( + testId, numTasks, "Task failed despite configured error mode."); + } + + @Test + public void testErrorModeDLQOverridesErrorMode() throws InterruptedException { + String dlqTopic = getTestCaseId() + System.currentTimeMillis(); + connect.kafka().createTopic(dlqTopic, numBrokers); + + Map props = baseConnectorProps(); + props.put(BigtableSinkConfig.CONFIG_ERROR_MODE, BigtableErrorMode.FAIL.name()); + props.put(SinkConnectorConfig.DLQ_TOPIC_NAME_CONFIG, dlqTopic); + props.put(SinkConnectorConfig.DLQ_CONTEXT_HEADERS_ENABLE_CONFIG, String.valueOf(true)); + props.put(SinkConnectorConfig.DLQ_TOPIC_REPLICATION_FACTOR_CONFIG, String.valueOf(numBrokers)); + props.put(SinkConnectorConfig.ERRORS_TOLERANCE_CONFIG, ToleranceType.ALL.value()); + + String key = "key"; + String value = "value"; + String testId = startSingleTopicConnector(props); + connect.kafka().produce(testId, key, value); + ConsumerRecords dlqRecords = + connect.kafka().consume(1, Duration.ofSeconds(120).toMillis(), dlqTopic); + assertEquals(1, dlqRecords.count()); + ConsumerRecord record = dlqRecords.iterator().next(); + assertArrayEquals(record.key(), key.getBytes(StandardCharsets.UTF_8)); + assertArrayEquals(record.key(), key.getBytes(StandardCharsets.UTF_8)); + assertTrue( + Arrays.stream(record.headers().toArray()) + .anyMatch(h -> h.key().equals("__connect.errors.exception.class" + ".name"))); + connect + .assertions() + .assertConnectorAndExactlyNumTasksAreRunning( + testId, numTasks, "Wrong number of tasks is running."); + } +} diff --git a/google-cloud-bigtable-kafka-connect-sink/src/test/java/com/google/cloud/kafka/connect/bigtable/integration/InsertUpsertIT.java b/google-cloud-bigtable-kafka-connect-sink/src/test/java/com/google/cloud/kafka/connect/bigtable/integration/InsertUpsertIT.java new file mode 100644 index 0000000000..cb0a5960d5 --- /dev/null +++ b/google-cloud-bigtable-kafka-connect-sink/src/test/java/com/google/cloud/kafka/connect/bigtable/integration/InsertUpsertIT.java @@ -0,0 +1,86 @@ +/* + * Copyright 2025 Google LLC + * + * Licensed under the Apache License, Version 2.0 (the "License"); + * you may not use this file except in compliance with the License. + * You may obtain a copy of the License at + * + * https://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ +package com.google.cloud.kafka.connect.bigtable.integration; + +import static org.junit.Assert.assertEquals; + +import com.google.cloud.bigtable.data.v2.models.Row; +import com.google.cloud.kafka.connect.bigtable.config.BigtableErrorMode; +import com.google.cloud.kafka.connect.bigtable.config.BigtableSinkConfig; +import com.google.cloud.kafka.connect.bigtable.config.InsertMode; +import com.google.protobuf.ByteString; +import java.nio.charset.StandardCharsets; +import java.util.Map; +import org.junit.Test; +import org.junit.runner.RunWith; +import org.junit.runners.JUnit4; + +@RunWith(JUnit4.class) +public class InsertUpsertIT extends BaseKafkaConnectBigtableIT { + private static final String KEY1 = "key1"; + private static final String KEY2 = "key2"; + private static final ByteString KEY1_BYTES = + ByteString.copyFrom(KEY1.getBytes(StandardCharsets.UTF_8)); + private static final ByteString KEY2_BYTES = + ByteString.copyFrom(KEY2.getBytes(StandardCharsets.UTF_8)); + private static final String VALUE1 = "value1"; + private static final String VALUE2 = "value2"; + private static final String VALUE3 = "value3"; + + @Test + public void testInsert() throws InterruptedException { + Map props = baseConnectorProps(); + props.put(BigtableSinkConfig.CONFIG_AUTO_CREATE_TABLES, "true"); + props.put(BigtableSinkConfig.CONFIG_AUTO_CREATE_COLUMN_FAMILIES, "true"); + props.put(BigtableSinkConfig.CONFIG_INSERT_MODE, InsertMode.INSERT.name()); + props.put(BigtableSinkConfig.CONFIG_ERROR_MODE, BigtableErrorMode.IGNORE.name()); + String testId = startSingleTopicConnector(props); + + connect.kafka().produce(testId, KEY1, VALUE1); + connect.kafka().produce(testId, KEY1, VALUE2); + connect.kafka().produce(testId, KEY2, VALUE3); + + waitUntilBigtableContainsNumberOfRows(testId, 2); + + Map rows = readAllRows(bigtableData, testId); + Row row1 = rows.get(KEY1_BYTES); + Row row2 = rows.get(KEY2_BYTES); + assertEquals(1, row1.getCells().size()); + assertEquals(1, row2.getCells().size()); + } + + @Test + public void testUpsert() throws InterruptedException { + Map props = baseConnectorProps(); + props.put(BigtableSinkConfig.CONFIG_AUTO_CREATE_TABLES, "true"); + props.put(BigtableSinkConfig.CONFIG_AUTO_CREATE_COLUMN_FAMILIES, "true"); + props.put(BigtableSinkConfig.CONFIG_INSERT_MODE, InsertMode.UPSERT.name()); + props.put(BigtableSinkConfig.CONFIG_ERROR_MODE, BigtableErrorMode.IGNORE.name()); + String testId = startSingleTopicConnector(props); + + connect.kafka().produce(testId, KEY1, VALUE1); + connect.kafka().produce(testId, KEY1, VALUE2); + connect.kafka().produce(testId, KEY2, VALUE3); + + waitUntilBigtableContainsNumberOfRows(testId, 2); + + Map rows = readAllRows(bigtableData, testId); + Row row1 = rows.get(KEY1_BYTES); + Row row2 = rows.get(KEY2_BYTES); + assertEquals(2, row1.getCells().size()); + assertEquals(1, row2.getCells().size()); + } +} diff --git a/google-cloud-bigtable-kafka-connect-sink/src/test/java/com/google/cloud/kafka/connect/bigtable/integration/NullHandlingIT.java b/google-cloud-bigtable-kafka-connect-sink/src/test/java/com/google/cloud/kafka/connect/bigtable/integration/NullHandlingIT.java new file mode 100644 index 0000000000..e4507de1db --- /dev/null +++ b/google-cloud-bigtable-kafka-connect-sink/src/test/java/com/google/cloud/kafka/connect/bigtable/integration/NullHandlingIT.java @@ -0,0 +1,354 @@ +/* + * Copyright 2025 Google LLC + * + * Licensed under the Apache License, Version 2.0 (the "License"); + * you may not use this file except in compliance with the License. + * You may obtain a copy of the License at + * + * https://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ +package com.google.cloud.kafka.connect.bigtable.integration; + +import static com.google.cloud.kafka.connect.bigtable.util.NestedNullStructFactory.NESTED_NULL_STRUCT_FIELD_NAME; +import static com.google.cloud.kafka.connect.bigtable.util.NestedNullStructFactory.NESTED_NULL_STRUCT_FIELD_NAME_BYTES; +import static com.google.cloud.kafka.connect.bigtable.util.NestedNullStructFactory.getStructhWithNullOnNthNestingLevel; +import static org.junit.Assert.assertEquals; +import static org.junit.Assert.assertTrue; + +import com.google.cloud.bigtable.admin.v2.models.CreateTableRequest; +import com.google.cloud.bigtable.data.v2.models.Row; +import com.google.cloud.bigtable.data.v2.models.RowCell; +import com.google.cloud.bigtable.data.v2.models.RowMutation; +import com.google.cloud.kafka.connect.bigtable.config.BigtableSinkConfig; +import com.google.cloud.kafka.connect.bigtable.config.InsertMode; +import com.google.cloud.kafka.connect.bigtable.config.NullValueMode; +import com.google.protobuf.ByteString; +import java.nio.charset.StandardCharsets; +import java.util.AbstractMap; +import java.util.ArrayList; +import java.util.HashSet; +import java.util.List; +import java.util.Map; +import java.util.Set; +import java.util.stream.Collectors; +import org.apache.kafka.connect.data.Schema; +import org.apache.kafka.connect.data.SchemaAndValue; +import org.apache.kafka.connect.data.SchemaBuilder; +import org.apache.kafka.connect.data.Struct; +import org.apache.kafka.connect.json.JsonConverter; +import org.apache.kafka.connect.json.JsonConverterConfig; +import org.apache.kafka.connect.runtime.ConnectorConfig; +import org.apache.kafka.connect.storage.StringConverter; +import org.junit.Test; +import org.junit.runner.RunWith; +import org.junit.runners.JUnit4; + +@RunWith(JUnit4.class) +public class NullHandlingIT extends BaseKafkaConnectBigtableIT { + private static final Map JSON_CONVERTER_PROPS = + Map.of(JsonConverterConfig.SCHEMAS_ENABLE_CONFIG, String.valueOf(true)); + private static final StringConverter KEY_CONVERTER = new StringConverter(); + private static final JsonConverter VALUE_CONVERTER = new JsonConverter(); + + private static final String KEY1 = "key1"; + private static final String KEY2 = "key2"; + private static final String KEY3 = "key3"; + private static final String KEY4 = "key4"; + private static final ByteString KEY1_BYTES = + ByteString.copyFrom(KEY1.getBytes(StandardCharsets.UTF_8)); + private static final ByteString KEY2_BYTES = + ByteString.copyFrom(KEY2.getBytes(StandardCharsets.UTF_8)); + private static final ByteString KEY3_BYTES = + ByteString.copyFrom(KEY3.getBytes(StandardCharsets.UTF_8)); + private static final ByteString KEY4_BYTES = + ByteString.copyFrom(KEY4.getBytes(StandardCharsets.UTF_8)); + + static { + VALUE_CONVERTER.configure(JSON_CONVERTER_PROPS, false); + } + + @Test + public void testIgnoreMode() throws InterruptedException { + Map connectorProps = connectorProps(); + connectorProps.put(BigtableSinkConfig.CONFIG_VALUE_NULL_MODE, NullValueMode.IGNORE.name()); + String testId = startSingleTopicConnector(connectorProps); + connect + .assertions() + .assertConnectorAndAtLeastNumTasksAreRunning(testId, numTasks, "Connector start timeout"); + + String keyFinish = "key_finish"; + ByteString keyFinishBytes = ByteString.copyFrom(keyFinish.getBytes(StandardCharsets.UTF_8)); + + List> records = new ArrayList<>(); + records.add( + new AbstractMap.SimpleImmutableEntry<>( + new SchemaAndValue(Schema.STRING_SCHEMA, KEY1), + new SchemaAndValue(SchemaBuilder.struct().optional().build(), null))); + Struct nested1 = getStructhWithNullOnNthNestingLevel(1); + records.add( + new AbstractMap.SimpleImmutableEntry<>( + new SchemaAndValue(Schema.STRING_SCHEMA, KEY2), + new SchemaAndValue(nested1.schema(), nested1))); + Struct nested2 = getStructhWithNullOnNthNestingLevel(2); + records.add( + new AbstractMap.SimpleImmutableEntry<>( + new SchemaAndValue(Schema.STRING_SCHEMA, KEY3), + new SchemaAndValue(nested2.schema(), nested2))); + Struct nested3 = getStructhWithNullOnNthNestingLevel(3); + records.add( + new AbstractMap.SimpleImmutableEntry<>( + new SchemaAndValue(Schema.STRING_SCHEMA, KEY4), + new SchemaAndValue(nested3.schema(), nested3))); + records.add( + new AbstractMap.SimpleImmutableEntry<>( + new SchemaAndValue(Schema.STRING_SCHEMA, keyFinish), + new SchemaAndValue(Schema.STRING_SCHEMA, "finish"))); + sendRecords(testId, records, KEY_CONVERTER, VALUE_CONVERTER); + + waitUntilBigtableContainsNumberOfRows(testId, 2); + Map rows = readAllRows(bigtableData, testId); + assertEquals(rows.keySet(), Set.of(KEY4_BYTES, keyFinishBytes)); + + assertEquals(1, rows.get(KEY4_BYTES).getCells().size()); + RowCell cell = rows.get(KEY4_BYTES).getCells().get(0); + assertCellContents( + cell, + NESTED_NULL_STRUCT_FIELD_NAME, + NESTED_NULL_STRUCT_FIELD_NAME_BYTES, + jsonifiedStructWithNullField()); + } + + @Test + public void testWriteMode() throws InterruptedException { + String defaultColumnFamily = "family"; + String defaultColumnQualifier = "qualifier"; + ByteString defaultColumnQualifierBytes = + ByteString.copyFrom(defaultColumnQualifier.getBytes(StandardCharsets.UTF_8)); + + Map connectorProps = connectorProps(); + connectorProps.put(BigtableSinkConfig.CONFIG_VALUE_NULL_MODE, NullValueMode.WRITE.name()); + connectorProps.put(BigtableSinkConfig.CONFIG_DEFAULT_COLUMN_FAMILY, defaultColumnFamily); + connectorProps.put(BigtableSinkConfig.CONFIG_DEFAULT_COLUMN_QUALIFIER, defaultColumnQualifier); + String testId = startSingleTopicConnector(connectorProps); + connect + .assertions() + .assertConnectorAndAtLeastNumTasksAreRunning(testId, numTasks, "Connector start timeout"); + + List> records = new ArrayList<>(); + records.add( + new AbstractMap.SimpleImmutableEntry<>( + new SchemaAndValue(Schema.STRING_SCHEMA, KEY1), + new SchemaAndValue(SchemaBuilder.struct().optional().build(), null))); + Struct nested1 = getStructhWithNullOnNthNestingLevel(1); + records.add( + new AbstractMap.SimpleImmutableEntry<>( + new SchemaAndValue(Schema.STRING_SCHEMA, KEY2), + new SchemaAndValue(nested1.schema(), nested1))); + Struct nested2 = getStructhWithNullOnNthNestingLevel(2); + records.add( + new AbstractMap.SimpleImmutableEntry<>( + new SchemaAndValue(Schema.STRING_SCHEMA, KEY3), + new SchemaAndValue(nested2.schema(), nested2))); + Struct nested3 = getStructhWithNullOnNthNestingLevel(3); + records.add( + new AbstractMap.SimpleImmutableEntry<>( + new SchemaAndValue(Schema.STRING_SCHEMA, KEY4), + new SchemaAndValue(nested3.schema(), nested3))); + sendRecords(testId, records, KEY_CONVERTER, VALUE_CONVERTER); + + waitUntilBigtableContainsNumberOfRows(testId, 4); + Map rows = readAllRows(bigtableData, testId); + assertEquals(rows.keySet(), Set.of(KEY1_BYTES, KEY2_BYTES, KEY3_BYTES, KEY4_BYTES)); + assertTrue(rows.values().stream().allMatch(vs -> vs.getCells().size() == 1)); + + ByteString nullValueWriteMode = ByteString.copyFrom(new byte[0]); + assertCellContents( + rows.get(KEY1_BYTES).getCells().get(0), + defaultColumnFamily, + defaultColumnQualifierBytes, + nullValueWriteMode); + assertCellContents( + rows.get(KEY2_BYTES).getCells().get(0), + defaultColumnFamily, + NESTED_NULL_STRUCT_FIELD_NAME_BYTES, + nullValueWriteMode); + assertCellContents( + rows.get(KEY3_BYTES).getCells().get(0), + NESTED_NULL_STRUCT_FIELD_NAME, + NESTED_NULL_STRUCT_FIELD_NAME_BYTES, + nullValueWriteMode); + assertCellContents( + rows.get(KEY4_BYTES).getCells().get(0), + NESTED_NULL_STRUCT_FIELD_NAME, + NESTED_NULL_STRUCT_FIELD_NAME_BYTES, + jsonifiedStructWithNullField()); + } + + @Test + public void testDeleteMode() throws InterruptedException { + Map connectorProps = connectorProps(); + connectorProps.put(BigtableSinkConfig.CONFIG_VALUE_NULL_MODE, NullValueMode.DELETE.name()); + connectorProps.put(BigtableSinkConfig.CONFIG_INSERT_MODE, InsertMode.UPSERT.name()); + String testId = startSingleTopicConnector(connectorProps); + connect + .assertions() + .assertConnectorAndAtLeastNumTasksAreRunning(testId, numTasks, "Connector start timeout"); + + String columnFamily1 = "cf1"; + String columnFamily2 = "cf2"; + bigtableAdmin.createTable( + CreateTableRequest.of(testId).addFamily(columnFamily1).addFamily(columnFamily2)); + String columnQualifier1 = "cq1"; + String columnQualifier2 = "cq2"; + ByteString columnQualifierBytes1 = + ByteString.copyFrom(columnQualifier1.getBytes(StandardCharsets.UTF_8)); + ByteString columnQualifierBytes2 = + ByteString.copyFrom(columnQualifier2.getBytes(StandardCharsets.UTF_8)); + + ByteString value = ByteString.copyFromUtf8("value"); + for (String key : List.of(KEY1, KEY2, KEY3, KEY4)) { + RowMutation mutation = RowMutation.create(testId, key); + for (String family : List.of(columnFamily1, columnFamily2)) { + for (ByteString qualifier : List.of(columnQualifierBytes1, columnQualifierBytes2)) { + mutation.setCell(family, qualifier, value); + } + } + bigtableData.mutateRow(mutation); + } + Map rowsBefore = readAllRows(bigtableData, testId); + assertEquals(Set.of(KEY1_BYTES, KEY2_BYTES, KEY3_BYTES, KEY4_BYTES), rowsBefore.keySet()); + + ByteString keyAddedJsonification = KEY1_BYTES; + Struct nestedNullToBeJsonified = getStructhWithNullOnNthNestingLevel(3); + + ByteString keyDeletedColumn = KEY2_BYTES; + Map.Entry deletedColumn = + new AbstractMap.SimpleImmutableEntry<>(columnFamily2, columnQualifier2); + Schema innerDeleteColumnSchema = + SchemaBuilder.struct().field(deletedColumn.getValue(), Schema.OPTIONAL_INT8_SCHEMA); + Struct deleteColumn = + new Struct( + SchemaBuilder.struct() + .field(deletedColumn.getKey(), innerDeleteColumnSchema) + .build()) + .put( + deletedColumn.getKey(), + new Struct(innerDeleteColumnSchema).put(deletedColumn.getValue(), null)); + + ByteString keyDeletedColumnFamily = KEY3_BYTES; + String deletedColumnFamily = columnFamily1; + Struct deleteColumnFamily = + new Struct(SchemaBuilder.struct().field(deletedColumnFamily, Schema.OPTIONAL_INT8_SCHEMA)) + .put(deletedColumnFamily, null); + + ByteString keyDeletedRow = KEY4_BYTES; + Schema deleteRowSchema = SchemaBuilder.struct().optional().build(); + + List> records = new ArrayList<>(); + records.add( + new AbstractMap.SimpleImmutableEntry<>( + new SchemaAndValue( + Schema.STRING_SCHEMA, + new String(keyAddedJsonification.toByteArray(), StandardCharsets.UTF_8)), + new SchemaAndValue(nestedNullToBeJsonified.schema(), nestedNullToBeJsonified))); + records.add( + new AbstractMap.SimpleImmutableEntry<>( + new SchemaAndValue( + Schema.STRING_SCHEMA, + new String(keyDeletedColumn.toByteArray(), StandardCharsets.UTF_8)), + new SchemaAndValue(deleteColumn.schema(), deleteColumn))); + records.add( + new AbstractMap.SimpleImmutableEntry<>( + new SchemaAndValue( + Schema.STRING_SCHEMA, + new String(keyDeletedColumnFamily.toByteArray(), StandardCharsets.UTF_8)), + new SchemaAndValue(deleteColumnFamily.schema(), deleteColumnFamily))); + records.add( + new AbstractMap.SimpleImmutableEntry<>( + new SchemaAndValue( + Schema.STRING_SCHEMA, + new String(keyDeletedRow.toByteArray(), StandardCharsets.UTF_8)), + new SchemaAndValue(deleteRowSchema, null))); + + sendRecords(testId, records, KEY_CONVERTER, VALUE_CONVERTER); + waitUntilBigtableContainsNumberOfRows(testId, 3); + Map rowsAfter = readAllRows(bigtableData, testId); + assertEquals( + Set.of(keyAddedJsonification, keyDeletedColumn, keyDeletedColumnFamily), + rowsAfter.keySet()); + + Row rowAddedJsonification = rowsAfter.get(keyAddedJsonification); + assertEquals( + 1, + rowAddedJsonification + .getCells(NESTED_NULL_STRUCT_FIELD_NAME, NESTED_NULL_STRUCT_FIELD_NAME_BYTES) + .size()); + assertEquals( + jsonifiedStructWithNullField(), + rowAddedJsonification + .getCells(NESTED_NULL_STRUCT_FIELD_NAME, NESTED_NULL_STRUCT_FIELD_NAME_BYTES) + .get(0) + .getValue()); + assertEquals( + new HashSet<>(rowsBefore.get(keyAddedJsonification).getCells()), + rowAddedJsonification.getCells().stream() + .filter(c -> !c.getFamily().equals(NESTED_NULL_STRUCT_FIELD_NAME)) + .collect(Collectors.toSet())); + + Row rowDeletedColumnFamily = rowsAfter.get(keyDeletedColumnFamily); + assertEquals(0, rowDeletedColumnFamily.getCells(deletedColumnFamily).size()); + assertEquals( + rowsBefore.get(keyDeletedColumnFamily).getCells().stream() + .filter(c -> !c.getFamily().equals(deletedColumnFamily)) + .collect(Collectors.toSet()), + new HashSet<>(rowDeletedColumnFamily.getCells())); + + Row rowDeletedColumn = rowsAfter.get(keyDeletedColumn); + assertEquals( + 0, rowDeletedColumn.getCells(deletedColumn.getKey(), deletedColumn.getValue()).size()); + assertEquals( + rowsBefore.get(keyDeletedColumn).getCells().stream() + .filter( + c -> + !(c.getFamily().equals(deletedColumn.getKey()) + && c.getQualifier() + .equals( + ByteString.copyFrom( + deletedColumn.getValue().getBytes(StandardCharsets.UTF_8))))) + .collect(Collectors.toSet()), + new HashSet<>(rowDeletedColumn.getCells())); + } + + protected Map connectorProps() { + Map props = super.baseConnectorProps(); + props.put(BigtableSinkConfig.CONFIG_AUTO_CREATE_TABLES, "true"); + props.put(BigtableSinkConfig.CONFIG_AUTO_CREATE_COLUMN_FAMILIES, "true"); + // We use JsonConverter since it doesn't care about schemas, so we may use differently-shaped + // data within a single test. + props.put(ConnectorConfig.VALUE_CONVERTER_CLASS_CONFIG, JsonConverter.class.getName()); + for (Map.Entry prop : JSON_CONVERTER_PROPS.entrySet()) { + props.put( + ConnectorConfig.VALUE_CONVERTER_CLASS_CONFIG + "." + prop.getKey(), prop.getValue()); + } + return props; + } + + protected ByteString jsonifiedStructWithNullField() { + String expectedJson = String.format("{\"%s\":null}", NESTED_NULL_STRUCT_FIELD_NAME); + byte[] expectedJsonBytes = expectedJson.getBytes(StandardCharsets.UTF_8); + return ByteString.copyFrom(expectedJsonBytes); + } + + protected void assertCellContents( + RowCell cell, String family, ByteString qualifier, ByteString value) { + assertEquals(family, cell.getFamily()); + assertEquals(qualifier, cell.getQualifier()); + assertEquals(value, cell.getValue()); + } +} diff --git a/google-cloud-bigtable-kafka-connect-sink/src/test/java/com/google/cloud/kafka/connect/bigtable/integration/ResourceAutoCreationIT.java b/google-cloud-bigtable-kafka-connect-sink/src/test/java/com/google/cloud/kafka/connect/bigtable/integration/ResourceAutoCreationIT.java new file mode 100644 index 0000000000..1e34cf2fa4 --- /dev/null +++ b/google-cloud-bigtable-kafka-connect-sink/src/test/java/com/google/cloud/kafka/connect/bigtable/integration/ResourceAutoCreationIT.java @@ -0,0 +1,159 @@ +/* + * Copyright 2025 Google LLC + * + * Licensed under the Apache License, Version 2.0 (the "License"); + * you may not use this file except in compliance with the License. + * You may obtain a copy of the License at + * + * https://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ +package com.google.cloud.kafka.connect.bigtable.integration; + +import static org.apache.kafka.test.TestUtils.waitForCondition; +import static org.junit.Assert.assertArrayEquals; +import static org.junit.Assert.assertEquals; +import static org.junit.Assert.assertTrue; + +import com.google.cloud.bigtable.admin.v2.models.CreateTableRequest; +import com.google.cloud.bigtable.admin.v2.models.ModifyColumnFamiliesRequest; +import com.google.cloud.kafka.connect.bigtable.util.JsonConverterFactory; +import com.google.protobuf.ByteString; +import java.nio.charset.StandardCharsets; +import java.time.Duration; +import java.util.Arrays; +import java.util.Map; +import java.util.concurrent.ExecutionException; +import java.util.concurrent.TimeoutException; +import java.util.stream.StreamSupport; +import org.apache.kafka.clients.consumer.ConsumerRecords; +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.json.JsonConverter; +import org.apache.kafka.connect.json.JsonConverterConfig; +import org.apache.kafka.connect.runtime.ConnectorConfig; +import org.apache.kafka.connect.runtime.SinkConnectorConfig; +import org.apache.kafka.connect.runtime.errors.ToleranceType; +import org.junit.Test; +import org.junit.runner.RunWith; +import org.junit.runners.JUnit4; + +@RunWith(JUnit4.class) +public class ResourceAutoCreationIT extends BaseKafkaConnectBigtableIT { + private final JsonConverter jsonConverter = JsonConverterFactory.create(true, false); + + @Test + public void testMissingAndLaterCreatedTableAndColumnFamily() + throws InterruptedException, ExecutionException, TimeoutException { + String dlqTopic = createDlq(); + Map props = dlqAndJsonValuesProps(dlqTopic); + + String testId = startSingleTopicConnector(props); + + String key1 = "key1"; + String key2 = "key2"; + String key3 = "key3"; + String key4 = "key4"; + String columnFamily1 = "cf1"; + String columnFamily2 = "cf2"; + String columnQualifier = "cq"; + + Integer value = 1; + Struct columnValue = + new Struct(SchemaBuilder.struct().field(columnQualifier, Schema.INT32_SCHEMA)) + .put(columnQualifier, value); + + Struct value1 = + new Struct(SchemaBuilder.struct().field(columnFamily1, columnValue.schema())) + .put(columnFamily1, columnValue); + Struct value2 = + new Struct(SchemaBuilder.struct().field(columnFamily2, columnValue.schema())) + .put(columnFamily2, columnValue); + + String serializedValue1 = + new String( + jsonConverter.fromConnectData(testId, value1.schema(), value1), StandardCharsets.UTF_8); + String serializedValue2 = + new String( + jsonConverter.fromConnectData(testId, value2.schema(), value2), StandardCharsets.UTF_8); + + // With the table missing. + connect.kafka().produce(testId, key1, serializedValue1); + ConsumerRecords dlqRecords = + connect.kafka().consume(1, Duration.ofSeconds(120).toMillis(), dlqTopic); + assertEquals(1, dlqRecords.count()); + assertArrayEquals(key1.getBytes(StandardCharsets.UTF_8), dlqRecords.iterator().next().key()); + connect + .assertions() + .assertConnectorAndExactlyNumTasksAreRunning( + testId, numTasks, "Wrong number of tasks is running"); + + // With the table and column family created. + bigtableAdmin.createTable(CreateTableRequest.of(testId).addFamily(columnFamily1)); + connect.kafka().produce(testId, key2, serializedValue1); + waitForCondition( + () -> readAllRows(bigtableData, testId).size() == 1, + Duration.ofSeconds(15).toMillis(), + "Records not processed in time"); + assertTrue( + readAllRows(bigtableData, testId) + .containsKey(ByteString.copyFrom(key2.getBytes(StandardCharsets.UTF_8)))); + connect + .assertions() + .assertConnectorAndExactlyNumTasksAreRunning( + testId, numTasks, "Wrong number of tasks is running"); + + // With the column family missing. + connect.kafka().produce(testId, key3, serializedValue2); + dlqRecords = connect.kafka().consume(2, Duration.ofSeconds(120).toMillis(), dlqTopic); + assertEquals(2, dlqRecords.count()); + assertTrue( + StreamSupport.stream(dlqRecords.spliterator(), false) + .anyMatch(r -> Arrays.equals(key3.getBytes(StandardCharsets.UTF_8), r.key()))); + connect + .assertions() + .assertConnectorAndExactlyNumTasksAreRunning( + testId, numTasks, "Wrong number of tasks is running"); + // With the column family created. + bigtableAdmin.modifyFamilies(ModifyColumnFamiliesRequest.of(testId).addFamily(columnFamily2)); + connect.kafka().produce(testId, key4, serializedValue2); + waitForCondition( + () -> readAllRows(bigtableData, testId).size() == 2, + Duration.ofSeconds(15).toMillis(), + "Records not " + "processed in time"); + assertTrue( + readAllRows(bigtableData, testId) + .containsKey(ByteString.copyFrom(key4.getBytes(StandardCharsets.UTF_8)))); + connect + .assertions() + .assertConnectorAndExactlyNumTasksAreRunning( + testId, numTasks, "Wrong number of tasks is running"); + } + + private String createDlq() { + String dlqTopic = getTestCaseId() + System.currentTimeMillis(); + connect.kafka().createTopic(dlqTopic, numBrokers); + return dlqTopic; + } + + private Map dlqAndJsonValuesProps(String dlqTopic) { + Map props = baseConnectorProps(); + props.put(SinkConnectorConfig.DLQ_TOPIC_NAME_CONFIG, dlqTopic); + props.put(SinkConnectorConfig.DLQ_CONTEXT_HEADERS_ENABLE_CONFIG, String.valueOf(false)); + props.put(SinkConnectorConfig.DLQ_TOPIC_REPLICATION_FACTOR_CONFIG, String.valueOf(numBrokers)); + props.put(SinkConnectorConfig.ERRORS_TOLERANCE_CONFIG, ToleranceType.ALL.value()); + props.put(ConnectorConfig.VALUE_CONVERTER_CLASS_CONFIG, JsonConverter.class.getName()); + props.put( + ConnectorConfig.VALUE_CONVERTER_CLASS_CONFIG + + "." + + JsonConverterConfig.SCHEMAS_ENABLE_CONFIG, + String.valueOf(true)); + return props; + } +} diff --git a/google-cloud-bigtable-kafka-connect-sink/src/test/java/com/google/cloud/kafka/connect/bigtable/integration/VersionIT.java b/google-cloud-bigtable-kafka-connect-sink/src/test/java/com/google/cloud/kafka/connect/bigtable/integration/VersionIT.java index 5b51056b04..49fa599512 100644 --- a/google-cloud-bigtable-kafka-connect-sink/src/test/java/com/google/cloud/kafka/connect/bigtable/integration/VersionIT.java +++ b/google-cloud-bigtable-kafka-connect-sink/src/test/java/com/google/cloud/kafka/connect/bigtable/integration/VersionIT.java @@ -29,8 +29,11 @@ import java.util.Arrays; import org.apache.kafka.connect.runtime.rest.entities.PluginInfo; import org.junit.Test; +import org.junit.runner.RunWith; +import org.junit.runners.JUnit4; -public class VersionIT extends BaseIT { +@RunWith(JUnit4.class) +public class VersionIT extends BaseKafkaConnectIT { @Test public void testVersionIsSet() throws IOException, InterruptedException { String url = connect.endpointForResource("connector-plugins"); diff --git a/google-cloud-bigtable-kafka-connect-sink/src/test/java/com/google/cloud/kafka/connect/bigtable/mapping/ValueMapperTest.java b/google-cloud-bigtable-kafka-connect-sink/src/test/java/com/google/cloud/kafka/connect/bigtable/mapping/ValueMapperTest.java index accd38dedb..cd75fd93e4 100644 --- a/google-cloud-bigtable-kafka-connect-sink/src/test/java/com/google/cloud/kafka/connect/bigtable/mapping/ValueMapperTest.java +++ b/google-cloud-bigtable-kafka-connect-sink/src/test/java/com/google/cloud/kafka/connect/bigtable/mapping/ValueMapperTest.java @@ -267,26 +267,17 @@ public void testValueNestedOnceNeedsOnlyDefaultColumnFamily() { @Test public void testDefaultColumnFamilyInterpolation() { - for (Map.Entry test : - List.of( - new AbstractMap.SimpleImmutableEntry<>("prefix_${topic}_suffix", "prefix_topic_suffix"), - new AbstractMap.SimpleImmutableEntry<>( - "prefix_${topic_suffix", "prefix_${topic_suffix"), - new AbstractMap.SimpleImmutableEntry<>("prefix_$topic_suffix", "prefix_$topic_suffix"), - new AbstractMap.SimpleImmutableEntry<>("prefix_${bad}_suffix", "prefix_${bad}_suffix"), - new AbstractMap.SimpleImmutableEntry<>("noSubstitution", "noSubstitution"))) { - String topic = "topic"; - String value = "value"; - ValueMapper mapper = new TestValueMapper(test.getKey(), DEFAULT_COLUMN, NullValueMode.WRITE); - MutationDataBuilder mutationDataBuilder = getRecordMutationDataBuilder(mapper, value, topic); - verify(mutationDataBuilder, times(1)) - .setCell( - test.getValue(), - DEFAULT_COLUMN_BYTES, - TIMESTAMP, - ByteString.copyFrom(value.getBytes(StandardCharsets.UTF_8))); - assertTotalNumberOfInvocations(mutationDataBuilder, 1); - } + String topic = "TOPIC"; + String value = "value"; + ValueMapper mapper = new TestValueMapper("${topic}", DEFAULT_COLUMN, NullValueMode.WRITE); + MutationDataBuilder mutationDataBuilder = getRecordMutationDataBuilder(mapper, value, topic); + verify(mutationDataBuilder, times(1)) + .setCell( + topic, + DEFAULT_COLUMN_BYTES, + TIMESTAMP, + ByteString.copyFrom(value.getBytes(StandardCharsets.UTF_8))); + assertTotalNumberOfInvocations(mutationDataBuilder, 1); } @Test diff --git a/google-cloud-bigtable-kafka-connect-sink/src/test/java/com/google/cloud/kafka/connect/bigtable/util/SchemaRegistryTestUtil.java b/google-cloud-bigtable-kafka-connect-sink/src/test/java/com/google/cloud/kafka/connect/bigtable/util/SchemaRegistryTestUtil.java new file mode 100644 index 0000000000..c954de94c6 --- /dev/null +++ b/google-cloud-bigtable-kafka-connect-sink/src/test/java/com/google/cloud/kafka/connect/bigtable/util/SchemaRegistryTestUtil.java @@ -0,0 +1,84 @@ +/* + * Copyright 2025 Google LLC + * + * Licensed under the Apache License, Version 2.0 (the "License"); + * you may not use this file except in compliance with the License. + * You may obtain a copy of the License at + * + * https://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ +package com.google.cloud.kafka.connect.bigtable.util; + +/* + * This software contains code derived from the BigQuery Connector for Apache Kafka, + * Copyright Aiven Oy, which in turn contains code derived from the Confluent BigQuery + * Kafka Connector, Copyright Confluent, Inc, which in turn contains code derived from + * the WePay BigQuery Kafka Connector, Copyright WePay, Inc. + */ + +import static io.confluent.kafka.schemaregistry.ClusterTestHarness.KAFKASTORE_TOPIC; +import static java.util.Objects.requireNonNull; + +import io.confluent.kafka.schemaregistry.CompatibilityLevel; +import io.confluent.kafka.schemaregistry.RestApp; +import java.io.IOException; +import java.net.ServerSocket; +import java.util.Properties; +import org.apache.kafka.test.TestUtils; + +public class SchemaRegistryTestUtil { + private static long STARTUP_TIMEOUT_MILLIS = 10000L; + + protected String bootstrapServers; + + private String schemaRegistryUrl; + + private RestApp restApp; + + public SchemaRegistryTestUtil(String bootstrapServers) { + this.bootstrapServers = requireNonNull(bootstrapServers); + } + + public void start() throws Exception { + int port = findAvailableOpenPort(); + restApp = + new RestApp( + port, + null, + this.bootstrapServers, + KAFKASTORE_TOPIC, + CompatibilityLevel.NONE.name, + true, + new Properties()); + restApp.start(); + + TestUtils.waitForCondition( + () -> restApp.restServer.isRunning(), + STARTUP_TIMEOUT_MILLIS, + "Schema Registry start timed out."); + + schemaRegistryUrl = restApp.restServer.getURI().toString(); + } + + public void stop() throws Exception { + if (restApp != null) { + restApp.stop(); + } + } + + public String schemaRegistryUrl() { + return schemaRegistryUrl; + } + + private Integer findAvailableOpenPort() throws IOException { + try (ServerSocket socket = new ServerSocket(0)) { + return socket.getLocalPort(); + } + } +} diff --git a/google-cloud-bigtable-kafka-connect-sink/src/test/resources/compatibility_test_cases/key_containers/confluent_sink_output/{a=1, b=4, c=3, d=2}#[1, 2, 3, 4]#Struct{integer=1}/confluent_compat_topic/KAFKA_VALUE b/google-cloud-bigtable-kafka-connect-sink/src/test/resources/compatibility_test_cases/key_containers/confluent_sink_output/{a=1, b=4, c=3, d=2}#[1, 2, 3, 4]#Struct{integer=1}/confluent_compat_topic/KAFKA_VALUE new file mode 100644 index 0000000000..499149dde4 --- /dev/null +++ b/google-cloud-bigtable-kafka-connect-sink/src/test/resources/compatibility_test_cases/key_containers/confluent_sink_output/{a=1, b=4, c=3, d=2}#[1, 2, 3, 4]#Struct{integer=1}/confluent_compat_topic/KAFKA_VALUE @@ -0,0 +1 @@ +containers1 \ No newline at end of file diff --git a/google-cloud-bigtable-kafka-connect-sink/src/test/resources/compatibility_test_cases/key_containers/data.json b/google-cloud-bigtable-kafka-connect-sink/src/test/resources/compatibility_test_cases/key_containers/data.json new file mode 100644 index 0000000000..5715ff2872 --- /dev/null +++ b/google-cloud-bigtable-kafka-connect-sink/src/test/resources/compatibility_test_cases/key_containers/data.json @@ -0,0 +1 @@ +{"map":{"a": 1,"d": 2,"c": 3,"b":4},"array":[1,2,3,4],"struct":{"integer":1}} "containers1" diff --git a/google-cloud-bigtable-kafka-connect-sink/src/test/resources/compatibility_test_cases/key_containers/key-schema.json b/google-cloud-bigtable-kafka-connect-sink/src/test/resources/compatibility_test_cases/key_containers/key-schema.json new file mode 100644 index 0000000000..ba57c02377 --- /dev/null +++ b/google-cloud-bigtable-kafka-connect-sink/src/test/resources/compatibility_test_cases/key_containers/key-schema.json @@ -0,0 +1,33 @@ +{ + "type": "record", + "name": "containers", + "fields": [ + { + "name": "map", + "type": { + "type": "map", + "values": "int" + } + }, + { + "name": "array", + "type": { + "type": "array", + "items": "int" + } + }, + { + "name": "struct", + "type": { + "type": "record", + "name": "structure", + "fields": [ + { + "name": "integer", + "type": "int" + } + ] + } + } + ] +} diff --git a/google-cloud-bigtable-kafka-connect-sink/src/test/resources/compatibility_test_cases/key_containers/value-schema.json b/google-cloud-bigtable-kafka-connect-sink/src/test/resources/compatibility_test_cases/key_containers/value-schema.json new file mode 100644 index 0000000000..4db187ebb3 --- /dev/null +++ b/google-cloud-bigtable-kafka-connect-sink/src/test/resources/compatibility_test_cases/key_containers/value-schema.json @@ -0,0 +1 @@ +{"type":"string"} \ No newline at end of file diff --git a/google-cloud-bigtable-kafka-connect-sink/src/test/resources/compatibility_test_cases/key_logicals/confluent_sink_output/Mon Jul 11 22:11:42 UTC 2016#Thu Jan 01 00:00:16 UTC 1970#Mon Jul 11 00:00:00 UTC 2016#1.089357896855742840E+68/confluent_compat_topic/KAFKA_VALUE b/google-cloud-bigtable-kafka-connect-sink/src/test/resources/compatibility_test_cases/key_logicals/confluent_sink_output/Mon Jul 11 22:11:42 UTC 2016#Thu Jan 01 00:00:16 UTC 1970#Mon Jul 11 00:00:00 UTC 2016#1.089357896855742840E+68/confluent_compat_topic/KAFKA_VALUE new file mode 100644 index 0000000000..d4d78296cb --- /dev/null +++ b/google-cloud-bigtable-kafka-connect-sink/src/test/resources/compatibility_test_cases/key_logicals/confluent_sink_output/Mon Jul 11 22:11:42 UTC 2016#Thu Jan 01 00:00:16 UTC 1970#Mon Jul 11 00:00:00 UTC 2016#1.089357896855742840E+68/confluent_compat_topic/KAFKA_VALUE @@ -0,0 +1 @@ +logical3 \ No newline at end of file diff --git a/google-cloud-bigtable-kafka-connect-sink/src/test/resources/compatibility_test_cases/key_logicals/confluent_sink_output/Thu Jan 01 00:00:00 UTC 1970#Thu Jan 01 00:00:00 UTC 1970#Thu Jan 01 00:00:00 UTC 1970#2.53635900E+58/confluent_compat_topic/KAFKA_VALUE b/google-cloud-bigtable-kafka-connect-sink/src/test/resources/compatibility_test_cases/key_logicals/confluent_sink_output/Thu Jan 01 00:00:00 UTC 1970#Thu Jan 01 00:00:00 UTC 1970#Thu Jan 01 00:00:00 UTC 1970#2.53635900E+58/confluent_compat_topic/KAFKA_VALUE new file mode 100644 index 0000000000..987b96f9e7 --- /dev/null +++ b/google-cloud-bigtable-kafka-connect-sink/src/test/resources/compatibility_test_cases/key_logicals/confluent_sink_output/Thu Jan 01 00:00:00 UTC 1970#Thu Jan 01 00:00:00 UTC 1970#Thu Jan 01 00:00:00 UTC 1970#2.53635900E+58/confluent_compat_topic/KAFKA_VALUE @@ -0,0 +1 @@ +logical1 \ No newline at end of file diff --git a/google-cloud-bigtable-kafka-connect-sink/src/test/resources/compatibility_test_cases/key_logicals/confluent_sink_output/Thu Jan 01 11:40:00 UTC 1970#Thu Jan 01 00:00:04 UTC 1970#Thu Jul 02 00:00:00 UTC 1981#1.6622282361690E+63/confluent_compat_topic/KAFKA_VALUE b/google-cloud-bigtable-kafka-connect-sink/src/test/resources/compatibility_test_cases/key_logicals/confluent_sink_output/Thu Jan 01 11:40:00 UTC 1970#Thu Jan 01 00:00:04 UTC 1970#Thu Jul 02 00:00:00 UTC 1981#1.6622282361690E+63/confluent_compat_topic/KAFKA_VALUE new file mode 100644 index 0000000000..cf0c11b93f --- /dev/null +++ b/google-cloud-bigtable-kafka-connect-sink/src/test/resources/compatibility_test_cases/key_logicals/confluent_sink_output/Thu Jan 01 11:40:00 UTC 1970#Thu Jan 01 00:00:04 UTC 1970#Thu Jul 02 00:00:00 UTC 1981#1.6622282361690E+63/confluent_compat_topic/KAFKA_VALUE @@ -0,0 +1 @@ +logical2 \ No newline at end of file diff --git a/google-cloud-bigtable-kafka-connect-sink/src/test/resources/compatibility_test_cases/key_logicals/data.json b/google-cloud-bigtable-kafka-connect-sink/src/test/resources/compatibility_test_cases/key_logicals/data.json new file mode 100644 index 0000000000..ecdc90c622 --- /dev/null +++ b/google-cloud-bigtable-kafka-connect-sink/src/test/resources/compatibility_test_cases/key_logicals/data.json @@ -0,0 +1,3 @@ +{"logical_timestamp":0,"logical_time":0,"logical_date":0,"logical_decimal":"\u0000\u000f\u001e\u002d\u003c"} "logical1" +{"logical_timestamp":42000000,"logical_time":4200,"logical_date":4200,"logical_decimal":"\u0000\u000f\u001e\u002d\u003c\u004b\u005a"} "logical2" +{"logical_timestamp":1468275102000,"logical_time":16993,"logical_date":16993,"logical_decimal":"\u0000\u000f\u001e\u002d\u003c\u004b\u005a\u0069\u0078"} "logical3" diff --git a/google-cloud-bigtable-kafka-connect-sink/src/test/resources/compatibility_test_cases/key_logicals/key-schema.json b/google-cloud-bigtable-kafka-connect-sink/src/test/resources/compatibility_test_cases/key_logicals/key-schema.json new file mode 100644 index 0000000000..1e9fafaa00 --- /dev/null +++ b/google-cloud-bigtable-kafka-connect-sink/src/test/resources/compatibility_test_cases/key_logicals/key-schema.json @@ -0,0 +1,37 @@ +{ + "type": "record", + "name": "logicals", + "fields": [ + { + "name": "logical_timestamp", + "type": { + "type": "long", + "connect.name": "org.apache.kafka.connect.data.Timestamp" + } + }, + { + "name": "logical_time", + "type": { + "type": "int", + "connect.name": "org.apache.kafka.connect.data.Time" + } + }, + { + "name": "logical_date", + "type": { + "type": "int", + "connect.name": "org.apache.kafka.connect.data.Date" + } + }, + { + "name": "logical_decimal", + "type": { + "type": "bytes", + "connect.name": "org.apache.kafka.connect.data.Decimal", + "connect.parameters": { + "scale": "-50" + } + } + } + ] +} \ No newline at end of file diff --git a/google-cloud-bigtable-kafka-connect-sink/src/test/resources/compatibility_test_cases/key_logicals/value-schema.json b/google-cloud-bigtable-kafka-connect-sink/src/test/resources/compatibility_test_cases/key_logicals/value-schema.json new file mode 100644 index 0000000000..4db187ebb3 --- /dev/null +++ b/google-cloud-bigtable-kafka-connect-sink/src/test/resources/compatibility_test_cases/key_logicals/value-schema.json @@ -0,0 +1 @@ +{"type":"string"} \ No newline at end of file diff --git a/google-cloud-bigtable-kafka-connect-sink/src/test/resources/compatibility_test_cases/key_matryoshkas/confluent_sink_output/1#Struct{middle_array=[42.0, 42.42, 42.4242],inner=Struct{inner_int=42,inner_string=42}}#Struct{inner_int=-42,inner_string=-42}/confluent_compat_topic/KAFKA_VALUE b/google-cloud-bigtable-kafka-connect-sink/src/test/resources/compatibility_test_cases/key_matryoshkas/confluent_sink_output/1#Struct{middle_array=[42.0, 42.42, 42.4242],inner=Struct{inner_int=42,inner_string=42}}#Struct{inner_int=-42,inner_string=-42}/confluent_compat_topic/KAFKA_VALUE new file mode 100644 index 0000000000..98b3558661 --- /dev/null +++ b/google-cloud-bigtable-kafka-connect-sink/src/test/resources/compatibility_test_cases/key_matryoshkas/confluent_sink_output/1#Struct{middle_array=[42.0, 42.42, 42.4242],inner=Struct{inner_int=42,inner_string=42}}#Struct{inner_int=-42,inner_string=-42}/confluent_compat_topic/KAFKA_VALUE @@ -0,0 +1 @@ +matryoshka-dolls1 \ No newline at end of file diff --git a/google-cloud-bigtable-kafka-connect-sink/src/test/resources/compatibility_test_cases/key_matryoshkas/data.json b/google-cloud-bigtable-kafka-connect-sink/src/test/resources/compatibility_test_cases/key_matryoshkas/data.json new file mode 100644 index 0000000000..02fb2b76c3 --- /dev/null +++ b/google-cloud-bigtable-kafka-connect-sink/src/test/resources/compatibility_test_cases/key_matryoshkas/data.json @@ -0,0 +1 @@ +{"row":1,"middle":{"middle_array":[42.0, 42.42, 42.4242],"inner":{"inner_int":42,"inner_string":"42"}},"inner":{"inner_int":-42,"inner_string": "-42"}} "matryoshka-dolls1" diff --git a/google-cloud-bigtable-kafka-connect-sink/src/test/resources/compatibility_test_cases/key_matryoshkas/key-schema.json b/google-cloud-bigtable-kafka-connect-sink/src/test/resources/compatibility_test_cases/key_matryoshkas/key-schema.json new file mode 100644 index 0000000000..0f442ca605 --- /dev/null +++ b/google-cloud-bigtable-kafka-connect-sink/src/test/resources/compatibility_test_cases/key_matryoshkas/key-schema.json @@ -0,0 +1,47 @@ +{ + "type": "record", + "name": "outer_doll", + "fields": [ + { + "name": "row", + "type": "int" + }, + { + "name": "middle", + "type": { + "type": "record", + "name": "middle_doll", + "fields": [ + { + "name": "middle_array", + "type": { + "type": "array", + "items": "float" + } + }, + { + "name": "inner", + "type": { + "type": "record", + "name": "inner_doll", + "fields": [ + { + "name": "inner_int", + "type": "int" + }, + { + "name": "inner_string", + "type": "string" + } + ] + } + } + ] + } + }, + { + "name": "inner", + "type": "inner_doll" + } + ] +} diff --git a/google-cloud-bigtable-kafka-connect-sink/src/test/resources/compatibility_test_cases/key_matryoshkas/value-schema.json b/google-cloud-bigtable-kafka-connect-sink/src/test/resources/compatibility_test_cases/key_matryoshkas/value-schema.json new file mode 100644 index 0000000000..4db187ebb3 --- /dev/null +++ b/google-cloud-bigtable-kafka-connect-sink/src/test/resources/compatibility_test_cases/key_matryoshkas/value-schema.json @@ -0,0 +1 @@ +{"type":"string"} \ No newline at end of file diff --git a/google-cloud-bigtable-kafka-connect-sink/src/test/resources/compatibility_test_cases/key_nestedlogicals/confluent_sink_output/Struct{inner=Struct{logical_timestamp=Thu Jan 01 00:00:04 UTC 1970,logical_time=Thu Jan 01 00:00:04 UTC 1970,logical_date=Thu Feb 12 00:00:00 UTC 1970,logical_decimal=1.089357896855742840E+68}}/confluent_compat_topic/KAFKA_VALUE b/google-cloud-bigtable-kafka-connect-sink/src/test/resources/compatibility_test_cases/key_nestedlogicals/confluent_sink_output/Struct{inner=Struct{logical_timestamp=Thu Jan 01 00:00:04 UTC 1970,logical_time=Thu Jan 01 00:00:04 UTC 1970,logical_date=Thu Feb 12 00:00:00 UTC 1970,logical_decimal=1.089357896855742840E+68}}/confluent_compat_topic/KAFKA_VALUE new file mode 100644 index 0000000000..41d3f3f489 --- /dev/null +++ b/google-cloud-bigtable-kafka-connect-sink/src/test/resources/compatibility_test_cases/key_nestedlogicals/confluent_sink_output/Struct{inner=Struct{logical_timestamp=Thu Jan 01 00:00:04 UTC 1970,logical_time=Thu Jan 01 00:00:04 UTC 1970,logical_date=Thu Feb 12 00:00:00 UTC 1970,logical_decimal=1.089357896855742840E+68}}/confluent_compat_topic/KAFKA_VALUE @@ -0,0 +1 @@ +nestedlogicals1 \ No newline at end of file diff --git a/google-cloud-bigtable-kafka-connect-sink/src/test/resources/compatibility_test_cases/key_nestedlogicals/data.json b/google-cloud-bigtable-kafka-connect-sink/src/test/resources/compatibility_test_cases/key_nestedlogicals/data.json new file mode 100644 index 0000000000..22f07a10c4 --- /dev/null +++ b/google-cloud-bigtable-kafka-connect-sink/src/test/resources/compatibility_test_cases/key_nestedlogicals/data.json @@ -0,0 +1 @@ +{"outer":{"inner":{"logical_timestamp":4242,"logical_time":4242,"logical_date":42,"logical_decimal":"\u0000\u000f\u001e\u002d\u003c\u004b\u005a\u0069\u0078"}}} "nestedlogicals1" diff --git a/google-cloud-bigtable-kafka-connect-sink/src/test/resources/compatibility_test_cases/key_nestedlogicals/key-schema.json b/google-cloud-bigtable-kafka-connect-sink/src/test/resources/compatibility_test_cases/key_nestedlogicals/key-schema.json new file mode 100644 index 0000000000..4502dc2c22 --- /dev/null +++ b/google-cloud-bigtable-kafka-connect-sink/src/test/resources/compatibility_test_cases/key_nestedlogicals/key-schema.json @@ -0,0 +1,55 @@ +{ + "type": "record", + "name": "outer_doll", + "fields": [ + { + "name": "outer", + "type": { + "type": "record", + "name": "outer", + "fields": [ + { + "name": "inner", + "type": { + "type": "record", + "name": "inner", + "fields": [ + { + "name": "logical_timestamp", + "type": { + "type": "long", + "connect.name": "org.apache.kafka.connect.data.Timestamp" + } + }, + { + "name": "logical_time", + "type": { + "type": "int", + "connect.name": "org.apache.kafka.connect.data.Time" + } + }, + { + "name": "logical_date", + "type": { + "type": "int", + "connect.name": "org.apache.kafka.connect.data.Date" + } + }, + { + "name": "logical_decimal", + "type": { + "type": "bytes", + "connect.name": "org.apache.kafka.connect.data.Decimal", + "connect.parameters": { + "scale": "-50" + } + } + } + ] + } + } + ] + } + } + ] +} \ No newline at end of file diff --git a/google-cloud-bigtable-kafka-connect-sink/src/test/resources/compatibility_test_cases/key_nestedlogicals/value-schema.json b/google-cloud-bigtable-kafka-connect-sink/src/test/resources/compatibility_test_cases/key_nestedlogicals/value-schema.json new file mode 100644 index 0000000000..4db187ebb3 --- /dev/null +++ b/google-cloud-bigtable-kafka-connect-sink/src/test/resources/compatibility_test_cases/key_nestedlogicals/value-schema.json @@ -0,0 +1 @@ +{"type":"string"} \ No newline at end of file diff --git a/google-cloud-bigtable-kafka-connect-sink/src/test/resources/compatibility_test_cases/key_primitives/confluent_sink_output/1#false#4242#42424242424242#42.42#4.242424242424242E7#forty-two/confluent_compat_topic/KAFKA_VALUE b/google-cloud-bigtable-kafka-connect-sink/src/test/resources/compatibility_test_cases/key_primitives/confluent_sink_output/1#false#4242#42424242424242#42.42#4.242424242424242E7#forty-two/confluent_compat_topic/KAFKA_VALUE new file mode 100644 index 0000000000..e3cf798959 --- /dev/null +++ b/google-cloud-bigtable-kafka-connect-sink/src/test/resources/compatibility_test_cases/key_primitives/confluent_sink_output/1#false#4242#42424242424242#42.42#4.242424242424242E7#forty-two/confluent_compat_topic/KAFKA_VALUE @@ -0,0 +1 @@ +primitives1 \ No newline at end of file diff --git a/google-cloud-bigtable-kafka-connect-sink/src/test/resources/compatibility_test_cases/key_primitives/data.json b/google-cloud-bigtable-kafka-connect-sink/src/test/resources/compatibility_test_cases/key_primitives/data.json new file mode 100644 index 0000000000..ea3c7eda44 --- /dev/null +++ b/google-cloud-bigtable-kafka-connect-sink/src/test/resources/compatibility_test_cases/key_primitives/data.json @@ -0,0 +1 @@ +{"row":1,"boolean_prim":false,"int_prim":4242,"long_prim":42424242424242,"float_prim":42.42,"double_prim":42424242.42424242,"string_prim":"forty-two"} "primitives1" diff --git a/google-cloud-bigtable-kafka-connect-sink/src/test/resources/compatibility_test_cases/key_primitives/key-schema.json b/google-cloud-bigtable-kafka-connect-sink/src/test/resources/compatibility_test_cases/key_primitives/key-schema.json new file mode 100644 index 0000000000..3f88196873 --- /dev/null +++ b/google-cloud-bigtable-kafka-connect-sink/src/test/resources/compatibility_test_cases/key_primitives/key-schema.json @@ -0,0 +1,34 @@ +{ + "type": "record", + "name": "primitives", + "fields": [ + { + "name": "row", + "type": "int" + }, + { + "name": "boolean_prim", + "type": "boolean" + }, + { + "name": "int_prim", + "type": "int" + }, + { + "name": "long_prim", + "type": "long" + }, + { + "name": "float_prim", + "type": "float" + }, + { + "name": "double_prim", + "type": "double" + }, + { + "name": "string_prim", + "type": "string" + } + ] +} \ No newline at end of file diff --git a/google-cloud-bigtable-kafka-connect-sink/src/test/resources/compatibility_test_cases/key_primitives/value-schema.json b/google-cloud-bigtable-kafka-connect-sink/src/test/resources/compatibility_test_cases/key_primitives/value-schema.json new file mode 100644 index 0000000000..4db187ebb3 --- /dev/null +++ b/google-cloud-bigtable-kafka-connect-sink/src/test/resources/compatibility_test_cases/key_primitives/value-schema.json @@ -0,0 +1 @@ +{"type":"string"} \ No newline at end of file diff --git a/google-cloud-bigtable-kafka-connect-sink/src/test/resources/compatibility_test_cases/value_containers/confluent_sink_output/containers1/confluent_compat_topic/array b/google-cloud-bigtable-kafka-connect-sink/src/test/resources/compatibility_test_cases/value_containers/confluent_sink_output/containers1/confluent_compat_topic/array new file mode 100644 index 0000000000..fde6c1d745 --- /dev/null +++ b/google-cloud-bigtable-kafka-connect-sink/src/test/resources/compatibility_test_cases/value_containers/confluent_sink_output/containers1/confluent_compat_topic/array @@ -0,0 +1 @@ +[1,2,3,4] \ No newline at end of file diff --git a/google-cloud-bigtable-kafka-connect-sink/src/test/resources/compatibility_test_cases/value_containers/confluent_sink_output/containers1/confluent_compat_topic/map b/google-cloud-bigtable-kafka-connect-sink/src/test/resources/compatibility_test_cases/value_containers/confluent_sink_output/containers1/confluent_compat_topic/map new file mode 100644 index 0000000000..5aed479781 --- /dev/null +++ b/google-cloud-bigtable-kafka-connect-sink/src/test/resources/compatibility_test_cases/value_containers/confluent_sink_output/containers1/confluent_compat_topic/map @@ -0,0 +1 @@ +{"a":1,"b":4,"c":3,"d":2} \ No newline at end of file diff --git a/google-cloud-bigtable-kafka-connect-sink/src/test/resources/compatibility_test_cases/value_containers/confluent_sink_output/containers1/struct/integer b/google-cloud-bigtable-kafka-connect-sink/src/test/resources/compatibility_test_cases/value_containers/confluent_sink_output/containers1/struct/integer new file mode 100644 index 0000000000000000000000000000000000000000..720d64f4baafc33efdf971f02084aca5f25b34a5 GIT binary patch literal 4 LcmZQzU|<9Q00jU7 literal 0 HcmV?d00001 diff --git a/google-cloud-bigtable-kafka-connect-sink/src/test/resources/compatibility_test_cases/value_containers/data.json b/google-cloud-bigtable-kafka-connect-sink/src/test/resources/compatibility_test_cases/value_containers/data.json new file mode 100644 index 0000000000..36196c2611 --- /dev/null +++ b/google-cloud-bigtable-kafka-connect-sink/src/test/resources/compatibility_test_cases/value_containers/data.json @@ -0,0 +1 @@ +"containers1" {"map":{"a": 1,"d": 2,"c": 3,"b":4},"array":[1,2,3,4],"struct":{"integer":1}} diff --git a/google-cloud-bigtable-kafka-connect-sink/src/test/resources/compatibility_test_cases/value_containers/key-schema.json b/google-cloud-bigtable-kafka-connect-sink/src/test/resources/compatibility_test_cases/value_containers/key-schema.json new file mode 100644 index 0000000000..4db187ebb3 --- /dev/null +++ b/google-cloud-bigtable-kafka-connect-sink/src/test/resources/compatibility_test_cases/value_containers/key-schema.json @@ -0,0 +1 @@ +{"type":"string"} \ No newline at end of file diff --git a/google-cloud-bigtable-kafka-connect-sink/src/test/resources/compatibility_test_cases/value_containers/value-schema.json b/google-cloud-bigtable-kafka-connect-sink/src/test/resources/compatibility_test_cases/value_containers/value-schema.json new file mode 100644 index 0000000000..ba57c02377 --- /dev/null +++ b/google-cloud-bigtable-kafka-connect-sink/src/test/resources/compatibility_test_cases/value_containers/value-schema.json @@ -0,0 +1,33 @@ +{ + "type": "record", + "name": "containers", + "fields": [ + { + "name": "map", + "type": { + "type": "map", + "values": "int" + } + }, + { + "name": "array", + "type": { + "type": "array", + "items": "int" + } + }, + { + "name": "struct", + "type": { + "type": "record", + "name": "structure", + "fields": [ + { + "name": "integer", + "type": "int" + } + ] + } + } + ] +} diff --git a/google-cloud-bigtable-kafka-connect-sink/src/test/resources/compatibility_test_cases/value_matryoshkas/confluent_sink_output/matryoshka-dolls1/confluent_compat_topic/row b/google-cloud-bigtable-kafka-connect-sink/src/test/resources/compatibility_test_cases/value_matryoshkas/confluent_sink_output/matryoshka-dolls1/confluent_compat_topic/row new file mode 100644 index 0000000000000000000000000000000000000000..720d64f4baafc33efdf971f02084aca5f25b34a5 GIT binary patch literal 4 LcmZQzU|<9Q00jU7 literal 0 HcmV?d00001 diff --git a/google-cloud-bigtable-kafka-connect-sink/src/test/resources/compatibility_test_cases/value_matryoshkas/confluent_sink_output/matryoshka-dolls1/inner/inner_int b/google-cloud-bigtable-kafka-connect-sink/src/test/resources/compatibility_test_cases/value_matryoshkas/confluent_sink_output/matryoshka-dolls1/inner/inner_int new file mode 100644 index 0000000000..a7697b31d3 --- /dev/null +++ b/google-cloud-bigtable-kafka-connect-sink/src/test/resources/compatibility_test_cases/value_matryoshkas/confluent_sink_output/matryoshka-dolls1/inner/inner_int @@ -0,0 +1 @@ +ÿÿÿÖ \ No newline at end of file diff --git a/google-cloud-bigtable-kafka-connect-sink/src/test/resources/compatibility_test_cases/value_matryoshkas/confluent_sink_output/matryoshka-dolls1/inner/inner_string b/google-cloud-bigtable-kafka-connect-sink/src/test/resources/compatibility_test_cases/value_matryoshkas/confluent_sink_output/matryoshka-dolls1/inner/inner_string new file mode 100644 index 0000000000..67f7ad0566 --- /dev/null +++ b/google-cloud-bigtable-kafka-connect-sink/src/test/resources/compatibility_test_cases/value_matryoshkas/confluent_sink_output/matryoshka-dolls1/inner/inner_string @@ -0,0 +1 @@ +-42 \ No newline at end of file diff --git a/google-cloud-bigtable-kafka-connect-sink/src/test/resources/compatibility_test_cases/value_matryoshkas/confluent_sink_output/matryoshka-dolls1/middle/inner b/google-cloud-bigtable-kafka-connect-sink/src/test/resources/compatibility_test_cases/value_matryoshkas/confluent_sink_output/matryoshka-dolls1/middle/inner new file mode 100644 index 0000000000..164adac427 --- /dev/null +++ b/google-cloud-bigtable-kafka-connect-sink/src/test/resources/compatibility_test_cases/value_matryoshkas/confluent_sink_output/matryoshka-dolls1/middle/inner @@ -0,0 +1 @@ +{"inner_int":42,"inner_string":"42"} \ No newline at end of file diff --git a/google-cloud-bigtable-kafka-connect-sink/src/test/resources/compatibility_test_cases/value_matryoshkas/confluent_sink_output/matryoshka-dolls1/middle/middle_array b/google-cloud-bigtable-kafka-connect-sink/src/test/resources/compatibility_test_cases/value_matryoshkas/confluent_sink_output/matryoshka-dolls1/middle/middle_array new file mode 100644 index 0000000000..12c84e81da --- /dev/null +++ b/google-cloud-bigtable-kafka-connect-sink/src/test/resources/compatibility_test_cases/value_matryoshkas/confluent_sink_output/matryoshka-dolls1/middle/middle_array @@ -0,0 +1 @@ +[42.0,42.42,42.4242] \ No newline at end of file diff --git a/google-cloud-bigtable-kafka-connect-sink/src/test/resources/compatibility_test_cases/value_matryoshkas/data.json b/google-cloud-bigtable-kafka-connect-sink/src/test/resources/compatibility_test_cases/value_matryoshkas/data.json new file mode 100644 index 0000000000..599c78c08d --- /dev/null +++ b/google-cloud-bigtable-kafka-connect-sink/src/test/resources/compatibility_test_cases/value_matryoshkas/data.json @@ -0,0 +1 @@ +"matryoshka-dolls1" {"row":1,"middle":{"middle_array":[42.0, 42.42, 42.4242],"inner":{"inner_int":42,"inner_string":"42"}},"inner":{"inner_int":-42,"inner_string": "-42"}} diff --git a/google-cloud-bigtable-kafka-connect-sink/src/test/resources/compatibility_test_cases/value_matryoshkas/key-schema.json b/google-cloud-bigtable-kafka-connect-sink/src/test/resources/compatibility_test_cases/value_matryoshkas/key-schema.json new file mode 100644 index 0000000000..4db187ebb3 --- /dev/null +++ b/google-cloud-bigtable-kafka-connect-sink/src/test/resources/compatibility_test_cases/value_matryoshkas/key-schema.json @@ -0,0 +1 @@ +{"type":"string"} \ No newline at end of file diff --git a/google-cloud-bigtable-kafka-connect-sink/src/test/resources/compatibility_test_cases/value_matryoshkas/value-schema.json b/google-cloud-bigtable-kafka-connect-sink/src/test/resources/compatibility_test_cases/value_matryoshkas/value-schema.json new file mode 100644 index 0000000000..0f442ca605 --- /dev/null +++ b/google-cloud-bigtable-kafka-connect-sink/src/test/resources/compatibility_test_cases/value_matryoshkas/value-schema.json @@ -0,0 +1,47 @@ +{ + "type": "record", + "name": "outer_doll", + "fields": [ + { + "name": "row", + "type": "int" + }, + { + "name": "middle", + "type": { + "type": "record", + "name": "middle_doll", + "fields": [ + { + "name": "middle_array", + "type": { + "type": "array", + "items": "float" + } + }, + { + "name": "inner", + "type": { + "type": "record", + "name": "inner_doll", + "fields": [ + { + "name": "inner_int", + "type": "int" + }, + { + "name": "inner_string", + "type": "string" + } + ] + } + } + ] + } + }, + { + "name": "inner", + "type": "inner_doll" + } + ] +} diff --git a/google-cloud-bigtable-kafka-connect-sink/src/test/resources/compatibility_test_cases/value_nestedlogicals/confluent_sink_output/nestedlogicals1/outer/inner b/google-cloud-bigtable-kafka-connect-sink/src/test/resources/compatibility_test_cases/value_nestedlogicals/confluent_sink_output/nestedlogicals1/outer/inner new file mode 100644 index 0000000000..9eae9a1459 --- /dev/null +++ b/google-cloud-bigtable-kafka-connect-sink/src/test/resources/compatibility_test_cases/value_nestedlogicals/confluent_sink_output/nestedlogicals1/outer/inner @@ -0,0 +1 @@ +{"logical_timestamp":4242,"logical_time":4242,"logical_date":42,"logical_decimal":"Dx4tPEtaaXg="} \ No newline at end of file diff --git a/google-cloud-bigtable-kafka-connect-sink/src/test/resources/compatibility_test_cases/value_nestedlogicals/data.json b/google-cloud-bigtable-kafka-connect-sink/src/test/resources/compatibility_test_cases/value_nestedlogicals/data.json new file mode 100644 index 0000000000..0ca9c68d5e --- /dev/null +++ b/google-cloud-bigtable-kafka-connect-sink/src/test/resources/compatibility_test_cases/value_nestedlogicals/data.json @@ -0,0 +1 @@ +"nestedlogicals1" {"outer":{"inner":{"logical_timestamp":4242,"logical_time":4242,"logical_date":42,"logical_decimal":"\u0000\u000f\u001e\u002d\u003c\u004b\u005a\u0069\u0078"}}} diff --git a/google-cloud-bigtable-kafka-connect-sink/src/test/resources/compatibility_test_cases/value_nestedlogicals/key-schema.json b/google-cloud-bigtable-kafka-connect-sink/src/test/resources/compatibility_test_cases/value_nestedlogicals/key-schema.json new file mode 100644 index 0000000000..4db187ebb3 --- /dev/null +++ b/google-cloud-bigtable-kafka-connect-sink/src/test/resources/compatibility_test_cases/value_nestedlogicals/key-schema.json @@ -0,0 +1 @@ +{"type":"string"} \ No newline at end of file diff --git a/google-cloud-bigtable-kafka-connect-sink/src/test/resources/compatibility_test_cases/value_nestedlogicals/value-schema.json b/google-cloud-bigtable-kafka-connect-sink/src/test/resources/compatibility_test_cases/value_nestedlogicals/value-schema.json new file mode 100644 index 0000000000..4502dc2c22 --- /dev/null +++ b/google-cloud-bigtable-kafka-connect-sink/src/test/resources/compatibility_test_cases/value_nestedlogicals/value-schema.json @@ -0,0 +1,55 @@ +{ + "type": "record", + "name": "outer_doll", + "fields": [ + { + "name": "outer", + "type": { + "type": "record", + "name": "outer", + "fields": [ + { + "name": "inner", + "type": { + "type": "record", + "name": "inner", + "fields": [ + { + "name": "logical_timestamp", + "type": { + "type": "long", + "connect.name": "org.apache.kafka.connect.data.Timestamp" + } + }, + { + "name": "logical_time", + "type": { + "type": "int", + "connect.name": "org.apache.kafka.connect.data.Time" + } + }, + { + "name": "logical_date", + "type": { + "type": "int", + "connect.name": "org.apache.kafka.connect.data.Date" + } + }, + { + "name": "logical_decimal", + "type": { + "type": "bytes", + "connect.name": "org.apache.kafka.connect.data.Decimal", + "connect.parameters": { + "scale": "-50" + } + } + } + ] + } + } + ] + } + } + ] +} \ No newline at end of file diff --git a/google-cloud-bigtable-kafka-connect-sink/src/test/resources/compatibility_test_cases/value_nulls/confluent_sink_output/nulls1/confluent_compat_topic/f1 b/google-cloud-bigtable-kafka-connect-sink/src/test/resources/compatibility_test_cases/value_nulls/confluent_sink_output/nulls1/confluent_compat_topic/f1 new file mode 100644 index 0000000000..890469e582 --- /dev/null +++ b/google-cloud-bigtable-kafka-connect-sink/src/test/resources/compatibility_test_cases/value_nulls/confluent_sink_output/nulls1/confluent_compat_topic/f1 @@ -0,0 +1 @@ +Required string \ No newline at end of file diff --git a/google-cloud-bigtable-kafka-connect-sink/src/test/resources/compatibility_test_cases/value_nulls/confluent_sink_output/nulls1/confluent_compat_topic/f2 b/google-cloud-bigtable-kafka-connect-sink/src/test/resources/compatibility_test_cases/value_nulls/confluent_sink_output/nulls1/confluent_compat_topic/f2 new file mode 100644 index 0000000000..e69de29bb2 diff --git a/google-cloud-bigtable-kafka-connect-sink/src/test/resources/compatibility_test_cases/value_nulls/confluent_sink_output/nulls1/confluent_compat_topic/f3 b/google-cloud-bigtable-kafka-connect-sink/src/test/resources/compatibility_test_cases/value_nulls/confluent_sink_output/nulls1/confluent_compat_topic/f3 new file mode 100644 index 0000000000000000000000000000000000000000..3fe959d6e15fb9443346556002dbd4a82afc72bf GIT binary patch literal 4 LcmZQzV9){p04@M4 literal 0 HcmV?d00001 diff --git a/google-cloud-bigtable-kafka-connect-sink/src/test/resources/compatibility_test_cases/value_nulls/confluent_sink_output/nulls1/confluent_compat_topic/f4 b/google-cloud-bigtable-kafka-connect-sink/src/test/resources/compatibility_test_cases/value_nulls/confluent_sink_output/nulls1/confluent_compat_topic/f4 new file mode 100644 index 0000000000000000000000000000000000000000..f76dd238ade08917e6712764a16a22005a50573d GIT binary patch literal 1 IcmZPo000310RR91 literal 0 HcmV?d00001 diff --git a/google-cloud-bigtable-kafka-connect-sink/src/test/resources/compatibility_test_cases/value_nulls/confluent_sink_output/nulls1/confluent_compat_topic/row b/google-cloud-bigtable-kafka-connect-sink/src/test/resources/compatibility_test_cases/value_nulls/confluent_sink_output/nulls1/confluent_compat_topic/row new file mode 100644 index 0000000000000000000000000000000000000000..720d64f4baafc33efdf971f02084aca5f25b34a5 GIT binary patch literal 4 LcmZQzU|<9Q00jU7 literal 0 HcmV?d00001 diff --git a/google-cloud-bigtable-kafka-connect-sink/src/test/resources/compatibility_test_cases/value_nulls/confluent_sink_output/nulls2/confluent_compat_topic/f1 b/google-cloud-bigtable-kafka-connect-sink/src/test/resources/compatibility_test_cases/value_nulls/confluent_sink_output/nulls2/confluent_compat_topic/f1 new file mode 100644 index 0000000000..890469e582 --- /dev/null +++ b/google-cloud-bigtable-kafka-connect-sink/src/test/resources/compatibility_test_cases/value_nulls/confluent_sink_output/nulls2/confluent_compat_topic/f1 @@ -0,0 +1 @@ +Required string \ No newline at end of file diff --git a/google-cloud-bigtable-kafka-connect-sink/src/test/resources/compatibility_test_cases/value_nulls/confluent_sink_output/nulls2/confluent_compat_topic/f2 b/google-cloud-bigtable-kafka-connect-sink/src/test/resources/compatibility_test_cases/value_nulls/confluent_sink_output/nulls2/confluent_compat_topic/f2 new file mode 100644 index 0000000000..3d7d0e812e --- /dev/null +++ b/google-cloud-bigtable-kafka-connect-sink/src/test/resources/compatibility_test_cases/value_nulls/confluent_sink_output/nulls2/confluent_compat_topic/f2 @@ -0,0 +1 @@ +Optional string \ No newline at end of file diff --git a/google-cloud-bigtable-kafka-connect-sink/src/test/resources/compatibility_test_cases/value_nulls/confluent_sink_output/nulls2/confluent_compat_topic/f3 b/google-cloud-bigtable-kafka-connect-sink/src/test/resources/compatibility_test_cases/value_nulls/confluent_sink_output/nulls2/confluent_compat_topic/f3 new file mode 100644 index 0000000000000000000000000000000000000000..189346e0e0b1279b4d5aed0ecdc6aa0062ee274a GIT binary patch literal 4 LcmZQzV2A_&09^oD literal 0 HcmV?d00001 diff --git a/google-cloud-bigtable-kafka-connect-sink/src/test/resources/compatibility_test_cases/value_nulls/confluent_sink_output/nulls2/confluent_compat_topic/f4 b/google-cloud-bigtable-kafka-connect-sink/src/test/resources/compatibility_test_cases/value_nulls/confluent_sink_output/nulls2/confluent_compat_topic/f4 new file mode 100644 index 0000000000..e69de29bb2 diff --git a/google-cloud-bigtable-kafka-connect-sink/src/test/resources/compatibility_test_cases/value_nulls/confluent_sink_output/nulls2/confluent_compat_topic/row b/google-cloud-bigtable-kafka-connect-sink/src/test/resources/compatibility_test_cases/value_nulls/confluent_sink_output/nulls2/confluent_compat_topic/row new file mode 100644 index 0000000000000000000000000000000000000000..b6a8ef3e7ca7c398cd8f65bb1e21a23c0d251536 GIT binary patch literal 4 LcmZQzU|<3O00sa9 literal 0 HcmV?d00001 diff --git a/google-cloud-bigtable-kafka-connect-sink/src/test/resources/compatibility_test_cases/value_nulls/confluent_sink_output/nulls3/confluent_compat_topic/f1 b/google-cloud-bigtable-kafka-connect-sink/src/test/resources/compatibility_test_cases/value_nulls/confluent_sink_output/nulls3/confluent_compat_topic/f1 new file mode 100644 index 0000000000..890469e582 --- /dev/null +++ b/google-cloud-bigtable-kafka-connect-sink/src/test/resources/compatibility_test_cases/value_nulls/confluent_sink_output/nulls3/confluent_compat_topic/f1 @@ -0,0 +1 @@ +Required string \ No newline at end of file diff --git a/google-cloud-bigtable-kafka-connect-sink/src/test/resources/compatibility_test_cases/value_nulls/confluent_sink_output/nulls3/confluent_compat_topic/f2 b/google-cloud-bigtable-kafka-connect-sink/src/test/resources/compatibility_test_cases/value_nulls/confluent_sink_output/nulls3/confluent_compat_topic/f2 new file mode 100644 index 0000000000..e69de29bb2 diff --git a/google-cloud-bigtable-kafka-connect-sink/src/test/resources/compatibility_test_cases/value_nulls/confluent_sink_output/nulls3/confluent_compat_topic/f3 b/google-cloud-bigtable-kafka-connect-sink/src/test/resources/compatibility_test_cases/value_nulls/confluent_sink_output/nulls3/confluent_compat_topic/f3 new file mode 100644 index 0000000000..e69de29bb2 diff --git a/google-cloud-bigtable-kafka-connect-sink/src/test/resources/compatibility_test_cases/value_nulls/confluent_sink_output/nulls3/confluent_compat_topic/f4 b/google-cloud-bigtable-kafka-connect-sink/src/test/resources/compatibility_test_cases/value_nulls/confluent_sink_output/nulls3/confluent_compat_topic/f4 new file mode 100644 index 0000000000..ce542efaa5 --- /dev/null +++ b/google-cloud-bigtable-kafka-connect-sink/src/test/resources/compatibility_test_cases/value_nulls/confluent_sink_output/nulls3/confluent_compat_topic/f4 @@ -0,0 +1 @@ +ÿ \ No newline at end of file diff --git a/google-cloud-bigtable-kafka-connect-sink/src/test/resources/compatibility_test_cases/value_nulls/confluent_sink_output/nulls3/confluent_compat_topic/row b/google-cloud-bigtable-kafka-connect-sink/src/test/resources/compatibility_test_cases/value_nulls/confluent_sink_output/nulls3/confluent_compat_topic/row new file mode 100644 index 0000000000000000000000000000000000000000..fde1ac19d2b083530bcab4cb4fd2dcaa285234ab GIT binary patch literal 4 LcmZQzU| Date: Thu, 30 Jan 2025 16:52:12 +0100 Subject: [PATCH 23/76] Provide service provider configuration file --- .../services/org.apache.kafka.connect.sink.SinkConnector | 1 + 1 file changed, 1 insertion(+) create mode 100644 google-cloud-bigtable-kafka-connect-sink/src/main/resources/META-INF/services/org.apache.kafka.connect.sink.SinkConnector diff --git a/google-cloud-bigtable-kafka-connect-sink/src/main/resources/META-INF/services/org.apache.kafka.connect.sink.SinkConnector b/google-cloud-bigtable-kafka-connect-sink/src/main/resources/META-INF/services/org.apache.kafka.connect.sink.SinkConnector new file mode 100644 index 0000000000..70eae2bf00 --- /dev/null +++ b/google-cloud-bigtable-kafka-connect-sink/src/main/resources/META-INF/services/org.apache.kafka.connect.sink.SinkConnector @@ -0,0 +1 @@ +com.google.cloud.kafka.connect.bigtable.BigtableSinkConnector From 9e6b20cae0aea704d4653f4267a425861235c83f Mon Sep 17 00:00:00 2001 From: Adam Czajkowski Date: Tue, 4 Feb 2025 12:14:24 +0100 Subject: [PATCH 24/76] Unbreak DifferentConvertersIT --- .../connect/bigtable/integration/DifferentConvertersIT.java | 6 ++++-- 1 file changed, 4 insertions(+), 2 deletions(-) diff --git a/google-cloud-bigtable-kafka-connect-sink/src/test/java/com/google/cloud/kafka/connect/bigtable/integration/DifferentConvertersIT.java b/google-cloud-bigtable-kafka-connect-sink/src/test/java/com/google/cloud/kafka/connect/bigtable/integration/DifferentConvertersIT.java index 3233c4acf3..5c92b40f74 100644 --- a/google-cloud-bigtable-kafka-connect-sink/src/test/java/com/google/cloud/kafka/connect/bigtable/integration/DifferentConvertersIT.java +++ b/google-cloud-bigtable-kafka-connect-sink/src/test/java/com/google/cloud/kafka/connect/bigtable/integration/DifferentConvertersIT.java @@ -23,7 +23,9 @@ */ import com.google.cloud.kafka.connect.bigtable.config.BigtableSinkConfig; +import io.confluent.connect.avro.AvroConverter; import io.confluent.connect.json.JsonSchemaConverter; +import io.confluent.connect.protobuf.ProtobufConverter; import io.confluent.kafka.serializers.AbstractKafkaSchemaSerDeConfig; import java.math.BigDecimal; import java.util.AbstractMap; @@ -99,7 +101,7 @@ public class DifferentConvertersIT extends BaseKafkaConnectBigtableSchemaRegistr @Test public void testAvro() throws InterruptedException { testConverter( - JsonSchemaConverter::new, + AvroConverter::new, Map.of( AbstractKafkaSchemaSerDeConfig.SCHEMA_REGISTRY_URL_CONFIG, schemaRegistry.schemaRegistryUrl())); @@ -108,7 +110,7 @@ public void testAvro() throws InterruptedException { @Test public void testProtobuf() throws InterruptedException { testConverter( - JsonSchemaConverter::new, + ProtobufConverter::new, Map.of( AbstractKafkaSchemaSerDeConfig.SCHEMA_REGISTRY_URL_CONFIG, schemaRegistry.schemaRegistryUrl())); From 3f23090cd7d0d4be2db37dd814d6a31322ff2269 Mon Sep 17 00:00:00 2001 From: Adam Czajkowski Date: Thu, 6 Feb 2025 13:20:32 +0100 Subject: [PATCH 25/76] Use Parameterized for DifferentConvertersIT --- .../integration/DifferentConvertersIT.java | 82 +++++++++---------- 1 file changed, 41 insertions(+), 41 deletions(-) diff --git a/google-cloud-bigtable-kafka-connect-sink/src/test/java/com/google/cloud/kafka/connect/bigtable/integration/DifferentConvertersIT.java b/google-cloud-bigtable-kafka-connect-sink/src/test/java/com/google/cloud/kafka/connect/bigtable/integration/DifferentConvertersIT.java index 5c92b40f74..d73188d26e 100644 --- a/google-cloud-bigtable-kafka-connect-sink/src/test/java/com/google/cloud/kafka/connect/bigtable/integration/DifferentConvertersIT.java +++ b/google-cloud-bigtable-kafka-connect-sink/src/test/java/com/google/cloud/kafka/connect/bigtable/integration/DifferentConvertersIT.java @@ -30,6 +30,9 @@ import java.math.BigDecimal; import java.util.AbstractMap; import java.util.ArrayList; +import java.util.Arrays; +import java.util.Collection; +import java.util.HashMap; import java.util.List; import java.util.Map; import java.util.function.Supplier; @@ -49,9 +52,9 @@ import org.apache.kafka.connect.storage.Converter; import org.junit.Test; import org.junit.runner.RunWith; -import org.junit.runners.JUnit4; +import org.junit.runners.Parameterized; -@RunWith(JUnit4.class) +@RunWith(Parameterized.class) public class DifferentConvertersIT extends BaseKafkaConnectBigtableSchemaRegistryIT { private static final Schema SUB_STRUCT_SCHEMA = SchemaBuilder.struct() @@ -98,50 +101,47 @@ public class DifferentConvertersIT extends BaseKafkaConnectBigtableSchemaRegistr .build(); private static final Long NUM_RECORDS = 100L; - @Test - public void testAvro() throws InterruptedException { - testConverter( - AvroConverter::new, - Map.of( - AbstractKafkaSchemaSerDeConfig.SCHEMA_REGISTRY_URL_CONFIG, - schemaRegistry.schemaRegistryUrl())); - } - - @Test - public void testProtobuf() throws InterruptedException { - testConverter( - ProtobufConverter::new, - Map.of( - AbstractKafkaSchemaSerDeConfig.SCHEMA_REGISTRY_URL_CONFIG, - schemaRegistry.schemaRegistryUrl())); + private Supplier converterConstructor; + private Map converterBaseConfig; + private boolean converterUsesSchemaRegistry; + + @Parameterized.Parameters + public static Collection testCases() { + return Arrays.asList( + new Object[][] { + {(Supplier) AvroConverter::new, Map.of(), true}, + {(Supplier) ProtobufConverter::new, Map.of(), true}, + {(Supplier) JsonSchemaConverter::new, Map.of(), true}, + { + (Supplier) JsonConverter::new, + Map.of(JsonConverterConfig.SCHEMAS_ENABLE_CONFIG, String.valueOf(false)), + false + }, + { + (Supplier) JsonConverter::new, + Map.of(JsonConverterConfig.SCHEMAS_ENABLE_CONFIG, String.valueOf(true)), + false + }, + }); } - @Test - public void testSchemaJson() throws InterruptedException { - testConverter( - JsonSchemaConverter::new, - Map.of( - AbstractKafkaSchemaSerDeConfig.SCHEMA_REGISTRY_URL_CONFIG, - schemaRegistry.schemaRegistryUrl())); + public DifferentConvertersIT( + Supplier converterConstructor, + Map converterBaseConfig, + boolean converterUsesSchemaRegistry) { + this.converterConstructor = converterConstructor; + this.converterBaseConfig = converterBaseConfig; + this.converterUsesSchemaRegistry = converterUsesSchemaRegistry; } @Test - public void testJsonWithInBandSchema() throws InterruptedException { - testConverter( - JsonConverter::new, - Map.of(JsonConverterConfig.SCHEMAS_ENABLE_CONFIG, String.valueOf(true))); - } - - @Test - public void testJsonWithoutSchema() throws InterruptedException { - testConverter( - JsonConverter::new, - Map.of(JsonConverterConfig.SCHEMAS_ENABLE_CONFIG, String.valueOf(false))); - } - - private void testConverter( - Supplier converterConstructor, Map converterProps) - throws InterruptedException { + public void testConverter() throws InterruptedException { + Map converterProps = new HashMap<>(converterBaseConfig); + if (converterUsesSchemaRegistry) { + converterProps.put( + AbstractKafkaSchemaSerDeConfig.SCHEMA_REGISTRY_URL_CONFIG, + schemaRegistry.schemaRegistryUrl()); + } Converter keyConverter = converterConstructor.get(); keyConverter.configure(converterProps, true); Converter valueConverter = converterConstructor.get(); From 5da58abc9d174a7d0caece9ca4582ed8384c47ce Mon Sep 17 00:00:00 2001 From: Adam Czajkowski Date: Thu, 6 Feb 2025 15:27:24 +0100 Subject: [PATCH 26/76] Warning about VersionIT --- .../cloud/kafka/connect/bigtable/integration/VersionIT.java | 1 + 1 file changed, 1 insertion(+) diff --git a/google-cloud-bigtable-kafka-connect-sink/src/test/java/com/google/cloud/kafka/connect/bigtable/integration/VersionIT.java b/google-cloud-bigtable-kafka-connect-sink/src/test/java/com/google/cloud/kafka/connect/bigtable/integration/VersionIT.java index 49fa599512..0500f0e11e 100644 --- a/google-cloud-bigtable-kafka-connect-sink/src/test/java/com/google/cloud/kafka/connect/bigtable/integration/VersionIT.java +++ b/google-cloud-bigtable-kafka-connect-sink/src/test/java/com/google/cloud/kafka/connect/bigtable/integration/VersionIT.java @@ -35,6 +35,7 @@ @RunWith(JUnit4.class) public class VersionIT extends BaseKafkaConnectIT { @Test + // Note that it fails within IntelliJ. Test with `mvn` directly. public void testVersionIsSet() throws IOException, InterruptedException { String url = connect.endpointForResource("connector-plugins"); HttpClient http = HttpClient.newHttpClient(); From e8eddd862726be18c9049f74a2a9ac18323f8e8b Mon Sep 17 00:00:00 2001 From: Adam Czajkowski Date: Wed, 5 Feb 2025 18:06:23 +0100 Subject: [PATCH 27/76] Stop considering FAILED_PRECONDITION an input error --- .../connect/bigtable/autocreate/BigtableSchemaManager.java | 1 - .../connect/bigtable/autocreate/SchemaApiExceptionsTest.java | 4 ++-- 2 files changed, 2 insertions(+), 3 deletions(-) diff --git a/google-cloud-bigtable-kafka-connect-sink/src/main/java/com/google/cloud/kafka/connect/bigtable/autocreate/BigtableSchemaManager.java b/google-cloud-bigtable-kafka-connect-sink/src/main/java/com/google/cloud/kafka/connect/bigtable/autocreate/BigtableSchemaManager.java index a43a22a52f..b31506d2a4 100644 --- a/google-cloud-bigtable-kafka-connect-sink/src/main/java/com/google/cloud/kafka/connect/bigtable/autocreate/BigtableSchemaManager.java +++ b/google-cloud-bigtable-kafka-connect-sink/src/main/java/com/google/cloud/kafka/connect/bigtable/autocreate/BigtableSchemaManager.java @@ -461,7 +461,6 @@ static Optional maybeExtractBigtableStatusCode(Throwable t) { static boolean isStatusCodeCausedByInputError(StatusCode.Code code) { switch (code) { case INVALID_ARGUMENT: - case FAILED_PRECONDITION: case OUT_OF_RANGE: return true; default: diff --git a/google-cloud-bigtable-kafka-connect-sink/src/test/java/com/google/cloud/kafka/connect/bigtable/autocreate/SchemaApiExceptionsTest.java b/google-cloud-bigtable-kafka-connect-sink/src/test/java/com/google/cloud/kafka/connect/bigtable/autocreate/SchemaApiExceptionsTest.java index c0f95e423d..d95ed55c48 100644 --- a/google-cloud-bigtable-kafka-connect-sink/src/test/java/com/google/cloud/kafka/connect/bigtable/autocreate/SchemaApiExceptionsTest.java +++ b/google-cloud-bigtable-kafka-connect-sink/src/test/java/com/google/cloud/kafka/connect/bigtable/autocreate/SchemaApiExceptionsTest.java @@ -36,8 +36,7 @@ public class SchemaApiExceptionsTest { @Test public void testIsStatusCodeCausedByInputError() { - for (Code causedByInputError : - List.of(Code.INVALID_ARGUMENT, Code.OUT_OF_RANGE, Code.FAILED_PRECONDITION)) { + for (Code causedByInputError : List.of(Code.INVALID_ARGUMENT, Code.OUT_OF_RANGE)) { assertTrue( BigtableSchemaManager.SchemaApiExceptions.isStatusCodeCausedByInputError( causedByInputError)); @@ -49,6 +48,7 @@ public void testIsStatusCodeCausedByInputError() { Code.NOT_FOUND, Code.RESOURCE_EXHAUSTED, Code.CANCELLED, + Code.FAILED_PRECONDITION, Code.UNKNOWN)) { assertFalse( BigtableSchemaManager.SchemaApiExceptions.isStatusCodeCausedByInputError( From b1799a3e0731d47db558ba5950c2478db0e44a8c Mon Sep 17 00:00:00 2001 From: Adam Czajkowski Date: Fri, 31 Jan 2025 15:00:32 +0100 Subject: [PATCH 28/76] Move DLQ helpers to a base class --- .../integration/BaseKafkaConnectIT.java | 15 ++++++++ .../integration/ErrorReportingIT.java | 11 +----- .../integration/ResourceAutoCreationIT.java | 37 +++++-------------- 3 files changed, 26 insertions(+), 37 deletions(-) diff --git a/google-cloud-bigtable-kafka-connect-sink/src/test/java/com/google/cloud/kafka/connect/bigtable/integration/BaseKafkaConnectIT.java b/google-cloud-bigtable-kafka-connect-sink/src/test/java/com/google/cloud/kafka/connect/bigtable/integration/BaseKafkaConnectIT.java index 533a686f8a..72844e5a98 100644 --- a/google-cloud-bigtable-kafka-connect-sink/src/test/java/com/google/cloud/kafka/connect/bigtable/integration/BaseKafkaConnectIT.java +++ b/google-cloud-bigtable-kafka-connect-sink/src/test/java/com/google/cloud/kafka/connect/bigtable/integration/BaseKafkaConnectIT.java @@ -28,6 +28,8 @@ import org.apache.kafka.clients.producer.RecordMetadata; import org.apache.kafka.common.serialization.ByteArraySerializer; import org.apache.kafka.connect.data.SchemaAndValue; +import org.apache.kafka.connect.runtime.SinkConnectorConfig; +import org.apache.kafka.connect.runtime.errors.ToleranceType; import org.apache.kafka.connect.storage.Converter; import org.junit.After; import org.junit.Before; @@ -81,4 +83,17 @@ public void sendRecords( }); } } + + public String createDlq() { + String dlqTopic = getTestCaseId() + System.currentTimeMillis(); + connect.kafka().createTopic(dlqTopic, numBrokers); + return dlqTopic; + } + + public void configureDlq(Map props, String dlqTopic) { + props.put(SinkConnectorConfig.DLQ_TOPIC_NAME_CONFIG, dlqTopic); + props.put(SinkConnectorConfig.DLQ_CONTEXT_HEADERS_ENABLE_CONFIG, String.valueOf(true)); + props.put(SinkConnectorConfig.DLQ_TOPIC_REPLICATION_FACTOR_CONFIG, String.valueOf(numBrokers)); + props.put(SinkConnectorConfig.ERRORS_TOLERANCE_CONFIG, ToleranceType.ALL.value()); + } } diff --git a/google-cloud-bigtable-kafka-connect-sink/src/test/java/com/google/cloud/kafka/connect/bigtable/integration/ErrorReportingIT.java b/google-cloud-bigtable-kafka-connect-sink/src/test/java/com/google/cloud/kafka/connect/bigtable/integration/ErrorReportingIT.java index 4e1838ae98..9e5c624c8b 100644 --- a/google-cloud-bigtable-kafka-connect-sink/src/test/java/com/google/cloud/kafka/connect/bigtable/integration/ErrorReportingIT.java +++ b/google-cloud-bigtable-kafka-connect-sink/src/test/java/com/google/cloud/kafka/connect/bigtable/integration/ErrorReportingIT.java @@ -27,8 +27,6 @@ import java.util.Map; import org.apache.kafka.clients.consumer.ConsumerRecord; import org.apache.kafka.clients.consumer.ConsumerRecords; -import org.apache.kafka.connect.runtime.SinkConnectorConfig; -import org.apache.kafka.connect.runtime.errors.ToleranceType; import org.junit.Test; import org.junit.runner.RunWith; import org.junit.runners.JUnit4; @@ -76,15 +74,10 @@ public void testErrorModeIgnore() throws InterruptedException { @Test public void testErrorModeDLQOverridesErrorMode() throws InterruptedException { - String dlqTopic = getTestCaseId() + System.currentTimeMillis(); - connect.kafka().createTopic(dlqTopic, numBrokers); - + String dlqTopic = createDlq(); Map props = baseConnectorProps(); props.put(BigtableSinkConfig.CONFIG_ERROR_MODE, BigtableErrorMode.FAIL.name()); - props.put(SinkConnectorConfig.DLQ_TOPIC_NAME_CONFIG, dlqTopic); - props.put(SinkConnectorConfig.DLQ_CONTEXT_HEADERS_ENABLE_CONFIG, String.valueOf(true)); - props.put(SinkConnectorConfig.DLQ_TOPIC_REPLICATION_FACTOR_CONFIG, String.valueOf(numBrokers)); - props.put(SinkConnectorConfig.ERRORS_TOLERANCE_CONFIG, ToleranceType.ALL.value()); + configureDlq(props, dlqTopic); String key = "key"; String value = "value"; diff --git a/google-cloud-bigtable-kafka-connect-sink/src/test/java/com/google/cloud/kafka/connect/bigtable/integration/ResourceAutoCreationIT.java b/google-cloud-bigtable-kafka-connect-sink/src/test/java/com/google/cloud/kafka/connect/bigtable/integration/ResourceAutoCreationIT.java index 1e34cf2fa4..385564346a 100644 --- a/google-cloud-bigtable-kafka-connect-sink/src/test/java/com/google/cloud/kafka/connect/bigtable/integration/ResourceAutoCreationIT.java +++ b/google-cloud-bigtable-kafka-connect-sink/src/test/java/com/google/cloud/kafka/connect/bigtable/integration/ResourceAutoCreationIT.java @@ -28,8 +28,6 @@ import java.time.Duration; import java.util.Arrays; import java.util.Map; -import java.util.concurrent.ExecutionException; -import java.util.concurrent.TimeoutException; import java.util.stream.StreamSupport; import org.apache.kafka.clients.consumer.ConsumerRecords; import org.apache.kafka.connect.data.Schema; @@ -38,8 +36,6 @@ import org.apache.kafka.connect.json.JsonConverter; import org.apache.kafka.connect.json.JsonConverterConfig; import org.apache.kafka.connect.runtime.ConnectorConfig; -import org.apache.kafka.connect.runtime.SinkConnectorConfig; -import org.apache.kafka.connect.runtime.errors.ToleranceType; import org.junit.Test; import org.junit.runner.RunWith; import org.junit.runners.JUnit4; @@ -49,10 +45,16 @@ public class ResourceAutoCreationIT extends BaseKafkaConnectBigtableIT { private final JsonConverter jsonConverter = JsonConverterFactory.create(true, false); @Test - public void testMissingAndLaterCreatedTableAndColumnFamily() - throws InterruptedException, ExecutionException, TimeoutException { + public void testMissingAndLaterCreatedTableAndColumnFamily() throws InterruptedException { String dlqTopic = createDlq(); - Map props = dlqAndJsonValuesProps(dlqTopic); + Map props = baseConnectorProps(); + configureDlq(props, dlqTopic); + props.put(ConnectorConfig.VALUE_CONVERTER_CLASS_CONFIG, JsonConverter.class.getName()); + props.put( + ConnectorConfig.VALUE_CONVERTER_CLASS_CONFIG + + "." + + JsonConverterConfig.SCHEMAS_ENABLE_CONFIG, + String.valueOf(true)); String testId = startSingleTopicConnector(props); @@ -135,25 +137,4 @@ public void testMissingAndLaterCreatedTableAndColumnFamily() .assertConnectorAndExactlyNumTasksAreRunning( testId, numTasks, "Wrong number of tasks is running"); } - - private String createDlq() { - String dlqTopic = getTestCaseId() + System.currentTimeMillis(); - connect.kafka().createTopic(dlqTopic, numBrokers); - return dlqTopic; - } - - private Map dlqAndJsonValuesProps(String dlqTopic) { - Map props = baseConnectorProps(); - props.put(SinkConnectorConfig.DLQ_TOPIC_NAME_CONFIG, dlqTopic); - props.put(SinkConnectorConfig.DLQ_CONTEXT_HEADERS_ENABLE_CONFIG, String.valueOf(false)); - props.put(SinkConnectorConfig.DLQ_TOPIC_REPLICATION_FACTOR_CONFIG, String.valueOf(numBrokers)); - props.put(SinkConnectorConfig.ERRORS_TOLERANCE_CONFIG, ToleranceType.ALL.value()); - props.put(ConnectorConfig.VALUE_CONVERTER_CLASS_CONFIG, JsonConverter.class.getName()); - props.put( - ConnectorConfig.VALUE_CONVERTER_CLASS_CONFIG - + "." - + JsonConverterConfig.SCHEMAS_ENABLE_CONFIG, - String.valueOf(true)); - return props; - } } From 4a10fa03bd646c48c6d71fa57d8e7d9da7c02cee Mon Sep 17 00:00:00 2001 From: Adam Czajkowski Date: Wed, 5 Feb 2025 17:39:16 +0100 Subject: [PATCH 29/76] Trace level logs for resource creation --- .../connect/bigtable/autocreate/BigtableSchemaManager.java | 7 +++++++ 1 file changed, 7 insertions(+) diff --git a/google-cloud-bigtable-kafka-connect-sink/src/main/java/com/google/cloud/kafka/connect/bigtable/autocreate/BigtableSchemaManager.java b/google-cloud-bigtable-kafka-connect-sink/src/main/java/com/google/cloud/kafka/connect/bigtable/autocreate/BigtableSchemaManager.java index b31506d2a4..6bd8c1a339 100644 --- a/google-cloud-bigtable-kafka-connect-sink/src/main/java/com/google/cloud/kafka/connect/bigtable/autocreate/BigtableSchemaManager.java +++ b/google-cloud-bigtable-kafka-connect-sink/src/main/java/com/google/cloud/kafka/connect/bigtable/autocreate/BigtableSchemaManager.java @@ -390,7 +390,9 @@ , Id> Set awaitResourceCreationAndHandleInv List sinkRecords = resourceAndRecords.getRecords(); try { fut.get(); + logger.trace("Resource {} created successfully.", resource); } catch (ExecutionException | InterruptedException e) { + logger.trace("Resource {} NOT created successfully.", resource); String errorMessage = String.format(errorMessageTemplate, resource.toString()); if (SchemaApiExceptions.isCausedByInputError(e)) { dataErrors.addAll(sinkRecords); @@ -425,6 +427,11 @@ public Id getResource() { public List getRecords() { return records; } + + @Override + public String toString() { + return String.format("Resource(id=%s,#records=%d)", resource, records.size()); + } } /** From f1b15590035d4e957b3bc9a9e52982f3f5ed1610 Mon Sep 17 00:00:00 2001 From: Adam Czajkowski Date: Fri, 31 Jan 2025 15:35:34 +0100 Subject: [PATCH 30/76] Error handling Integration Tests --- .../connect/bigtable/integration/BaseIT.java | 14 +- .../integration/BaseKafkaConnectIT.java | 9 +- .../bigtable/integration/ErrorHandlingIT.java | 192 +++++++++++++++++- .../integration/ErrorReportingIT.java | 5 +- 4 files changed, 209 insertions(+), 11 deletions(-) diff --git a/google-cloud-bigtable-kafka-connect-sink/src/test/java/com/google/cloud/kafka/connect/bigtable/integration/BaseIT.java b/google-cloud-bigtable-kafka-connect-sink/src/test/java/com/google/cloud/kafka/connect/bigtable/integration/BaseIT.java index 6e5481ade5..7d7434cd79 100644 --- a/google-cloud-bigtable-kafka-connect-sink/src/test/java/com/google/cloud/kafka/connect/bigtable/integration/BaseIT.java +++ b/google-cloud-bigtable-kafka-connect-sink/src/test/java/com/google/cloud/kafka/connect/bigtable/integration/BaseIT.java @@ -35,6 +35,8 @@ import java.util.Properties; import java.util.stream.Collectors; import org.apache.kafka.clients.admin.Admin; +import org.apache.kafka.clients.consumer.ConsumerConfig; +import org.apache.kafka.clients.producer.ProducerConfig; import org.apache.kafka.common.utils.Utils; import org.apache.kafka.connect.runtime.SinkConnectorConfig; import org.apache.kafka.connect.runtime.WorkerConfig; @@ -51,7 +53,7 @@ public abstract class BaseIT { protected int numWorkers = 1; protected int numBrokers = 1; protected int numTasks = 1; - protected int maxKafkaMessageSizeBytes = 10 * 1024 * 1024; + protected int maxKafkaMessageSizeBytes = 300 * 1024 * 1024; protected void startConnect() { logger.info("Starting embedded Kafka Connect cluster..."); @@ -60,9 +62,18 @@ protected void startConnect() { workerProps.put(WorkerConfig.PLUGIN_DISCOVERY_CONFIG, PluginDiscoveryMode.HYBRID_WARN.name()); Properties brokerProps = new Properties(); + brokerProps.put("socket.request.max.bytes", maxKafkaMessageSizeBytes); brokerProps.put("message.max.bytes", maxKafkaMessageSizeBytes); brokerProps.put("auto.create.topics.enable", "false"); brokerProps.put("delete.topic.enable", "true"); + + Map clientConfigs = new HashMap<>(); + clientConfigs.put( + ProducerConfig.MAX_REQUEST_SIZE_CONFIG, String.valueOf(maxKafkaMessageSizeBytes)); + clientConfigs.put( + ProducerConfig.BUFFER_MEMORY_CONFIG, String.valueOf(maxKafkaMessageSizeBytes)); + clientConfigs.put( + ConsumerConfig.MAX_PARTITION_FETCH_BYTES_CONFIG, String.valueOf(maxKafkaMessageSizeBytes)); connect = new EmbeddedConnectCluster.Builder() .name("kcbt-connect-cluster-" + getTestClassId()) @@ -70,6 +81,7 @@ protected void startConnect() { .numBrokers(numBrokers) .brokerProps(brokerProps) .workerProps(workerProps) + .clientConfigs(clientConfigs) .build(); // Start the clusters diff --git a/google-cloud-bigtable-kafka-connect-sink/src/test/java/com/google/cloud/kafka/connect/bigtable/integration/BaseKafkaConnectIT.java b/google-cloud-bigtable-kafka-connect-sink/src/test/java/com/google/cloud/kafka/connect/bigtable/integration/BaseKafkaConnectIT.java index 72844e5a98..d816abefe9 100644 --- a/google-cloud-bigtable-kafka-connect-sink/src/test/java/com/google/cloud/kafka/connect/bigtable/integration/BaseKafkaConnectIT.java +++ b/google-cloud-bigtable-kafka-connect-sink/src/test/java/com/google/cloud/kafka/connect/bigtable/integration/BaseKafkaConnectIT.java @@ -17,16 +17,14 @@ import java.util.ArrayList; import java.util.Collection; -import java.util.HashMap; +import java.util.Collections; import java.util.List; import java.util.Map; import java.util.concurrent.Future; import java.util.concurrent.TimeUnit; import org.apache.kafka.clients.producer.KafkaProducer; -import org.apache.kafka.clients.producer.ProducerConfig; import org.apache.kafka.clients.producer.ProducerRecord; import org.apache.kafka.clients.producer.RecordMetadata; -import org.apache.kafka.common.serialization.ByteArraySerializer; import org.apache.kafka.connect.data.SchemaAndValue; import org.apache.kafka.connect.runtime.SinkConnectorConfig; import org.apache.kafka.connect.runtime.errors.ToleranceType; @@ -48,10 +46,7 @@ public void tearDownConnect() { } public KafkaProducer getKafkaProducer() { - Map producerProps = new HashMap<>(); - producerProps.put(ProducerConfig.BOOTSTRAP_SERVERS_CONFIG, connect.kafka().bootstrapServers()); - producerProps.put(ProducerConfig.MAX_REQUEST_SIZE_CONFIG, maxKafkaMessageSizeBytes); - return new KafkaProducer<>(producerProps, new ByteArraySerializer(), new ByteArraySerializer()); + return connect.kafka().createProducer(Collections.emptyMap()); } public void sendRecords( diff --git a/google-cloud-bigtable-kafka-connect-sink/src/test/java/com/google/cloud/kafka/connect/bigtable/integration/ErrorHandlingIT.java b/google-cloud-bigtable-kafka-connect-sink/src/test/java/com/google/cloud/kafka/connect/bigtable/integration/ErrorHandlingIT.java index 67c9f1afa7..fa2ff0b2bc 100644 --- a/google-cloud-bigtable-kafka-connect-sink/src/test/java/com/google/cloud/kafka/connect/bigtable/integration/ErrorHandlingIT.java +++ b/google-cloud-bigtable-kafka-connect-sink/src/test/java/com/google/cloud/kafka/connect/bigtable/integration/ErrorHandlingIT.java @@ -15,10 +15,36 @@ */ package com.google.cloud.kafka.connect.bigtable.integration; +import static org.junit.Assert.assertArrayEquals; +import static org.junit.Assert.assertEquals; import static org.junit.Assert.assertThrows; +import static org.junit.Assert.assertTrue; +import com.google.cloud.bigtable.data.v2.models.Row; import com.google.cloud.kafka.connect.bigtable.config.BigtableSinkConfig; +import com.google.cloud.kafka.connect.bigtable.config.InsertMode; +import com.google.cloud.kafka.connect.bigtable.exception.InvalidBigtableSchemaModificationException; +import com.google.protobuf.ByteString; +import java.nio.charset.StandardCharsets; +import java.time.Duration; +import java.util.AbstractMap; +import java.util.ArrayList; +import java.util.Arrays; +import java.util.HashMap; +import java.util.List; import java.util.Map; +import java.util.concurrent.ExecutionException; +import java.util.function.Function; +import org.apache.kafka.clients.consumer.ConsumerConfig; +import org.apache.kafka.clients.consumer.ConsumerRecord; +import org.apache.kafka.clients.consumer.ConsumerRecords; +import org.apache.kafka.clients.producer.ProducerRecord; +import org.apache.kafka.connect.converters.ByteArrayConverter; +import org.apache.kafka.connect.data.Schema; +import org.apache.kafka.connect.data.SchemaAndValue; +import org.apache.kafka.connect.runtime.ConnectorConfig; +import org.apache.kafka.connect.runtime.errors.DeadLetterQueueReporter; +import org.apache.kafka.connect.storage.StringConverter; import org.junit.Test; import org.junit.runner.RunWith; import org.junit.runners.JUnit4; @@ -26,7 +52,7 @@ @RunWith(JUnit4.class) public class ErrorHandlingIT extends BaseKafkaConnectBigtableIT { @Test - public void testBigtableCredentialsAreCheckedOnStartup() throws InterruptedException { + public void testBigtableCredentialsAreCheckedOnStartup() { Map props = baseConnectorProps(); props.put(BigtableSinkConfig.CONFIG_GCP_CREDENTIALS_JSON, "{}"); @@ -34,4 +60,168 @@ public void testBigtableCredentialsAreCheckedOnStartup() throws InterruptedExcep assertThrows(Throwable.class, () -> connect.configureConnector(testId, props)); assertThrows(Throwable.class, () -> connect.connectorStatus(testId)); } + + @org.junit.Ignore // TODO: unignore. For now, the emulator does not cause an exception. + @Test + public void testCreationOfInvalidTable() throws InterruptedException { + String dlqTopic = createDlq(); + Map props = baseConnectorProps(); + String invalidTableName = "T".repeat(10000); + props.put(BigtableSinkConfig.CONFIG_TABLE_NAME_FORMAT, invalidTableName); + props.put(BigtableSinkConfig.CONFIG_AUTO_CREATE_TABLES, String.valueOf(true)); + + configureDlq(props, dlqTopic); + String testId = startSingleTopicConnector(props); + + String key = "key"; + String value = "value"; + + connect.kafka().produce(testId, key, value); + + ConsumerRecords dlqRecords = + connect.kafka().consume(1, Duration.ofSeconds(120).toMillis(), dlqTopic); + assertEquals(1, dlqRecords.count()); + ConsumerRecord record = dlqRecords.iterator().next(); + assertArrayEquals(record.key(), key.getBytes(StandardCharsets.UTF_8)); + assertArrayEquals(record.value(), value.getBytes(StandardCharsets.UTF_8)); + assertTrue( + Arrays.stream(record.headers().toArray()) + .anyMatch( + h -> + h.key().equals(DeadLetterQueueReporter.ERROR_HEADER_EXCEPTION) + && Arrays.equals( + h.value(), + InvalidBigtableSchemaModificationException.class + .getName() + .getBytes(StandardCharsets.UTF_8)))); + connect + .assertions() + .assertConnectorAndExactlyNumTasksAreRunning( + testId, numTasks, "Wrong number of tasks is running."); + } + + @org.junit.Ignore // TODO: unignore. For now, the emulator does not cause an exception. + @Test + public void testTooLargeData() throws InterruptedException, ExecutionException { + String dlqTopic = createDlq(); + Map props = baseConnectorProps(); + props.put(BigtableSinkConfig.CONFIG_AUTO_CREATE_TABLES, String.valueOf(true)); + props.put(BigtableSinkConfig.CONFIG_AUTO_CREATE_COLUMN_FAMILIES, String.valueOf(true)); + props.put(ConnectorConfig.VALUE_CONVERTER_CLASS_CONFIG, ByteArrayConverter.class.getName()); + configureDlq(props, dlqTopic); + String testId = startSingleTopicConnector(props); + + byte[] key = "key".getBytes(StandardCharsets.UTF_8); + // The hard limit is 100 MB as per https://cloud.google.com/bigtable/quotas#limits-data-size + int twoHundredMegabytes = 200 * 1000 * 1000; + byte[] value = new byte[twoHundredMegabytes]; + getKafkaProducer().send(new ProducerRecord<>(testId, key, value)).get(); + + ConsumerRecords dlqRecords = + connect.kafka().consume(1, Duration.ofSeconds(120).toMillis(), dlqTopic); + assertEquals(1, dlqRecords.count()); + ConsumerRecord record = dlqRecords.iterator().next(); + assertArrayEquals(record.key(), key); + assertArrayEquals(record.value(), value); + connect + .assertions() + .assertConnectorAndExactlyNumTasksAreRunning( + testId, numTasks, "Wrong number of tasks is running."); + } + + @Test + public void testSecondInsertIntoARowCausesAnError() throws InterruptedException { + String dlqTopic = createDlq(); + Map props = baseConnectorProps(); + props.put(BigtableSinkConfig.CONFIG_AUTO_CREATE_TABLES, String.valueOf(true)); + props.put(BigtableSinkConfig.CONFIG_AUTO_CREATE_COLUMN_FAMILIES, String.valueOf(true)); + props.put(BigtableSinkConfig.CONFIG_INSERT_MODE, InsertMode.INSERT.name()); + configureDlq(props, dlqTopic); + String testId = startSingleTopicConnector(props); + + String key = "key"; + String valueOk = "ok"; + String valueRejected = "rejected"; + + connect.kafka().produce(testId, key, valueOk); + + waitUntilBigtableContainsNumberOfRows(testId, 1); + Map rows = readAllRows(bigtableData, testId); + assertEquals(1, rows.size()); + Row rowOk = rows.get(ByteString.copyFrom(key.getBytes(StandardCharsets.UTF_8))); + assertEquals(1, rowOk.getCells().size()); + assertArrayEquals( + valueOk.getBytes(StandardCharsets.UTF_8), rowOk.getCells().get(0).getValue().toByteArray()); + + connect.kafka().produce(testId, key, valueRejected); + ConsumerRecords dlqRecords = + connect.kafka().consume(1, Duration.ofSeconds(120).toMillis(), dlqTopic); + assertEquals(1, dlqRecords.count()); + ConsumerRecord record = dlqRecords.iterator().next(); + assertArrayEquals(record.key(), key.getBytes(StandardCharsets.UTF_8)); + assertArrayEquals(record.value(), valueRejected.getBytes(StandardCharsets.UTF_8)); + } + + @Test + public void testPartialBatchError() throws InterruptedException { + long dataSize = 1000; + + String dlqTopic = createDlq(); + Map props = baseConnectorProps(); + props.put(BigtableSinkConfig.CONFIG_AUTO_CREATE_TABLES, String.valueOf(true)); + props.put(BigtableSinkConfig.CONFIG_AUTO_CREATE_COLUMN_FAMILIES, String.valueOf(true)); + props.put(BigtableSinkConfig.CONFIG_INSERT_MODE, InsertMode.INSERT.name()); + props.put( + ConnectorConfig.CONNECTOR_CLIENT_CONSUMER_OVERRIDES_PREFIX + + ConsumerConfig.MAX_POLL_RECORDS_CONFIG, + Long.toString(dataSize)); + props.put( + ConnectorConfig.CONNECTOR_CLIENT_CONSUMER_OVERRIDES_PREFIX + + ConsumerConfig.MAX_PARTITION_FETCH_BYTES_CONFIG, + Integer.toString(Integer.MAX_VALUE)); + props.put( + ConnectorConfig.CONNECTOR_CLIENT_CONSUMER_OVERRIDES_PREFIX + + ConsumerConfig.FETCH_MAX_BYTES_CONFIG, + Integer.toString(Integer.MAX_VALUE)); + configureDlq(props, dlqTopic); + String testId = startSingleTopicConnector(props); + + connect.pauseConnector(testId); + List> keysAndValues = new ArrayList<>(); + // Every second record fails since every two consecutive records share a key and we use insert + // mode. + Function keyGenerator = i -> "key" + (i / 2); + Function valueGenerator = i -> "value" + i; + for (long i = 0; i < dataSize; i++) { + SchemaAndValue key = new SchemaAndValue(Schema.STRING_SCHEMA, keyGenerator.apply(i)); + SchemaAndValue value = new SchemaAndValue(Schema.STRING_SCHEMA, valueGenerator.apply(i)); + keysAndValues.add(new AbstractMap.SimpleImmutableEntry<>(key, value)); + } + sendRecords(testId, keysAndValues, new StringConverter(), new StringConverter()); + connect.resumeConnector(testId); + + waitUntilBigtableContainsNumberOfRows(testId, dataSize / 2); + + Map rows = readAllRows(bigtableData, testId); + assertEquals(dataSize / 2, rows.size()); + + ConsumerRecords dlqRecords = + connect.kafka().consume((int) dataSize / 2, Duration.ofSeconds(120).toMillis(), dlqTopic); + Map dlqValues = new HashMap<>(); + for (ConsumerRecord r : dlqRecords) { + ByteString key = ByteString.copyFrom(r.key()); + dlqValues.put(key, r.value()); + } + assertEquals(dataSize / 2, dlqValues.size()); + + // TODO: consider if we want to fix the fact that we do not know the order the batch records + // will be executed in since we use unordered hashmaps in BigtableSinkTask and + // BigtableSchemaManager. It also makes batch size related setting unintuitive. If we fix it, + // then we can assert against values here. + for (long i = 0; i < dataSize; i += 2) { + ByteString key = ByteString.copyFrom(keyGenerator.apply(i).getBytes(StandardCharsets.UTF_8)); + assertTrue(rows.containsKey(key)); + assertTrue(dlqValues.containsKey(key)); + } + } } diff --git a/google-cloud-bigtable-kafka-connect-sink/src/test/java/com/google/cloud/kafka/connect/bigtable/integration/ErrorReportingIT.java b/google-cloud-bigtable-kafka-connect-sink/src/test/java/com/google/cloud/kafka/connect/bigtable/integration/ErrorReportingIT.java index 9e5c624c8b..54ebfef47e 100644 --- a/google-cloud-bigtable-kafka-connect-sink/src/test/java/com/google/cloud/kafka/connect/bigtable/integration/ErrorReportingIT.java +++ b/google-cloud-bigtable-kafka-connect-sink/src/test/java/com/google/cloud/kafka/connect/bigtable/integration/ErrorReportingIT.java @@ -15,6 +15,7 @@ */ package com.google.cloud.kafka.connect.bigtable.integration; +import static org.apache.kafka.connect.runtime.errors.DeadLetterQueueReporter.ERROR_HEADER_EXCEPTION; import static org.junit.Assert.assertArrayEquals; import static org.junit.Assert.assertEquals; import static org.junit.Assert.assertTrue; @@ -88,10 +89,10 @@ public void testErrorModeDLQOverridesErrorMode() throws InterruptedException { assertEquals(1, dlqRecords.count()); ConsumerRecord record = dlqRecords.iterator().next(); assertArrayEquals(record.key(), key.getBytes(StandardCharsets.UTF_8)); - assertArrayEquals(record.key(), key.getBytes(StandardCharsets.UTF_8)); + assertArrayEquals(record.value(), value.getBytes(StandardCharsets.UTF_8)); assertTrue( Arrays.stream(record.headers().toArray()) - .anyMatch(h -> h.key().equals("__connect.errors.exception.class" + ".name"))); + .anyMatch(h -> h.key().equals(ERROR_HEADER_EXCEPTION))); connect .assertions() .assertConnectorAndExactlyNumTasksAreRunning( From 21b244557a41c46b89a2776dfa2a84ab62faa689 Mon Sep 17 00:00:00 2001 From: Adam Czajkowski Date: Mon, 3 Feb 2025 14:38:38 +0100 Subject: [PATCH 31/76] Sort input records for predictable batching behavior --- .../connect/bigtable/BigtableSinkTask.java | 29 ++++++++++ .../exception/BigtableSinkLogicError.java | 25 ++++++++ .../bigtable/BigtableSinkTaskTest.java | 57 +++++++++++++++++++ .../bigtable/integration/ErrorHandlingIT.java | 21 ++++--- 4 files changed, 125 insertions(+), 7 deletions(-) create mode 100644 google-cloud-bigtable-kafka-connect-sink/src/main/java/com/google/cloud/kafka/connect/bigtable/exception/BigtableSinkLogicError.java diff --git a/google-cloud-bigtable-kafka-connect-sink/src/main/java/com/google/cloud/kafka/connect/bigtable/BigtableSinkTask.java b/google-cloud-bigtable-kafka-connect-sink/src/main/java/com/google/cloud/kafka/connect/bigtable/BigtableSinkTask.java index 4e436b3898..b63796b9fe 100644 --- a/google-cloud-bigtable-kafka-connect-sink/src/main/java/com/google/cloud/kafka/connect/bigtable/BigtableSinkTask.java +++ b/google-cloud-bigtable-kafka-connect-sink/src/main/java/com/google/cloud/kafka/connect/bigtable/BigtableSinkTask.java @@ -31,6 +31,7 @@ import com.google.cloud.kafka.connect.bigtable.config.BigtableSinkTaskConfig; import com.google.cloud.kafka.connect.bigtable.config.ConfigInterpolation; import com.google.cloud.kafka.connect.bigtable.exception.BatchException; +import com.google.cloud.kafka.connect.bigtable.exception.BigtableSinkLogicError; import com.google.cloud.kafka.connect.bigtable.exception.InvalidBigtableSchemaModificationException; import com.google.cloud.kafka.connect.bigtable.mapping.KeyMapper; import com.google.cloud.kafka.connect.bigtable.mapping.MutationData; @@ -43,6 +44,7 @@ import java.util.ArrayList; import java.util.Collection; import java.util.HashMap; +import java.util.LinkedHashMap; import java.util.List; import java.util.Map; import java.util.Optional; @@ -172,6 +174,7 @@ public void put(Collection records) { if (config.getBoolean(BigtableSinkTaskConfig.CONFIG_AUTO_CREATE_COLUMN_FAMILIES)) { mutations = autoCreateColumnFamiliesAndHandleErrors(mutations); } + mutations = orderMap(mutations, records); Map> perRecordResults = new HashMap<>(); switch (config.getInsertMode()) { @@ -305,6 +308,32 @@ void reportError(SinkRecord record, Throwable throwable) { } } + /** + * Generates a {@link Map} with desired key ordering. + * + * @param map A {@link Map} to be sorted. + * @param order A {@link Collection} defining desired order of the output {@link Map}. Must be a + * superset of {@code mutations}'s key set. + * @return A {@link Map} with the same keys and corresponding values as {@code map} with the same + * key ordering as {@code order}. + */ + @VisibleForTesting + // It is generic so that we can test it with naturally ordered values easily. + static Map orderMap(Map map, Collection order) { + if (!order.containsAll(map.keySet())) { + throw new BigtableSinkLogicError( + "A collection defining order of keys must be a superset of the input map's key set."); + } + Map sorted = new LinkedHashMap<>(); + for (K key : order) { + V value = map.get(key); + if (value != null) { + sorted.put(key, value); + } + } + return sorted; + } + /** * Attempts to create Cloud Bigtable tables so that all the mutations can be applied and handles * errors. diff --git a/google-cloud-bigtable-kafka-connect-sink/src/main/java/com/google/cloud/kafka/connect/bigtable/exception/BigtableSinkLogicError.java b/google-cloud-bigtable-kafka-connect-sink/src/main/java/com/google/cloud/kafka/connect/bigtable/exception/BigtableSinkLogicError.java new file mode 100644 index 0000000000..6009d505dc --- /dev/null +++ b/google-cloud-bigtable-kafka-connect-sink/src/main/java/com/google/cloud/kafka/connect/bigtable/exception/BigtableSinkLogicError.java @@ -0,0 +1,25 @@ +/* + * Copyright 2025 Google LLC + * + * Licensed under the Apache License, Version 2.0 (the "License"); + * you may not use this file except in compliance with the License. + * You may obtain a copy of the License at + * + * https://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ +package com.google.cloud.kafka.connect.bigtable.exception; + +import org.apache.kafka.connect.errors.ConnectException; + +/** A wrapper exception class that may be thrown to explicitly signify a logic bug in this sink. */ +public class BigtableSinkLogicError extends ConnectException { + public BigtableSinkLogicError(String errorMessage) { + super(errorMessage); + } +} diff --git a/google-cloud-bigtable-kafka-connect-sink/src/test/java/com/google/cloud/kafka/connect/bigtable/BigtableSinkTaskTest.java b/google-cloud-bigtable-kafka-connect-sink/src/test/java/com/google/cloud/kafka/connect/bigtable/BigtableSinkTaskTest.java index 77cfcf9dc5..0dd944ea4e 100644 --- a/google-cloud-bigtable-kafka-connect-sink/src/test/java/com/google/cloud/kafka/connect/bigtable/BigtableSinkTaskTest.java +++ b/google-cloud-bigtable-kafka-connect-sink/src/test/java/com/google/cloud/kafka/connect/bigtable/BigtableSinkTaskTest.java @@ -56,6 +56,7 @@ import com.google.cloud.kafka.connect.bigtable.config.BigtableSinkTaskConfig; import com.google.cloud.kafka.connect.bigtable.config.InsertMode; import com.google.cloud.kafka.connect.bigtable.config.NullValueMode; +import com.google.cloud.kafka.connect.bigtable.exception.BigtableSinkLogicError; import com.google.cloud.kafka.connect.bigtable.exception.InvalidBigtableSchemaModificationException; import com.google.cloud.kafka.connect.bigtable.mapping.KeyMapper; import com.google.cloud.kafka.connect.bigtable.mapping.MutationData; @@ -66,6 +67,7 @@ import com.google.cloud.kafka.connect.bigtable.util.FutureUtil; import com.google.protobuf.ByteString; import java.nio.charset.StandardCharsets; +import java.util.AbstractMap; import java.util.ArrayList; import java.util.Arrays; import java.util.Collections; @@ -80,6 +82,7 @@ import java.util.concurrent.Future; import java.util.stream.Collectors; import java.util.stream.IntStream; +import org.apache.commons.collections4.iterators.PermutationIterator; import org.apache.kafka.common.record.TimestampType; import org.apache.kafka.connect.errors.ConnectException; import org.apache.kafka.connect.sink.ErrantRecordReporter; @@ -307,6 +310,60 @@ public void testPrepareRecords() { assertTotalNumberOfInvocations(errorReporter, 1); } + @Test + public void testOrderMapSuccesses() { + Integer key1 = 1; + Integer key2 = 2; + Integer key3 = 3; + Integer key4 = 4; + + String value1 = "value1"; + String value2 = "value2"; + String value3 = "value3"; + String value4 = "value4"; + + Map map1 = new LinkedHashMap<>(); + map1.put(key4, value4); + map1.put(key3, value3); + map1.put(key2, value2); + map1.put(key1, value1); + + assertEquals(List.of(key4, key3, key2, key1), new ArrayList<>(map1.keySet())); + assertEquals(List.of(value4, value3, value2, value1), new ArrayList<>(map1.values())); + assertEquals( + List.of(key1, key2, key3, key4), + new ArrayList<>(BigtableSinkTask.orderMap(map1, List.of(key1, key2, key3, key4)).keySet())); + assertEquals( + List.of(value1, value2, value3, value4), + new ArrayList<>(BigtableSinkTask.orderMap(map1, List.of(key1, key2, key3, key4)).values())); + assertEquals( + List.of( + new AbstractMap.SimpleImmutableEntry<>(key1, value1), + new AbstractMap.SimpleImmutableEntry<>(key2, value2), + new AbstractMap.SimpleImmutableEntry<>(key3, value3), + new AbstractMap.SimpleImmutableEntry<>(key4, value4)), + new ArrayList<>( + BigtableSinkTask.orderMap(map1, List.of(key1, key2, key3, key4)).entrySet())); + + assertEquals( + List.of(key1, key2, key3, key4), + new ArrayList<>( + BigtableSinkTask.orderMap(map1, List.of(-1, key1, -2, key2, -3, key3, -4, key4, -5)) + .keySet())); + + PermutationIterator permutations = + new PermutationIterator<>(List.of(key1, key2, key3, key4)); + permutations.forEachRemaining( + p -> assertEquals(p, new ArrayList<>(BigtableSinkTask.orderMap(map1, p).keySet()))); + } + + @Test + public void testOrderMapError() { + Map map = Map.of(1, "1", 2, "2", -1, "-1"); + assertThrows( + BigtableSinkLogicError.class, () -> BigtableSinkTask.orderMap(map, Set.of(1, 2))); + } + @Test public void testAutoCreateTablesAndHandleErrors() { task = spy(new TestBigtableSinkTask(null, null, null, null, null, schemaManager, context)); diff --git a/google-cloud-bigtable-kafka-connect-sink/src/test/java/com/google/cloud/kafka/connect/bigtable/integration/ErrorHandlingIT.java b/google-cloud-bigtable-kafka-connect-sink/src/test/java/com/google/cloud/kafka/connect/bigtable/integration/ErrorHandlingIT.java index fa2ff0b2bc..7afef13967 100644 --- a/google-cloud-bigtable-kafka-connect-sink/src/test/java/com/google/cloud/kafka/connect/bigtable/integration/ErrorHandlingIT.java +++ b/google-cloud-bigtable-kafka-connect-sink/src/test/java/com/google/cloud/kafka/connect/bigtable/integration/ErrorHandlingIT.java @@ -21,6 +21,7 @@ import static org.junit.Assert.assertTrue; import com.google.cloud.bigtable.data.v2.models.Row; +import com.google.cloud.bigtable.data.v2.models.RowCell; import com.google.cloud.kafka.connect.bigtable.config.BigtableSinkConfig; import com.google.cloud.kafka.connect.bigtable.config.InsertMode; import com.google.cloud.kafka.connect.bigtable.exception.InvalidBigtableSchemaModificationException; @@ -163,8 +164,8 @@ public void testSecondInsertIntoARowCausesAnError() throws InterruptedException } @Test - public void testPartialBatchError() throws InterruptedException { - long dataSize = 1000; + public void testPartialBatchErrorWhenRelyingOnInputOrdering() throws InterruptedException { + long dataSize = 10000; String dlqTopic = createDlq(); Map props = baseConnectorProps(); @@ -189,7 +190,7 @@ public void testPartialBatchError() throws InterruptedException { connect.pauseConnector(testId); List> keysAndValues = new ArrayList<>(); // Every second record fails since every two consecutive records share a key and we use insert - // mode. + // mode. We rely on max batch size of 1 to know which of the records is going to fail. Function keyGenerator = i -> "key" + (i / 2); Function valueGenerator = i -> "value" + i; for (long i = 0; i < dataSize; i++) { @@ -214,14 +215,20 @@ public void testPartialBatchError() throws InterruptedException { } assertEquals(dataSize / 2, dlqValues.size()); - // TODO: consider if we want to fix the fact that we do not know the order the batch records - // will be executed in since we use unordered hashmaps in BigtableSinkTask and - // BigtableSchemaManager. It also makes batch size related setting unintuitive. If we fix it, - // then we can assert against values here. for (long i = 0; i < dataSize; i += 2) { ByteString key = ByteString.copyFrom(keyGenerator.apply(i).getBytes(StandardCharsets.UTF_8)); + byte[] expectedValue = valueGenerator.apply(i).getBytes(StandardCharsets.UTF_8); + byte[] value; + if (i % 2 == 0) { + List cells = rows.get(key).getCells(); + assertEquals(1, cells.size()); + value = cells.get(0).getValue().toByteArray(); + } else { + value = dlqValues.get(key); + } assertTrue(rows.containsKey(key)); assertTrue(dlqValues.containsKey(key)); + assertArrayEquals(expectedValue, value); } } } From 4e45d564913824686fe6e9831e24a9d1237867e6 Mon Sep 17 00:00:00 2001 From: Adam Czajkowski Date: Mon, 3 Feb 2025 16:22:18 +0100 Subject: [PATCH 32/76] Error handling and resource auto creation integration tests (and some integration test refactors) --- .../connect/bigtable/BigtableSinkTask.java | 1 + .../bigtable/BigtableSinkTaskTest.java | 3 +- .../connect/bigtable/integration/BaseIT.java | 44 +- .../BaseKafkaConnectBigtableIT.java | 26 +- .../integration/BaseKafkaConnectIT.java | 66 +++ .../integration/ConfluentCompatibilityIT.java | 16 - .../bigtable/integration/ErrorHandlingIT.java | 177 +++++--- .../integration/ErrorReportingIT.java | 25 +- .../integration/ResourceAutoCreationIT.java | 418 +++++++++++++++--- .../kafka/connect/bigtable/util/TestId.java | 2 +- 10 files changed, 608 insertions(+), 170 deletions(-) diff --git a/google-cloud-bigtable-kafka-connect-sink/src/main/java/com/google/cloud/kafka/connect/bigtable/BigtableSinkTask.java b/google-cloud-bigtable-kafka-connect-sink/src/main/java/com/google/cloud/kafka/connect/bigtable/BigtableSinkTask.java index b63796b9fe..4f9985b1cf 100644 --- a/google-cloud-bigtable-kafka-connect-sink/src/main/java/com/google/cloud/kafka/connect/bigtable/BigtableSinkTask.java +++ b/google-cloud-bigtable-kafka-connect-sink/src/main/java/com/google/cloud/kafka/connect/bigtable/BigtableSinkTask.java @@ -174,6 +174,7 @@ public void put(Collection records) { if (config.getBoolean(BigtableSinkTaskConfig.CONFIG_AUTO_CREATE_COLUMN_FAMILIES)) { mutations = autoCreateColumnFamiliesAndHandleErrors(mutations); } + // Needed so that the batch ordering is more predictable from the user's point of view. mutations = orderMap(mutations, records); Map> perRecordResults = new HashMap<>(); diff --git a/google-cloud-bigtable-kafka-connect-sink/src/test/java/com/google/cloud/kafka/connect/bigtable/BigtableSinkTaskTest.java b/google-cloud-bigtable-kafka-connect-sink/src/test/java/com/google/cloud/kafka/connect/bigtable/BigtableSinkTaskTest.java index 0dd944ea4e..eab9857aa4 100644 --- a/google-cloud-bigtable-kafka-connect-sink/src/test/java/com/google/cloud/kafka/connect/bigtable/BigtableSinkTaskTest.java +++ b/google-cloud-bigtable-kafka-connect-sink/src/test/java/com/google/cloud/kafka/connect/bigtable/BigtableSinkTaskTest.java @@ -360,8 +360,7 @@ public void testOrderMapSuccesses() { @Test public void testOrderMapError() { Map map = Map.of(1, "1", 2, "2", -1, "-1"); - assertThrows( - BigtableSinkLogicError.class, () -> BigtableSinkTask.orderMap(map, Set.of(1, 2))); + assertThrows(BigtableSinkLogicError.class, () -> BigtableSinkTask.orderMap(map, Set.of(1, 2))); } @Test diff --git a/google-cloud-bigtable-kafka-connect-sink/src/test/java/com/google/cloud/kafka/connect/bigtable/integration/BaseIT.java b/google-cloud-bigtable-kafka-connect-sink/src/test/java/com/google/cloud/kafka/connect/bigtable/integration/BaseIT.java index 7d7434cd79..34d6b8abc9 100644 --- a/google-cloud-bigtable-kafka-connect-sink/src/test/java/com/google/cloud/kafka/connect/bigtable/integration/BaseIT.java +++ b/google-cloud-bigtable-kafka-connect-sink/src/test/java/com/google/cloud/kafka/connect/bigtable/integration/BaseIT.java @@ -30,14 +30,18 @@ import com.google.cloud.kafka.connect.bigtable.config.BigtableSinkConfig; import com.google.cloud.kafka.connect.bigtable.util.TestId; import com.google.protobuf.ByteString; +import java.util.Collections; import java.util.HashMap; import java.util.Map; import java.util.Properties; +import java.util.Set; import java.util.stream.Collectors; +import org.apache.commons.lang.StringUtils; import org.apache.kafka.clients.admin.Admin; import org.apache.kafka.clients.consumer.ConsumerConfig; import org.apache.kafka.clients.producer.ProducerConfig; import org.apache.kafka.common.utils.Utils; +import org.apache.kafka.connect.runtime.ConnectorConfig; import org.apache.kafka.connect.runtime.SinkConnectorConfig; import org.apache.kafka.connect.runtime.WorkerConfig; import org.apache.kafka.connect.runtime.isolation.PluginDiscoveryMode; @@ -47,6 +51,9 @@ import org.slf4j.LoggerFactory; public abstract class BaseIT { + // https://cloud.google.com/bigtable/docs/reference/admin/rpc/google.bigtable.admin.v2#createtablerequest + public static int MAX_BIGTABLE_TABLE_NAME_LENGTH = 50; + private final Logger logger = LoggerFactory.getLogger(BaseIT.class); protected EmbeddedConnectCluster connect; private Admin kafkaAdminClient; @@ -117,6 +124,16 @@ protected Map baseConnectorProps() { result.put(TASKS_MAX_CONFIG, Integer.toString(numTasks)); result.put(KEY_CONVERTER_CLASS_CONFIG, StringConverter.class.getName()); result.put(VALUE_CONVERTER_CLASS_CONFIG, StringConverter.class.getName()); + // Needed so that all messages we send to the input topics can be also sent to the DLQ by + // DeadLetterQueueReporter. + result.put( + ConnectorConfig.CONNECTOR_CLIENT_PRODUCER_OVERRIDES_PREFIX + + ProducerConfig.MAX_REQUEST_SIZE_CONFIG, + String.valueOf(maxKafkaMessageSizeBytes)); + result.put( + ConnectorConfig.CONNECTOR_CLIENT_PRODUCER_OVERRIDES_PREFIX + + ProducerConfig.BUFFER_MEMORY_CONFIG, + String.valueOf(maxKafkaMessageSizeBytes)); // TODO: get it from environment variables after migrating to kokoro. result.put(CONFIG_GCP_PROJECT_ID, "todotodo"); @@ -148,9 +165,30 @@ protected String getTestCaseId() { protected String startSingleTopicConnector(Map configProps) throws InterruptedException { - String id = getTestCaseId() + System.currentTimeMillis(); - configProps.put(SinkConnectorConfig.TOPICS_CONFIG, id); - connect.kafka().createTopic(id, numBrokers); + return startConnector(configProps, Collections.emptySet()); + } + + protected String startMultipleTopicConnector( + Map configProps, Set topicNameSuffixes) throws InterruptedException { + return startConnector(configProps, topicNameSuffixes); + } + + private String startConnector(Map configProps, Set topicNameSuffixes) + throws InterruptedException { + int longestSuffix = topicNameSuffixes.stream().mapToInt(String::length).max().orElse(0); + String id = + StringUtils.right( + getTestCaseId() + System.currentTimeMillis(), + MAX_BIGTABLE_TABLE_NAME_LENGTH - longestSuffix); + if (topicNameSuffixes.isEmpty()) { + configProps.put(SinkConnectorConfig.TOPICS_CONFIG, id); + connect.kafka().createTopic(id, numBrokers); + } else { + configProps.put(SinkConnectorConfig.TOPICS_REGEX_CONFIG, id + ".*"); + for (String suffix : topicNameSuffixes) { + connect.kafka().createTopic(id + suffix, numBrokers); + } + } connect.configureConnector(id, configProps); connect .assertions() diff --git a/google-cloud-bigtable-kafka-connect-sink/src/test/java/com/google/cloud/kafka/connect/bigtable/integration/BaseKafkaConnectBigtableIT.java b/google-cloud-bigtable-kafka-connect-sink/src/test/java/com/google/cloud/kafka/connect/bigtable/integration/BaseKafkaConnectBigtableIT.java index 2a64e0b8ff..f379549973 100644 --- a/google-cloud-bigtable-kafka-connect-sink/src/test/java/com/google/cloud/kafka/connect/bigtable/integration/BaseKafkaConnectBigtableIT.java +++ b/google-cloud-bigtable-kafka-connect-sink/src/test/java/com/google/cloud/kafka/connect/bigtable/integration/BaseKafkaConnectBigtableIT.java @@ -24,7 +24,9 @@ import org.junit.Before; public abstract class BaseKafkaConnectBigtableIT extends BaseKafkaConnectIT { - public static long CONSUME_TIMEOUT_MILLIS = 30000; + // Not copied from BigtableSinkConfig since it isn't present in its public API. + public static long DEFAULT_BIGTABLE_RETRY_TIMEOUT_MILLIS = 90000; + protected BigtableDataClient bigtableData; protected BigtableTableAdminClient bigtableAdmin; @@ -49,7 +51,27 @@ public void waitUntilBigtableContainsNumberOfRows(String tableId, long numberOfR throws InterruptedException { waitForCondition( () -> readAllRows(bigtableData, tableId).size() == numberOfRows, - CONSUME_TIMEOUT_MILLIS, + DEFAULT_BIGTABLE_RETRY_TIMEOUT_MILLIS, "Records not consumed in time."); } + + public void waitUntilBigtableTableExists(String tableId) throws InterruptedException { + waitForCondition( + () -> { + bigtableAdmin.getTable(tableId); + return true; + }, + DEFAULT_BIGTABLE_RETRY_TIMEOUT_MILLIS, + "Table not created in time."); + } + + public void waitUntilBigtableTableHasColumnFamily(String tableId, String columnFamily) + throws InterruptedException { + waitForCondition( + () -> + bigtableAdmin.getTable(tableId).getColumnFamilies().stream() + .anyMatch(cf -> cf.getId().equals(columnFamily)), + DEFAULT_BIGTABLE_RETRY_TIMEOUT_MILLIS, + "Column Family not created in time."); + } } diff --git a/google-cloud-bigtable-kafka-connect-sink/src/test/java/com/google/cloud/kafka/connect/bigtable/integration/BaseKafkaConnectIT.java b/google-cloud-bigtable-kafka-connect-sink/src/test/java/com/google/cloud/kafka/connect/bigtable/integration/BaseKafkaConnectIT.java index d816abefe9..fbd4580c81 100644 --- a/google-cloud-bigtable-kafka-connect-sink/src/test/java/com/google/cloud/kafka/connect/bigtable/integration/BaseKafkaConnectIT.java +++ b/google-cloud-bigtable-kafka-connect-sink/src/test/java/com/google/cloud/kafka/connect/bigtable/integration/BaseKafkaConnectIT.java @@ -15,18 +15,31 @@ */ package com.google.cloud.kafka.connect.bigtable.integration; +import static org.junit.Assert.assertArrayEquals; +import static org.junit.Assert.assertEquals; +import static org.junit.Assert.assertTrue; + +import java.nio.charset.StandardCharsets; +import java.time.Duration; import java.util.ArrayList; +import java.util.Arrays; import java.util.Collection; import java.util.Collections; import java.util.List; import java.util.Map; +import java.util.Optional; +import java.util.concurrent.ExecutionException; import java.util.concurrent.Future; import java.util.concurrent.TimeUnit; +import java.util.concurrent.TimeoutException; +import org.apache.kafka.clients.consumer.ConsumerRecord; +import org.apache.kafka.clients.consumer.ConsumerRecords; import org.apache.kafka.clients.producer.KafkaProducer; import org.apache.kafka.clients.producer.ProducerRecord; import org.apache.kafka.clients.producer.RecordMetadata; import org.apache.kafka.connect.data.SchemaAndValue; import org.apache.kafka.connect.runtime.SinkConnectorConfig; +import org.apache.kafka.connect.runtime.errors.DeadLetterQueueReporter; import org.apache.kafka.connect.runtime.errors.ToleranceType; import org.apache.kafka.connect.storage.Converter; import org.junit.After; @@ -91,4 +104,57 @@ public void configureDlq(Map props, String dlqTopic) { props.put(SinkConnectorConfig.DLQ_TOPIC_REPLICATION_FACTOR_CONFIG, String.valueOf(numBrokers)); props.put(SinkConnectorConfig.ERRORS_TOLERANCE_CONFIG, ToleranceType.ALL.value()); } + + public void assertDlqIsEmpty(String dlqTopic) + throws ExecutionException, InterruptedException, TimeoutException { + ConsumerRecords dlqRecords = + connect + .kafka() + .consumeAll( + Duration.ofSeconds(15).toMillis(), + Collections.emptyMap(), + Collections.emptyMap(), + dlqTopic); + assertEquals(0, dlqRecords.count()); + } + + public void assertConnectorAndAllTasksAreRunning(String connectorId) throws InterruptedException { + connect + .assertions() + .assertConnectorAndExactlyNumTasksAreRunning( + connectorId, numTasks, "Wrong number of tasks is running."); + } + + public void assertSingleDlqEntry( + String dlqTopic, String key, String value, Class exceptionClass) { + assertSingleDlqEntry( + dlqTopic, + Optional.ofNullable(key).map(s -> s.getBytes(StandardCharsets.UTF_8)).orElse(null), + Optional.ofNullable(value).map(s -> s.getBytes(StandardCharsets.UTF_8)).orElse(null), + exceptionClass); + } + + public void assertSingleDlqEntry( + String dlqTopic, byte[] key, byte[] value, Class exceptionClass) { + ConsumerRecords dlqRecords = + connect.kafka().consume(1, Duration.ofSeconds(120).toMillis(), dlqTopic); + assertEquals(1, dlqRecords.count()); + ConsumerRecord record = dlqRecords.iterator().next(); + if (key != null) { + assertArrayEquals(record.key(), key); + } + if (value != null) { + assertArrayEquals(record.value(), value); + } + if (exceptionClass != null) { + assertTrue( + Arrays.stream(record.headers().toArray()) + .anyMatch( + h -> + h.key().equals(DeadLetterQueueReporter.ERROR_HEADER_EXCEPTION) + && Arrays.equals( + h.value(), + exceptionClass.getName().getBytes(StandardCharsets.UTF_8)))); + } + } } diff --git a/google-cloud-bigtable-kafka-connect-sink/src/test/java/com/google/cloud/kafka/connect/bigtable/integration/ConfluentCompatibilityIT.java b/google-cloud-bigtable-kafka-connect-sink/src/test/java/com/google/cloud/kafka/connect/bigtable/integration/ConfluentCompatibilityIT.java index 8c5f87ff0b..96523db31a 100644 --- a/google-cloud-bigtable-kafka-connect-sink/src/test/java/com/google/cloud/kafka/connect/bigtable/integration/ConfluentCompatibilityIT.java +++ b/google-cloud-bigtable-kafka-connect-sink/src/test/java/com/google/cloud/kafka/connect/bigtable/integration/ConfluentCompatibilityIT.java @@ -57,8 +57,6 @@ import org.apache.kafka.clients.producer.ProducerRecord; import org.apache.kafka.connect.runtime.ConnectorConfig; import org.junit.Test; -import org.junit.jupiter.api.AfterEach; -import org.junit.jupiter.api.BeforeEach; import org.junit.runner.RunWith; import org.junit.runners.Parameterized; @@ -71,20 +69,6 @@ public class ConfluentCompatibilityIT extends BaseKafkaConnectBigtableSchemaRegi private String testCase; private Compatibility compatibility; - @BeforeEach - public void setUp() throws Exception { - setUpConnect(); - setUpBigtable(); - setUpSchemaRegistry(); - } - - @AfterEach - public void tearDown() throws Exception { - tearDownSchemaRegistry(); - tearDownBigtable(); - tearDownConnect(); - } - public ConfluentCompatibilityIT(String testCase, Compatibility compatibility) { this.testCase = testCase; this.compatibility = compatibility; diff --git a/google-cloud-bigtable-kafka-connect-sink/src/test/java/com/google/cloud/kafka/connect/bigtable/integration/ErrorHandlingIT.java b/google-cloud-bigtable-kafka-connect-sink/src/test/java/com/google/cloud/kafka/connect/bigtable/integration/ErrorHandlingIT.java index 7afef13967..bf181c70a8 100644 --- a/google-cloud-bigtable-kafka-connect-sink/src/test/java/com/google/cloud/kafka/connect/bigtable/integration/ErrorHandlingIT.java +++ b/google-cloud-bigtable-kafka-connect-sink/src/test/java/com/google/cloud/kafka/connect/bigtable/integration/ErrorHandlingIT.java @@ -24,17 +24,18 @@ import com.google.cloud.bigtable.data.v2.models.RowCell; import com.google.cloud.kafka.connect.bigtable.config.BigtableSinkConfig; import com.google.cloud.kafka.connect.bigtable.config.InsertMode; -import com.google.cloud.kafka.connect.bigtable.exception.InvalidBigtableSchemaModificationException; +import com.google.cloud.kafka.connect.bigtable.config.NullValueMode; +import com.google.cloud.kafka.connect.bigtable.util.JsonConverterFactory; import com.google.protobuf.ByteString; import java.nio.charset.StandardCharsets; import java.time.Duration; import java.util.AbstractMap; import java.util.ArrayList; -import java.util.Arrays; import java.util.HashMap; import java.util.List; import java.util.Map; import java.util.concurrent.ExecutionException; +import java.util.concurrent.TimeoutException; import java.util.function.Function; import org.apache.kafka.clients.consumer.ConsumerConfig; import org.apache.kafka.clients.consumer.ConsumerRecord; @@ -43,8 +44,11 @@ import org.apache.kafka.connect.converters.ByteArrayConverter; import org.apache.kafka.connect.data.Schema; import org.apache.kafka.connect.data.SchemaAndValue; +import org.apache.kafka.connect.data.SchemaBuilder; +import org.apache.kafka.connect.data.Struct; +import org.apache.kafka.connect.json.JsonConverter; +import org.apache.kafka.connect.json.JsonConverterConfig; import org.apache.kafka.connect.runtime.ConnectorConfig; -import org.apache.kafka.connect.runtime.errors.DeadLetterQueueReporter; import org.apache.kafka.connect.storage.StringConverter; import org.junit.Test; import org.junit.runner.RunWith; @@ -62,46 +66,6 @@ public void testBigtableCredentialsAreCheckedOnStartup() { assertThrows(Throwable.class, () -> connect.connectorStatus(testId)); } - @org.junit.Ignore // TODO: unignore. For now, the emulator does not cause an exception. - @Test - public void testCreationOfInvalidTable() throws InterruptedException { - String dlqTopic = createDlq(); - Map props = baseConnectorProps(); - String invalidTableName = "T".repeat(10000); - props.put(BigtableSinkConfig.CONFIG_TABLE_NAME_FORMAT, invalidTableName); - props.put(BigtableSinkConfig.CONFIG_AUTO_CREATE_TABLES, String.valueOf(true)); - - configureDlq(props, dlqTopic); - String testId = startSingleTopicConnector(props); - - String key = "key"; - String value = "value"; - - connect.kafka().produce(testId, key, value); - - ConsumerRecords dlqRecords = - connect.kafka().consume(1, Duration.ofSeconds(120).toMillis(), dlqTopic); - assertEquals(1, dlqRecords.count()); - ConsumerRecord record = dlqRecords.iterator().next(); - assertArrayEquals(record.key(), key.getBytes(StandardCharsets.UTF_8)); - assertArrayEquals(record.value(), value.getBytes(StandardCharsets.UTF_8)); - assertTrue( - Arrays.stream(record.headers().toArray()) - .anyMatch( - h -> - h.key().equals(DeadLetterQueueReporter.ERROR_HEADER_EXCEPTION) - && Arrays.equals( - h.value(), - InvalidBigtableSchemaModificationException.class - .getName() - .getBytes(StandardCharsets.UTF_8)))); - connect - .assertions() - .assertConnectorAndExactlyNumTasksAreRunning( - testId, numTasks, "Wrong number of tasks is running."); - } - - @org.junit.Ignore // TODO: unignore. For now, the emulator does not cause an exception. @Test public void testTooLargeData() throws InterruptedException, ExecutionException { String dlqTopic = createDlq(); @@ -118,16 +82,8 @@ public void testTooLargeData() throws InterruptedException, ExecutionException { byte[] value = new byte[twoHundredMegabytes]; getKafkaProducer().send(new ProducerRecord<>(testId, key, value)).get(); - ConsumerRecords dlqRecords = - connect.kafka().consume(1, Duration.ofSeconds(120).toMillis(), dlqTopic); - assertEquals(1, dlqRecords.count()); - ConsumerRecord record = dlqRecords.iterator().next(); - assertArrayEquals(record.key(), key); - assertArrayEquals(record.value(), value); - connect - .assertions() - .assertConnectorAndExactlyNumTasksAreRunning( - testId, numTasks, "Wrong number of tasks is running."); + assertSingleDlqEntry(dlqTopic, key, value, null); + assertConnectorAndAllTasksAreRunning(testId); } @Test @@ -136,7 +92,6 @@ public void testSecondInsertIntoARowCausesAnError() throws InterruptedException Map props = baseConnectorProps(); props.put(BigtableSinkConfig.CONFIG_AUTO_CREATE_TABLES, String.valueOf(true)); props.put(BigtableSinkConfig.CONFIG_AUTO_CREATE_COLUMN_FAMILIES, String.valueOf(true)); - props.put(BigtableSinkConfig.CONFIG_INSERT_MODE, InsertMode.INSERT.name()); configureDlq(props, dlqTopic); String testId = startSingleTopicConnector(props); @@ -155,17 +110,14 @@ public void testSecondInsertIntoARowCausesAnError() throws InterruptedException valueOk.getBytes(StandardCharsets.UTF_8), rowOk.getCells().get(0).getValue().toByteArray()); connect.kafka().produce(testId, key, valueRejected); - ConsumerRecords dlqRecords = - connect.kafka().consume(1, Duration.ofSeconds(120).toMillis(), dlqTopic); - assertEquals(1, dlqRecords.count()); - ConsumerRecord record = dlqRecords.iterator().next(); - assertArrayEquals(record.key(), key.getBytes(StandardCharsets.UTF_8)); - assertArrayEquals(record.value(), valueRejected.getBytes(StandardCharsets.UTF_8)); + assertSingleDlqEntry(dlqTopic, key, valueRejected, null); + + assertConnectorAndAllTasksAreRunning(testId); } @Test public void testPartialBatchErrorWhenRelyingOnInputOrdering() throws InterruptedException { - long dataSize = 10000; + long dataSize = 1000; String dlqTopic = createDlq(); Map props = baseConnectorProps(); @@ -215,7 +167,7 @@ public void testPartialBatchErrorWhenRelyingOnInputOrdering() throws Interrupted } assertEquals(dataSize / 2, dlqValues.size()); - for (long i = 0; i < dataSize; i += 2) { + for (long i = 0; i < dataSize; i++) { ByteString key = ByteString.copyFrom(keyGenerator.apply(i).getBytes(StandardCharsets.UTF_8)); byte[] expectedValue = valueGenerator.apply(i).getBytes(StandardCharsets.UTF_8); byte[] value; @@ -230,5 +182,106 @@ public void testPartialBatchErrorWhenRelyingOnInputOrdering() throws Interrupted assertTrue(dlqValues.containsKey(key)); assertArrayEquals(expectedValue, value); } + assertConnectorAndAllTasksAreRunning(testId); + } + + @Test + public void testDeletingARowTwiceWorks() + throws InterruptedException, ExecutionException, TimeoutException { + String dlqTopic = createDlq(); + Map props = baseConnectorProps(); + props.put(BigtableSinkConfig.CONFIG_AUTO_CREATE_TABLES, String.valueOf(true)); + props.put(BigtableSinkConfig.CONFIG_AUTO_CREATE_COLUMN_FAMILIES, String.valueOf(true)); + props.put(BigtableSinkConfig.CONFIG_VALUE_NULL_MODE, NullValueMode.DELETE.name()); + props.put(BigtableSinkConfig.CONFIG_INSERT_MODE, InsertMode.UPSERT.name()); + configureDlq(props, dlqTopic); + props.put(ConnectorConfig.VALUE_CONVERTER_CLASS_CONFIG, JsonConverter.class.getName()); + props.put( + ConnectorConfig.VALUE_CONVERTER_CLASS_CONFIG + + "." + + JsonConverterConfig.SCHEMAS_ENABLE_CONFIG, + String.valueOf(false)); + String testId = startSingleTopicConnector(props); + + String key = "key"; + String putValue = "1"; + String deleteValue = "null"; + + connect.kafka().produce(testId, key, putValue); + waitUntilBigtableContainsNumberOfRows(testId, 1); + assertEquals( + key, + new String( + bigtableData.readRow(testId, key).getKey().toByteArray(), StandardCharsets.UTF_8)); + + connect.kafka().produce(testId, key, deleteValue); + waitUntilBigtableContainsNumberOfRows(testId, 0); + assertTrue(readAllRows(bigtableData, testId).isEmpty()); + + connect.kafka().produce(testId, key, deleteValue); + assertDlqIsEmpty(dlqTopic); + assertConnectorAndAllTasksAreRunning(testId); + } + + @Test + public void testNonexistentCellDeletionWorks() + throws InterruptedException, ExecutionException, TimeoutException { + String dlqTopic = createDlq(); + Map props = baseConnectorProps(); + props.put(BigtableSinkConfig.CONFIG_AUTO_CREATE_TABLES, String.valueOf(true)); + props.put(BigtableSinkConfig.CONFIG_AUTO_CREATE_COLUMN_FAMILIES, String.valueOf(true)); + props.put(BigtableSinkConfig.CONFIG_VALUE_NULL_MODE, NullValueMode.DELETE.name()); + props.put(BigtableSinkConfig.CONFIG_INSERT_MODE, InsertMode.UPSERT.name()); + configureDlq(props, dlqTopic); + props.put(ConnectorConfig.VALUE_CONVERTER_CLASS_CONFIG, JsonConverter.class.getName()); + props.put( + ConnectorConfig.VALUE_CONVERTER_CLASS_CONFIG + + "." + + JsonConverterConfig.SCHEMAS_ENABLE_CONFIG, + String.valueOf(true)); + String testId = startSingleTopicConnector(props); + + String key = "nonexistentKey"; + Struct innerStruct = + new Struct(SchemaBuilder.struct().field("b", Schema.OPTIONAL_INT8_SCHEMA)).put("b", null); + Struct struct = + new Struct(SchemaBuilder.struct().field("a", innerStruct.schema())).put("a", innerStruct); + byte[] valueBytes = + JsonConverterFactory.create(true, false).fromConnectData(testId, struct.schema(), struct); + String value = new String(valueBytes, StandardCharsets.UTF_8); + connect.kafka().produce(testId, key, value); + + assertDlqIsEmpty(dlqTopic); + assertConnectorAndAllTasksAreRunning(testId); + } + + @Test + public void testNonexistentColumnFamilyDeletionWorks() + throws ExecutionException, InterruptedException, TimeoutException { + String dlqTopic = createDlq(); + Map props = baseConnectorProps(); + props.put(BigtableSinkConfig.CONFIG_AUTO_CREATE_TABLES, String.valueOf(true)); + props.put(BigtableSinkConfig.CONFIG_AUTO_CREATE_COLUMN_FAMILIES, String.valueOf(true)); + props.put(BigtableSinkConfig.CONFIG_VALUE_NULL_MODE, NullValueMode.DELETE.name()); + props.put(BigtableSinkConfig.CONFIG_INSERT_MODE, InsertMode.UPSERT.name()); + configureDlq(props, dlqTopic); + props.put(ConnectorConfig.VALUE_CONVERTER_CLASS_CONFIG, JsonConverter.class.getName()); + props.put( + ConnectorConfig.VALUE_CONVERTER_CLASS_CONFIG + + "." + + JsonConverterConfig.SCHEMAS_ENABLE_CONFIG, + String.valueOf(true)); + String testId = startSingleTopicConnector(props); + + String key = "nonexistentKey"; + Struct struct = + new Struct(SchemaBuilder.struct().field("b", Schema.OPTIONAL_INT8_SCHEMA)).put("b", null); + byte[] valueBytes = + JsonConverterFactory.create(true, false).fromConnectData(testId, struct.schema(), struct); + String value = new String(valueBytes, StandardCharsets.UTF_8); + connect.kafka().produce(testId, key, value); + + assertDlqIsEmpty(dlqTopic); + assertConnectorAndAllTasksAreRunning(testId); } } diff --git a/google-cloud-bigtable-kafka-connect-sink/src/test/java/com/google/cloud/kafka/connect/bigtable/integration/ErrorReportingIT.java b/google-cloud-bigtable-kafka-connect-sink/src/test/java/com/google/cloud/kafka/connect/bigtable/integration/ErrorReportingIT.java index 54ebfef47e..7df9a03bc2 100644 --- a/google-cloud-bigtable-kafka-connect-sink/src/test/java/com/google/cloud/kafka/connect/bigtable/integration/ErrorReportingIT.java +++ b/google-cloud-bigtable-kafka-connect-sink/src/test/java/com/google/cloud/kafka/connect/bigtable/integration/ErrorReportingIT.java @@ -15,19 +15,9 @@ */ package com.google.cloud.kafka.connect.bigtable.integration; -import static org.apache.kafka.connect.runtime.errors.DeadLetterQueueReporter.ERROR_HEADER_EXCEPTION; -import static org.junit.Assert.assertArrayEquals; -import static org.junit.Assert.assertEquals; -import static org.junit.Assert.assertTrue; - import com.google.cloud.kafka.connect.bigtable.config.BigtableErrorMode; import com.google.cloud.kafka.connect.bigtable.config.BigtableSinkConfig; -import java.nio.charset.StandardCharsets; -import java.time.Duration; -import java.util.Arrays; import java.util.Map; -import org.apache.kafka.clients.consumer.ConsumerRecord; -import org.apache.kafka.clients.consumer.ConsumerRecords; import org.junit.Test; import org.junit.runner.RunWith; import org.junit.runners.JUnit4; @@ -84,18 +74,7 @@ public void testErrorModeDLQOverridesErrorMode() throws InterruptedException { String value = "value"; String testId = startSingleTopicConnector(props); connect.kafka().produce(testId, key, value); - ConsumerRecords dlqRecords = - connect.kafka().consume(1, Duration.ofSeconds(120).toMillis(), dlqTopic); - assertEquals(1, dlqRecords.count()); - ConsumerRecord record = dlqRecords.iterator().next(); - assertArrayEquals(record.key(), key.getBytes(StandardCharsets.UTF_8)); - assertArrayEquals(record.value(), value.getBytes(StandardCharsets.UTF_8)); - assertTrue( - Arrays.stream(record.headers().toArray()) - .anyMatch(h -> h.key().equals(ERROR_HEADER_EXCEPTION))); - connect - .assertions() - .assertConnectorAndExactlyNumTasksAreRunning( - testId, numTasks, "Wrong number of tasks is running."); + assertSingleDlqEntry(dlqTopic, key, value, null); + assertConnectorAndAllTasksAreRunning(testId); } } diff --git a/google-cloud-bigtable-kafka-connect-sink/src/test/java/com/google/cloud/kafka/connect/bigtable/integration/ResourceAutoCreationIT.java b/google-cloud-bigtable-kafka-connect-sink/src/test/java/com/google/cloud/kafka/connect/bigtable/integration/ResourceAutoCreationIT.java index 385564346a..d13a15a439 100644 --- a/google-cloud-bigtable-kafka-connect-sink/src/test/java/com/google/cloud/kafka/connect/bigtable/integration/ResourceAutoCreationIT.java +++ b/google-cloud-bigtable-kafka-connect-sink/src/test/java/com/google/cloud/kafka/connect/bigtable/integration/ResourceAutoCreationIT.java @@ -15,19 +15,31 @@ */ package com.google.cloud.kafka.connect.bigtable.integration; -import static org.apache.kafka.test.TestUtils.waitForCondition; -import static org.junit.Assert.assertArrayEquals; import static org.junit.Assert.assertEquals; +import static org.junit.Assert.assertFalse; +import static org.junit.Assert.assertThrows; import static org.junit.Assert.assertTrue; +import com.google.cloud.bigtable.admin.v2.models.ColumnFamily; import com.google.cloud.bigtable.admin.v2.models.CreateTableRequest; import com.google.cloud.bigtable.admin.v2.models.ModifyColumnFamiliesRequest; +import com.google.cloud.kafka.connect.bigtable.config.BigtableSinkConfig; +import com.google.cloud.kafka.connect.bigtable.config.BigtableSinkTaskConfig; +import com.google.cloud.kafka.connect.bigtable.config.InsertMode; +import com.google.cloud.kafka.connect.bigtable.config.NullValueMode; +import com.google.cloud.kafka.connect.bigtable.exception.InvalidBigtableSchemaModificationException; import com.google.cloud.kafka.connect.bigtable.util.JsonConverterFactory; import com.google.protobuf.ByteString; import java.nio.charset.StandardCharsets; import java.time.Duration; import java.util.Arrays; import java.util.Map; +import java.util.Set; +import java.util.concurrent.ExecutionException; +import java.util.concurrent.TimeoutException; +import java.util.function.Function; +import java.util.stream.Collectors; +import java.util.stream.IntStream; import java.util.stream.StreamSupport; import org.apache.kafka.clients.consumer.ConsumerRecords; import org.apache.kafka.connect.data.Schema; @@ -42,10 +54,17 @@ @RunWith(JUnit4.class) public class ResourceAutoCreationIT extends BaseKafkaConnectBigtableIT { - private final JsonConverter jsonConverter = JsonConverterFactory.create(true, false); + private static final JsonConverter JSON_CONVERTER = JsonConverterFactory.create(true, false); + private static final String KEY1 = "key1"; + private static final String KEY2 = "key2"; + private static final String KEY3 = "key3"; + private static final String KEY4 = "key4"; + private static final String COLUMN_FAMILY1 = "cf1"; + private static final String COLUMN_FAMILY2 = "cf2"; + private static final String COLUMN_QUALIFIER = "cq"; @Test - public void testMissingAndLaterCreatedTableAndColumnFamily() throws InterruptedException { + public void testDisabledResourceAutoCreation() throws InterruptedException { String dlqTopic = createDlq(); Map props = baseConnectorProps(); configureDlq(props, dlqTopic); @@ -58,83 +77,360 @@ public void testMissingAndLaterCreatedTableAndColumnFamily() throws InterruptedE String testId = startSingleTopicConnector(props); - String key1 = "key1"; - String key2 = "key2"; - String key3 = "key3"; - String key4 = "key4"; - String columnFamily1 = "cf1"; - String columnFamily2 = "cf2"; - String columnQualifier = "cq"; - Integer value = 1; Struct columnValue = - new Struct(SchemaBuilder.struct().field(columnQualifier, Schema.INT32_SCHEMA)) - .put(columnQualifier, value); + new Struct(SchemaBuilder.struct().field(COLUMN_QUALIFIER, Schema.INT32_SCHEMA)) + .put(COLUMN_QUALIFIER, value); Struct value1 = - new Struct(SchemaBuilder.struct().field(columnFamily1, columnValue.schema())) - .put(columnFamily1, columnValue); + new Struct(SchemaBuilder.struct().field(COLUMN_FAMILY1, columnValue.schema())) + .put(COLUMN_FAMILY1, columnValue); Struct value2 = - new Struct(SchemaBuilder.struct().field(columnFamily2, columnValue.schema())) - .put(columnFamily2, columnValue); + new Struct(SchemaBuilder.struct().field(COLUMN_FAMILY2, columnValue.schema())) + .put(COLUMN_FAMILY2, columnValue); - String serializedValue1 = - new String( - jsonConverter.fromConnectData(testId, value1.schema(), value1), StandardCharsets.UTF_8); - String serializedValue2 = - new String( - jsonConverter.fromConnectData(testId, value2.schema(), value2), StandardCharsets.UTF_8); + String serializedValue1 = jsonify(testId, value1.schema(), value1); + String serializedValue2 = jsonify(testId, value2.schema(), value2); // With the table missing. - connect.kafka().produce(testId, key1, serializedValue1); - ConsumerRecords dlqRecords = - connect.kafka().consume(1, Duration.ofSeconds(120).toMillis(), dlqTopic); - assertEquals(1, dlqRecords.count()); - assertArrayEquals(key1.getBytes(StandardCharsets.UTF_8), dlqRecords.iterator().next().key()); - connect - .assertions() - .assertConnectorAndExactlyNumTasksAreRunning( - testId, numTasks, "Wrong number of tasks is running"); + connect.kafka().produce(testId, KEY1, serializedValue1); + assertSingleDlqEntry(dlqTopic, KEY1, null, null); + assertConnectorAndAllTasksAreRunning(testId); // With the table and column family created. - bigtableAdmin.createTable(CreateTableRequest.of(testId).addFamily(columnFamily1)); - connect.kafka().produce(testId, key2, serializedValue1); - waitForCondition( - () -> readAllRows(bigtableData, testId).size() == 1, - Duration.ofSeconds(15).toMillis(), - "Records not processed in time"); + bigtableAdmin.createTable(CreateTableRequest.of(testId).addFamily(COLUMN_FAMILY1)); + connect.kafka().produce(testId, KEY2, serializedValue1); + waitUntilBigtableContainsNumberOfRows(testId, 1); assertTrue( readAllRows(bigtableData, testId) - .containsKey(ByteString.copyFrom(key2.getBytes(StandardCharsets.UTF_8)))); - connect - .assertions() - .assertConnectorAndExactlyNumTasksAreRunning( - testId, numTasks, "Wrong number of tasks is running"); + .containsKey(ByteString.copyFrom(KEY2.getBytes(StandardCharsets.UTF_8)))); + assertConnectorAndAllTasksAreRunning(testId); // With the column family missing. - connect.kafka().produce(testId, key3, serializedValue2); - dlqRecords = connect.kafka().consume(2, Duration.ofSeconds(120).toMillis(), dlqTopic); + connect.kafka().produce(testId, KEY3, serializedValue2); + ConsumerRecords dlqRecords = + connect.kafka().consume(2, Duration.ofSeconds(120).toMillis(), dlqTopic); assertEquals(2, dlqRecords.count()); assertTrue( StreamSupport.stream(dlqRecords.spliterator(), false) - .anyMatch(r -> Arrays.equals(key3.getBytes(StandardCharsets.UTF_8), r.key()))); - connect - .assertions() - .assertConnectorAndExactlyNumTasksAreRunning( - testId, numTasks, "Wrong number of tasks is running"); + .anyMatch(r -> Arrays.equals(KEY3.getBytes(StandardCharsets.UTF_8), r.key()))); + assertConnectorAndAllTasksAreRunning(testId); // With the column family created. - bigtableAdmin.modifyFamilies(ModifyColumnFamiliesRequest.of(testId).addFamily(columnFamily2)); - connect.kafka().produce(testId, key4, serializedValue2); - waitForCondition( - () -> readAllRows(bigtableData, testId).size() == 2, - Duration.ofSeconds(15).toMillis(), - "Records not " + "processed in time"); + bigtableAdmin.modifyFamilies(ModifyColumnFamiliesRequest.of(testId).addFamily(COLUMN_FAMILY2)); + connect.kafka().produce(testId, KEY4, serializedValue2); + waitUntilBigtableContainsNumberOfRows(testId, 2); assertTrue( readAllRows(bigtableData, testId) - .containsKey(ByteString.copyFrom(key4.getBytes(StandardCharsets.UTF_8)))); + .containsKey(ByteString.copyFrom(KEY4.getBytes(StandardCharsets.UTF_8)))); + assertConnectorAndAllTasksAreRunning(testId); + } + + @Test + public void testTableAutoCreationEnabledColumnFamilyAutoCreationDisabled() + throws InterruptedException { + String dlqTopic = createDlq(); + Map props = baseConnectorProps(); + configureDlq(props, dlqTopic); + props.put(BigtableSinkTaskConfig.CONFIG_AUTO_CREATE_TABLES, String.valueOf(true)); + props.put(ConnectorConfig.VALUE_CONVERTER_CLASS_CONFIG, JsonConverter.class.getName()); + props.put( + ConnectorConfig.VALUE_CONVERTER_CLASS_CONFIG + + "." + + JsonConverterConfig.SCHEMAS_ENABLE_CONFIG, + String.valueOf(true)); + + String testId = startSingleTopicConnector(props); + + String value = jsonify(testId, Schema.INT64_SCHEMA, 1234L); + + assertThrows(Throwable.class, () -> bigtableAdmin.getTable(testId)); + connect.kafka().produce(testId, KEY1, value); + waitUntilBigtableTableExists(testId); + + assertSingleDlqEntry(dlqTopic, KEY1, value, null); + assertConnectorAndAllTasksAreRunning(testId); + } + + @Test + public void testTableAutoCreationDisabledColumnFamilyAutoCreationEnabled() + throws InterruptedException { + String dlqTopic = createDlq(); + Map props = baseConnectorProps(); + configureDlq(props, dlqTopic); + props.put(BigtableSinkTaskConfig.CONFIG_AUTO_CREATE_COLUMN_FAMILIES, String.valueOf(true)); + props.put(ConnectorConfig.VALUE_CONVERTER_CLASS_CONFIG, JsonConverter.class.getName()); + props.put( + ConnectorConfig.VALUE_CONVERTER_CLASS_CONFIG + + "." + + JsonConverterConfig.SCHEMAS_ENABLE_CONFIG, + String.valueOf(true)); + + String testId = startSingleTopicConnector(props); + assertThrows(Throwable.class, () -> bigtableAdmin.getTable(testId)); + String value = jsonify(testId, Schema.INT64_SCHEMA, 1234L); + connect.kafka().produce(testId, KEY1, value); + assertSingleDlqEntry(dlqTopic, KEY1, value, null); + + bigtableAdmin.createTable(CreateTableRequest.of(testId)); + assertTrue(bigtableAdmin.getTable(testId).getColumnFamilies().isEmpty()); + connect.kafka().produce(testId, KEY2, value); + waitUntilBigtableContainsNumberOfRows(testId, 1); + assertFalse(bigtableAdmin.getTable(testId).getColumnFamilies().isEmpty()); + + assertConnectorAndAllTasksAreRunning(testId); + } + + @Test + public void testAutoTableAndColumnFamilyAutoCreationWhenReadingMultipleTopics() + throws InterruptedException, ExecutionException, TimeoutException { + String dlqTopic = createDlq(); + Map props = baseConnectorProps(); + configureDlq(props, dlqTopic); + props.put(BigtableSinkConfig.CONFIG_AUTO_CREATE_TABLES, String.valueOf(true)); + props.put(BigtableSinkConfig.CONFIG_AUTO_CREATE_COLUMN_FAMILIES, String.valueOf(true)); + + Set topicSuffixes = Set.of("topic1", "topic2", "topic3", "topic4"); + String testId = startMultipleTopicConnector(props, topicSuffixes); + Set topics = topicSuffixes.stream().map(s -> testId + s).collect(Collectors.toSet()); + + for (String topic : topics) { + assertThrows(Throwable.class, () -> bigtableAdmin.getTable(topic)); + connect.kafka().produce(topic, topic, "value"); + waitUntilBigtableContainsNumberOfRows(topic, 1); + assertEquals( + Set.of(topic), + bigtableAdmin.getTable(topic).getColumnFamilies().stream() + .map(ColumnFamily::getId) + .collect(Collectors.toSet())); + } + assertDlqIsEmpty(dlqTopic); + + assertConnectorAndAllTasksAreRunning(testId); + } + + @Test + public void testCreationOfInvalidTable() throws InterruptedException { + String dlqTopic = createDlq(); + Map props = baseConnectorProps(); + String invalidTableName = "T".repeat(10000); + props.put(BigtableSinkConfig.CONFIG_TABLE_NAME_FORMAT, invalidTableName); + props.put(BigtableSinkConfig.CONFIG_AUTO_CREATE_TABLES, String.valueOf(true)); + props.put(BigtableSinkConfig.CONFIG_RETRY_TIMEOUT_MILLIS, "10000"); + + configureDlq(props, dlqTopic); + String testId = startSingleTopicConnector(props); + + String value = "value"; + + connect.kafka().produce(testId, KEY1, value); + + assertSingleDlqEntry(dlqTopic, KEY1, value, InvalidBigtableSchemaModificationException.class); + assertConnectorAndAllTasksAreRunning(testId); + } + + @org.junit.Ignore // TODO: unignore. For now, the emulator does not cause an exception. + @Test + public void testCreationOfTooManyColumnFamilies() throws InterruptedException { + int numberOfColumnFamilies = 1000; + + String dlqTopic = createDlq(); + Map props = baseConnectorProps(); + props.put(BigtableSinkConfig.CONFIG_AUTO_CREATE_TABLES, String.valueOf(true)); + props.put(BigtableSinkConfig.CONFIG_AUTO_CREATE_COLUMN_FAMILIES, String.valueOf(true)); + props.put(ConnectorConfig.VALUE_CONVERTER_CLASS_CONFIG, JsonConverter.class.getName()); + props.put( + ConnectorConfig.VALUE_CONVERTER_CLASS_CONFIG + + "." + + JsonConverterConfig.SCHEMAS_ENABLE_CONFIG, + String.valueOf(true)); + + configureDlq(props, dlqTopic); + String testId = startSingleTopicConnector(props); + + Struct innerStruct = + new Struct(SchemaBuilder.struct().field(COLUMN_QUALIFIER, Schema.INT32_SCHEMA)) + .put(COLUMN_QUALIFIER, 1); + + Function fieldNameGenerator = i -> "f" + i; + SchemaBuilder schemaBuilder = SchemaBuilder.struct(); + IntStream.range(0, numberOfColumnFamilies) + .forEach(i -> schemaBuilder.field(fieldNameGenerator.apply(i), innerStruct.schema())); + Struct struct = new Struct(schemaBuilder.build()); + IntStream.range(0, numberOfColumnFamilies) + .forEach(i -> struct.put(fieldNameGenerator.apply(i), innerStruct)); + + String value = jsonify(testId, struct.schema(), struct); + + assertThrows(Throwable.class, () -> bigtableAdmin.getTable(testId)); + connect.kafka().produce(testId, KEY1, value); + assertSingleDlqEntry(dlqTopic, KEY1, value, InvalidBigtableSchemaModificationException.class); + bigtableAdmin.getTable(testId); + + assertConnectorAndAllTasksAreRunning(testId); + } + + @Test + public void testRowDeletionCreatesTableWhenAutoCreationEnabled() throws InterruptedException { + String dlqTopic = createDlq(); + Map props = baseConnectorProps(); + configureDlq(props, dlqTopic); + props.put(BigtableSinkConfig.CONFIG_AUTO_CREATE_TABLES, String.valueOf(true)); + props.put(BigtableSinkConfig.CONFIG_AUTO_CREATE_COLUMN_FAMILIES, String.valueOf(true)); + props.put(BigtableSinkConfig.CONFIG_VALUE_NULL_MODE, NullValueMode.DELETE.name()); + props.put(BigtableSinkConfig.CONFIG_INSERT_MODE, InsertMode.UPSERT.name()); + props.put(ConnectorConfig.VALUE_CONVERTER_CLASS_CONFIG, JsonConverter.class.getName()); + props.put( + ConnectorConfig.VALUE_CONVERTER_CLASS_CONFIG + + "." + + JsonConverterConfig.SCHEMAS_ENABLE_CONFIG, + String.valueOf(true)); + String testId = startSingleTopicConnector(props); + + String rowDeletionValue = jsonify(testId, Schema.OPTIONAL_BYTES_SCHEMA, null); + assertThrows(Throwable.class, () -> bigtableAdmin.getTable(testId)); + connect.kafka().produce(testId, KEY1, rowDeletionValue); + waitUntilBigtableTableExists(testId); + assertTrue(bigtableAdmin.getTable(testId).getColumnFamilies().isEmpty()); + + assertSingleDlqEntry(dlqTopic, KEY1, null, null); + assertConnectorAndAllTasksAreRunning(testId); + } + + @Test + public void testColumnFamilyDeletionCreatesTableAndColumnFamilyWhenAutoCreationEnabled() + throws InterruptedException, ExecutionException, TimeoutException { + String dlqTopic = createDlq(); + Map props = baseConnectorProps(); + configureDlq(props, dlqTopic); + props.put(BigtableSinkConfig.AUTO_CREATE_TABLES_CONFIG, String.valueOf(true)); + props.put(BigtableSinkConfig.AUTO_CREATE_COLUMN_FAMILIES_CONFIG, String.valueOf(true)); + props.put(BigtableSinkConfig.VALUE_NULL_MODE_CONFIG, NullValueMode.DELETE.name()); + props.put(BigtableSinkConfig.INSERT_MODE_CONFIG, InsertMode.UPSERT.name()); + props.put(ConnectorConfig.VALUE_CONVERTER_CLASS_CONFIG, JsonConverter.class.getName()); + props.put( + ConnectorConfig.VALUE_CONVERTER_CLASS_CONFIG + + "." + + JsonConverterConfig.SCHEMAS_ENABLE_CONFIG, + String.valueOf(true)); + String testId = startSingleTopicConnector(props); + + Struct deleteColumnFamily = + new Struct(SchemaBuilder.struct().field(COLUMN_FAMILY1, Schema.OPTIONAL_BYTES_SCHEMA)) + .put(COLUMN_FAMILY1, null); + assertThrows(Throwable.class, () -> bigtableAdmin.getTable(testId)); + connect + .kafka() + .produce(testId, KEY1, jsonify(testId, deleteColumnFamily.schema(), deleteColumnFamily)); + waitUntilBigtableTableHasColumnFamily(testId, COLUMN_FAMILY1); + assertEquals( + Set.of(COLUMN_FAMILY1), + bigtableAdmin.getTable(testId).getColumnFamilies().stream() + .map(ColumnFamily::getId) + .collect(Collectors.toSet())); + + assertDlqIsEmpty(dlqTopic); + assertConnectorAndAllTasksAreRunning(testId); + } + + @Test + public void testColumnDeletionCreatesTableAndColumnFamilyWhenAutoCreationEnabled() + throws InterruptedException, ExecutionException, TimeoutException { + String dlqTopic = createDlq(); + Map props = baseConnectorProps(); + configureDlq(props, dlqTopic); + props.put(BigtableSinkConfig.AUTO_CREATE_TABLES_CONFIG, String.valueOf(true)); + props.put(BigtableSinkConfig.AUTO_CREATE_COLUMN_FAMILIES_CONFIG, String.valueOf(true)); + props.put(BigtableSinkConfig.VALUE_NULL_MODE_CONFIG, NullValueMode.DELETE.name()); + props.put(BigtableSinkConfig.INSERT_MODE_CONFIG, InsertMode.UPSERT.name()); + props.put(ConnectorConfig.VALUE_CONVERTER_CLASS_CONFIG, JsonConverter.class.getName()); + props.put( + ConnectorConfig.VALUE_CONVERTER_CLASS_CONFIG + + "." + + JsonConverterConfig.SCHEMAS_ENABLE_CONFIG, + String.valueOf(true)); + String testId = startSingleTopicConnector(props); + + Struct innerStruct = + new Struct(SchemaBuilder.struct().field(COLUMN_QUALIFIER, Schema.OPTIONAL_BYTES_SCHEMA)) + .put(COLUMN_QUALIFIER, null); + Struct deleteColumn = + new Struct(SchemaBuilder.struct().field(COLUMN_FAMILY2, innerStruct.schema())) + .put(COLUMN_FAMILY2, innerStruct); + assertThrows(Throwable.class, () -> bigtableAdmin.getTable(testId)); + connect.kafka().produce(testId, KEY3, jsonify(testId, deleteColumn.schema(), deleteColumn)); + waitUntilBigtableTableHasColumnFamily(testId, COLUMN_FAMILY2); + assertEquals( + Set.of(COLUMN_FAMILY2), + bigtableAdmin.getTable(testId).getColumnFamilies().stream() + .map(ColumnFamily::getId) + .collect(Collectors.toSet())); + + assertDlqIsEmpty(dlqTopic); + assertConnectorAndAllTasksAreRunning(testId); + } + + @Test + public void testDeletionFailsWhenAutoCreationDisabled() throws InterruptedException { + String dlqTopic = createDlq(); + Map props = baseConnectorProps(); + configureDlq(props, dlqTopic); + props.put(BigtableSinkConfig.CONFIG_VALUE_NULL_MODE, NullValueMode.DELETE.name()); + props.put(BigtableSinkConfig.CONFIG_INSERT_MODE, InsertMode.UPSERT.name()); + props.put(ConnectorConfig.VALUE_CONVERTER_CLASS_CONFIG, JsonConverter.class.getName()); + props.put( + ConnectorConfig.VALUE_CONVERTER_CLASS_CONFIG + + "." + + JsonConverterConfig.SCHEMAS_ENABLE_CONFIG, + String.valueOf(true)); + + String deleteRowSuffix = "deleteRow"; + String deleteColumnFamilySuffix = "deleteColumnFamily"; + String deleteColumnSuffix = "deleteColumn"; + String testId = + startMultipleTopicConnector( + props, Set.of(deleteRowSuffix, deleteColumnFamilySuffix, deleteColumnSuffix)); + + String deleteRowTopic = testId + deleteRowSuffix; + String rowDeletionValue = jsonify(deleteRowTopic, Schema.OPTIONAL_BYTES_SCHEMA, null); + assertThrows(Throwable.class, () -> bigtableAdmin.getTable(deleteRowTopic)); + connect.kafka().produce(deleteRowTopic, KEY1, rowDeletionValue); + + String deleteColumnFamilyTopic = testId + deleteColumnFamilySuffix; + Struct deleteColumnFamily = + new Struct(SchemaBuilder.struct().field(COLUMN_FAMILY1, Schema.OPTIONAL_BYTES_SCHEMA)) + .put(COLUMN_FAMILY1, null); + assertThrows(Throwable.class, () -> bigtableAdmin.getTable(deleteColumnFamilyTopic)); + connect + .kafka() + .produce( + deleteColumnFamilyTopic, + KEY2, + jsonify(deleteColumnFamilyTopic, deleteColumnFamily.schema(), deleteColumnFamily)); + + String deleteColumnTopic = testId + deleteColumnSuffix; + Struct deleteColumn = + new Struct(SchemaBuilder.struct().field(COLUMN_FAMILY2, deleteColumnFamily.schema())) + .put(COLUMN_FAMILY2, deleteColumnFamily); + assertThrows(Throwable.class, () -> bigtableAdmin.getTable(deleteColumnTopic)); connect - .assertions() - .assertConnectorAndExactlyNumTasksAreRunning( - testId, numTasks, "Wrong number of tasks is running"); + .kafka() + .produce( + deleteColumnTopic, + KEY3, + jsonify(deleteColumnTopic, deleteColumn.schema(), deleteColumn)); + + ConsumerRecords dlqRecords = + connect.kafka().consume(3, Duration.ofSeconds(120).toMillis(), dlqTopic); + + Set dlqKeys = + StreamSupport.stream(dlqRecords.spliterator(), false) + .map(r -> new String(r.key(), StandardCharsets.UTF_8)) + .collect(Collectors.toSet()); + assertEquals(Set.of(KEY1, KEY2, KEY3), dlqKeys); + } + + private static String jsonify(String topic, Schema schema, Object value) { + byte[] bytes = JSON_CONVERTER.fromConnectData(topic, schema, value); + return new String(bytes, StandardCharsets.UTF_8); } } diff --git a/google-cloud-bigtable-kafka-connect-sink/src/test/java/com/google/cloud/kafka/connect/bigtable/util/TestId.java b/google-cloud-bigtable-kafka-connect-sink/src/test/java/com/google/cloud/kafka/connect/bigtable/util/TestId.java index f32e8a63d1..0869130023 100644 --- a/google-cloud-bigtable-kafka-connect-sink/src/test/java/com/google/cloud/kafka/connect/bigtable/util/TestId.java +++ b/google-cloud-bigtable-kafka-connect-sink/src/test/java/com/google/cloud/kafka/connect/bigtable/util/TestId.java @@ -27,6 +27,6 @@ public static String getTestCaseId(Class testClass) { StackWalker.getInstance(StackWalker.Option.RETAIN_CLASS_REFERENCE) .walk(s -> Streams.findLast(s.filter(f -> f.getDeclaringClass().equals(testClass)))) .get(); - return getTestClassId(frame.getDeclaringClass()) + frame.getMethodName(); + return frame.getMethodName(); } } From 0e495562de27e70662f532fdfb5ceb68435d0c44 Mon Sep 17 00:00:00 2001 From: Adam Czajkowski Date: Tue, 4 Feb 2025 13:57:33 +0100 Subject: [PATCH 33/76] BigtableSinkConfig: use "_CONFIG" as a suffix of static variables denoting configuration keys for consistency with Kafka convention --- .../bigtable/BigtableSinkConnector.java | 2 +- .../connect/bigtable/BigtableSinkTask.java | 18 +- .../bigtable/config/BigtableSinkConfig.java | 168 +++++++++--------- .../config/BigtableSinkTaskConfig.java | 4 +- .../connect/bigtable/mapping/KeyMapper.java | 4 +- .../connect/bigtable/mapping/ValueMapper.java | 4 +- .../bigtable/BigtableSinkConnectorTest.java | 6 +- .../bigtable/BigtableSinkTaskTest.java | 26 +-- .../config/BigtableSinkConfigTest.java | 46 ++--- .../connect/bigtable/integration/BaseIT.java | 8 +- .../connect/bigtable/integration/BasicIT.java | 4 +- .../integration/ConfluentCompatibilityIT.java | 8 +- .../integration/DifferentConvertersIT.java | 4 +- .../bigtable/integration/ErrorHandlingIT.java | 40 ++--- .../integration/ErrorReportingIT.java | 8 +- .../bigtable/integration/InsertUpsertIT.java | 16 +- .../bigtable/integration/NullHandlingIT.java | 16 +- .../integration/ResourceAutoCreationIT.java | 30 ++-- .../bigtable/util/BasicPropertiesFactory.java | 6 +- 19 files changed, 209 insertions(+), 209 deletions(-) diff --git a/google-cloud-bigtable-kafka-connect-sink/src/main/java/com/google/cloud/kafka/connect/bigtable/BigtableSinkConnector.java b/google-cloud-bigtable-kafka-connect-sink/src/main/java/com/google/cloud/kafka/connect/bigtable/BigtableSinkConnector.java index 183686c6e3..3ec1b31b14 100644 --- a/google-cloud-bigtable-kafka-connect-sink/src/main/java/com/google/cloud/kafka/connect/bigtable/BigtableSinkConnector.java +++ b/google-cloud-bigtable-kafka-connect-sink/src/main/java/com/google/cloud/kafka/connect/bigtable/BigtableSinkConnector.java @@ -69,7 +69,7 @@ public List> taskConfigs(int maxTasks) { List> configs = new ArrayList<>(maxTasks); for (int i = 0; i < maxTasks; i++) { Map config = new HashMap<>(configProperties); - config.put(BigtableSinkTaskConfig.CONFIG_TASK_ID, Integer.toString(i)); + config.put(BigtableSinkTaskConfig.TASK_ID_CONFIG, Integer.toString(i)); configs.add(config); } return configs; diff --git a/google-cloud-bigtable-kafka-connect-sink/src/main/java/com/google/cloud/kafka/connect/bigtable/BigtableSinkTask.java b/google-cloud-bigtable-kafka-connect-sink/src/main/java/com/google/cloud/kafka/connect/bigtable/BigtableSinkTask.java index 4f9985b1cf..80bea50840 100644 --- a/google-cloud-bigtable-kafka-connect-sink/src/main/java/com/google/cloud/kafka/connect/bigtable/BigtableSinkTask.java +++ b/google-cloud-bigtable-kafka-connect-sink/src/main/java/com/google/cloud/kafka/connect/bigtable/BigtableSinkTask.java @@ -111,17 +111,17 @@ public void start(Map props) { logger = LoggerFactory.getLogger( BigtableSinkTask.class.getName() - + config.getInt(BigtableSinkTaskConfig.CONFIG_TASK_ID)); + + config.getInt(BigtableSinkTaskConfig.TASK_ID_CONFIG)); bigtableData = config.getBigtableDataClient(); bigtableAdmin = config.getBigtableAdminClient(); keyMapper = new KeyMapper( - config.getString(BigtableSinkTaskConfig.CONFIG_ROW_KEY_DELIMITER), - config.getList(BigtableSinkTaskConfig.CONFIG_ROW_KEY_DEFINITION)); + config.getString(BigtableSinkTaskConfig.ROW_KEY_DELIMITER_CONFIG), + config.getList(BigtableSinkTaskConfig.ROW_KEY_DEFINITION_CONFIG)); valueMapper = new ValueMapper( - config.getString(BigtableSinkTaskConfig.CONFIG_DEFAULT_COLUMN_FAMILY), - config.getString(BigtableSinkTaskConfig.CONFIG_DEFAULT_COLUMN_QUALIFIER), + config.getString(BigtableSinkTaskConfig.DEFAULT_COLUMN_FAMILY_CONFIG), + config.getString(BigtableSinkTaskConfig.DEFAULT_COLUMN_QUALIFIER_CONFIG), config.getNullValueMode()); schemaManager = new BigtableSchemaManager(bigtableAdmin); } @@ -168,10 +168,10 @@ public void put(Collection records) { } Map mutations = prepareRecords(records); - if (config.getBoolean(BigtableSinkTaskConfig.CONFIG_AUTO_CREATE_TABLES)) { + if (config.getBoolean(BigtableSinkTaskConfig.AUTO_CREATE_TABLES_CONFIG)) { mutations = autoCreateTablesAndHandleErrors(mutations); } - if (config.getBoolean(BigtableSinkTaskConfig.CONFIG_AUTO_CREATE_COLUMN_FAMILIES)) { + if (config.getBoolean(BigtableSinkTaskConfig.AUTO_CREATE_COLUMN_FAMILIES_CONFIG)) { mutations = autoCreateColumnFamiliesAndHandleErrors(mutations); } // Needed so that the batch ordering is more predictable from the user's point of view. @@ -247,7 +247,7 @@ Optional createRecordMutationData(SinkRecord record) { */ @VisibleForTesting String getTableName(SinkRecord record) { - String template = config.getString(BigtableSinkTaskConfig.CONFIG_TABLE_NAME_FORMAT); + String template = config.getString(BigtableSinkTaskConfig.TABLE_NAME_FORMAT_CONFIG); return ConfigInterpolation.replace(template, record.topic()); } @@ -397,7 +397,7 @@ void upsertRows( Map mutations, Map> perRecordResults) { List> mutationsToApply = new ArrayList<>(mutations.entrySet()); - int maxBatchSize = config.getInt(BigtableSinkTaskConfig.CONFIG_MAX_BATCH_SIZE); + int maxBatchSize = config.getInt(BigtableSinkTaskConfig.MAX_BATCH_SIZE_CONFIG); List>> batches = Lists.partition(mutationsToApply, maxBatchSize); diff --git a/google-cloud-bigtable-kafka-connect-sink/src/main/java/com/google/cloud/kafka/connect/bigtable/config/BigtableSinkConfig.java b/google-cloud-bigtable-kafka-connect-sink/src/main/java/com/google/cloud/kafka/connect/bigtable/config/BigtableSinkConfig.java index 9abe10c7d8..d8275e3b01 100644 --- a/google-cloud-bigtable-kafka-connect-sink/src/main/java/com/google/cloud/kafka/connect/bigtable/config/BigtableSinkConfig.java +++ b/google-cloud-bigtable-kafka-connect-sink/src/main/java/com/google/cloud/kafka/connect/bigtable/config/BigtableSinkConfig.java @@ -54,34 +54,34 @@ *

    It's responsible for the validation and parsing of the user-provided values. */ public class BigtableSinkConfig extends AbstractConfig { - public static final String CONFIG_GCP_PROJECT_ID = "gcp.bigtable.project.id"; - public static final String CONFIG_GCP_CREDENTIALS_PATH = "gcp.bigtable.credentials.path"; - public static final String CONFIG_GCP_CREDENTIALS_JSON = "gcp.bigtable.credentials.json"; - public static final String CONFIG_BIGTABLE_INSTANCE_ID = "gcp.bigtable.instance.id"; - public static final String CONFIG_BIGTABLE_APP_PROFILE_ID = "gcp.bigtable.app.profile.id"; - public static final String CONFIG_INSERT_MODE = "insert.mode"; - public static final String CONFIG_MAX_BATCH_SIZE = "max.batch.size"; - public static final String CONFIG_VALUE_NULL_MODE = "value.null.mode"; - public static final String CONFIG_ERROR_MODE = "error.mode"; - public static final String CONFIG_TABLE_NAME_FORMAT = "table.name.format"; - public static final String CONFIG_ROW_KEY_DEFINITION = "row.key.definition"; - public static final String CONFIG_ROW_KEY_DELIMITER = "row.key.delimiter"; - public static final String CONFIG_AUTO_CREATE_TABLES = "auto.create.tables"; - public static final String CONFIG_AUTO_CREATE_COLUMN_FAMILIES = "auto.create.column.families"; - public static final String CONFIG_DEFAULT_COLUMN_FAMILY = "default.column.family"; - public static final String CONFIG_DEFAULT_COLUMN_QUALIFIER = "default.column.qualifier"; - public static final String CONFIG_RETRY_TIMEOUT_MILLIS = "retry.timeout.ms"; - private static final InsertMode DEFAULT_INSERT_MODE = InsertMode.INSERT; - private static final NullValueMode DEFAULT_NULL_VALUE_MODE = NullValueMode.WRITE; - private static final BigtableErrorMode DEFAULT_ERROR_MODE = BigtableErrorMode.FAIL; - private static final Integer DEFAULT_MAX_BATCH_SIZE = 1; + public static final String GCP_PROJECT_ID_CONFIG = "gcp.bigtable.project.id"; + public static final String GCP_CREDENTIALS_PATH_CONFIG = "gcp.bigtable.credentials.path"; + public static final String GCP_CREDENTIALS_JSON_CONFIG = "gcp.bigtable.credentials.json"; + public static final String BIGTABLE_INSTANCE_ID_CONFIG = "gcp.bigtable.instance.id"; + public static final String BIGTABLE_APP_PROFILE_ID_CONFIG = "gcp.bigtable.app.profile.id"; + public static final String INSERT_MODE_CONFIG = "insert.mode"; + public static final String MAX_BATCH_SIZE_CONFIG = "max.batch.size"; + public static final String VALUE_NULL_MODE_CONFIG = "value.null.mode"; + public static final String ERROR_MODE_CONFIG = "error.mode"; + public static final String TABLE_NAME_FORMAT_CONFIG = "table.name.format"; + public static final String ROW_KEY_DEFINITION_CONFIG = "row.key.definition"; + public static final String ROW_KEY_DELIMITER_CONFIG = "row.key.delimiter"; + public static final String AUTO_CREATE_TABLES_CONFIG = "auto.create.tables"; + public static final String AUTO_CREATE_COLUMN_FAMILIES_CONFIG = "auto.create.column.families"; + public static final String DEFAULT_COLUMN_FAMILY_CONFIG = "default.column.family"; + public static final String DEFAULT_COLUMN_QUALIFIER_CONFIG = "default.column.qualifier"; + public static final String RETRY_TIMEOUT_MILLIS_CONFIG = "retry.timeout.ms"; + private static final InsertMode INSERT_MODE_DEFAULT = InsertMode.INSERT; + private static final NullValueMode NULL_VALUE_MODE_DEFAULT = NullValueMode.WRITE; + private static final BigtableErrorMode ERROR_MODE_DEFAULT = BigtableErrorMode.FAIL; + private static final Integer MAX_BATCH_SIZE_DEFAULT = 1; private static final List BIGTABLE_CONFIGURATION_PROPERTIES = List.of( - CONFIG_GCP_CREDENTIALS_JSON, - CONFIG_GCP_CREDENTIALS_PATH, - CONFIG_GCP_PROJECT_ID, - CONFIG_BIGTABLE_INSTANCE_ID, - CONFIG_BIGTABLE_APP_PROFILE_ID); + GCP_CREDENTIALS_JSON_CONFIG, + GCP_CREDENTIALS_PATH_CONFIG, + GCP_PROJECT_ID_CONFIG, + BIGTABLE_INSTANCE_ID_CONFIG, + BIGTABLE_APP_PROFILE_ID_CONFIG); private static final int BIGTABLE_CREDENTIALS_CHECK_TIMEOUT_SECONDS = 2; protected BigtableSinkConfig(ConfigDef definition, Map properties) { @@ -120,55 +120,55 @@ public static Config validate(Map props) { static Config validate(Map props, boolean accessBigtableToValidateConfiguration) { // Note that we only need to verify the properties we define, the generic Sink configuration is // handled in SinkConnectorConfig::validate(). - String credentialsPath = props.get(CONFIG_GCP_CREDENTIALS_PATH); - String credentialsJson = props.get(CONFIG_GCP_CREDENTIALS_JSON); - String insertMode = props.get(CONFIG_INSERT_MODE); - String nullValueMode = props.get(CONFIG_VALUE_NULL_MODE); - String maxBatchSize = props.get(CONFIG_MAX_BATCH_SIZE); + String credentialsPath = props.get(GCP_CREDENTIALS_PATH_CONFIG); + String credentialsJson = props.get(GCP_CREDENTIALS_JSON_CONFIG); + String insertMode = props.get(INSERT_MODE_CONFIG); + String nullValueMode = props.get(VALUE_NULL_MODE_CONFIG); + String maxBatchSize = props.get(MAX_BATCH_SIZE_CONFIG); String effectiveInsertMode = - Optional.ofNullable(insertMode).orElse(DEFAULT_INSERT_MODE.name()).toUpperCase(); + Optional.ofNullable(insertMode).orElse(INSERT_MODE_DEFAULT.name()).toUpperCase(); String effectiveNullValueMode = - Optional.ofNullable(nullValueMode).orElse(DEFAULT_NULL_VALUE_MODE.name()).toUpperCase(); + Optional.ofNullable(nullValueMode).orElse(NULL_VALUE_MODE_DEFAULT.name()).toUpperCase(); String effectiveMaxBatchSize = - Optional.ofNullable(maxBatchSize).orElse(DEFAULT_MAX_BATCH_SIZE.toString()).trim(); + Optional.ofNullable(maxBatchSize).orElse(MAX_BATCH_SIZE_DEFAULT.toString()).trim(); Map validationResult = getDefinition().validateAll(props); if (!Utils.isBlank(credentialsPath) && !Utils.isBlank(credentialsJson)) { String errorMessage = - CONFIG_GCP_CREDENTIALS_JSON + GCP_CREDENTIALS_JSON_CONFIG + " and " - + CONFIG_GCP_CREDENTIALS_PATH + + GCP_CREDENTIALS_PATH_CONFIG + " are mutually exclusive options, but both are set."; - addErrorMessage(validationResult, CONFIG_GCP_CREDENTIALS_JSON, credentialsJson, errorMessage); - addErrorMessage(validationResult, CONFIG_GCP_CREDENTIALS_PATH, credentialsPath, errorMessage); + addErrorMessage(validationResult, GCP_CREDENTIALS_JSON_CONFIG, credentialsJson, errorMessage); + addErrorMessage(validationResult, GCP_CREDENTIALS_PATH_CONFIG, credentialsPath, errorMessage); } if (effectiveInsertMode.equals(InsertMode.INSERT.name()) && !effectiveMaxBatchSize.equals("1")) { String errorMessage = "When using `" - + CONFIG_INSERT_MODE + + INSERT_MODE_CONFIG + "` of `" + InsertMode.INSERT.name() + "`, " - + CONFIG_MAX_BATCH_SIZE + + MAX_BATCH_SIZE_CONFIG + " must be set to `1`."; - addErrorMessage(validationResult, CONFIG_INSERT_MODE, insertMode, errorMessage); - addErrorMessage(validationResult, CONFIG_MAX_BATCH_SIZE, maxBatchSize, errorMessage); + addErrorMessage(validationResult, INSERT_MODE_CONFIG, insertMode, errorMessage); + addErrorMessage(validationResult, MAX_BATCH_SIZE_CONFIG, maxBatchSize, errorMessage); } if (effectiveInsertMode.equals(InsertMode.INSERT.name()) && effectiveNullValueMode.equals(NullValueMode.DELETE.name())) { String errorMessage = "When using `" - + CONFIG_VALUE_NULL_MODE + + VALUE_NULL_MODE_CONFIG + "` of `" + NullValueMode.DELETE.name() + "`, " - + CONFIG_INSERT_MODE + + INSERT_MODE_CONFIG + " must not be set to `" + InsertMode.INSERT.name() + "`."; - addErrorMessage(validationResult, CONFIG_INSERT_MODE, insertMode, errorMessage); - addErrorMessage(validationResult, CONFIG_VALUE_NULL_MODE, nullValueMode, errorMessage); + addErrorMessage(validationResult, INSERT_MODE_CONFIG, insertMode, errorMessage); + addErrorMessage(validationResult, VALUE_NULL_MODE_CONFIG, nullValueMode, errorMessage); } if (accessBigtableToValidateConfiguration @@ -196,7 +196,7 @@ static Config validate(Map props, boolean accessBigtableToValida public static ConfigDef getDefinition() { return new ConfigDef() .define( - CONFIG_GCP_PROJECT_ID, + GCP_PROJECT_ID_CONFIG, ConfigDef.Type.STRING, ConfigDef.NO_DEFAULT_VALUE, ConfigDef.CompositeValidator.of( @@ -204,7 +204,7 @@ public static ConfigDef getDefinition() { ConfigDef.Importance.HIGH, "The ID of the GCP project.") .define( - CONFIG_BIGTABLE_INSTANCE_ID, + BIGTABLE_INSTANCE_ID_CONFIG, ConfigDef.Type.STRING, ConfigDef.NO_DEFAULT_VALUE, ConfigDef.CompositeValidator.of( @@ -212,36 +212,36 @@ public static ConfigDef getDefinition() { ConfigDef.Importance.HIGH, "The ID of the Cloud Bigtable instance.") .define( - CONFIG_BIGTABLE_APP_PROFILE_ID, + BIGTABLE_APP_PROFILE_ID_CONFIG, ConfigDef.Type.STRING, null, ConfigDef.Importance.MEDIUM, "The application profile that the connector should use. If none is supplied," + " the default app profile will be used.") .define( - CONFIG_GCP_CREDENTIALS_PATH, + GCP_CREDENTIALS_PATH_CONFIG, ConfigDef.Type.STRING, null, ConfigDef.Importance.HIGH, "The path to the JSON service key file. Configure at most one of `" - + CONFIG_GCP_CREDENTIALS_PATH + + GCP_CREDENTIALS_PATH_CONFIG + "` and `" - + CONFIG_GCP_CREDENTIALS_JSON + + GCP_CREDENTIALS_JSON_CONFIG + "`. If neither is provided, Application Default Credentials will be used.") .define( - CONFIG_GCP_CREDENTIALS_JSON, + GCP_CREDENTIALS_JSON_CONFIG, ConfigDef.Type.STRING, null, ConfigDef.Importance.HIGH, "The path to the JSON service key file. Configure at most one of `" - + CONFIG_GCP_CREDENTIALS_PATH + + GCP_CREDENTIALS_PATH_CONFIG + "` and `" - + CONFIG_GCP_CREDENTIALS_JSON + + GCP_CREDENTIALS_JSON_CONFIG + "`. If neither is provided, Application Default Credentials will be used.") .define( - CONFIG_INSERT_MODE, + INSERT_MODE_CONFIG, ConfigDef.Type.STRING, - DEFAULT_INSERT_MODE.name(), + INSERT_MODE_DEFAULT.name(), enumValidator(InsertMode.values()), ConfigDef.Importance.HIGH, "Defines the insertion mode to use. Supported modes are:" @@ -250,21 +250,21 @@ public static ConfigDef getDefinition() { + "\n- upsert - If the row to be written already exists," + " then its column values are overwritten with the ones provided.") .define( - CONFIG_MAX_BATCH_SIZE, + MAX_BATCH_SIZE_CONFIG, ConfigDef.Type.INT, - DEFAULT_MAX_BATCH_SIZE, + MAX_BATCH_SIZE_DEFAULT, ConfigDef.Range.atLeast(1), ConfigDef.Importance.MEDIUM, "The maximum number of records that can be batched into a batch of upserts." + " Note that since only a batch size of 1 for inserts is supported, `" - + CONFIG_MAX_BATCH_SIZE + + MAX_BATCH_SIZE_CONFIG + "` must be exactly `1` when `" - + CONFIG_INSERT_MODE + + INSERT_MODE_CONFIG + "` is set to `INSERT`.") .define( - CONFIG_VALUE_NULL_MODE, + VALUE_NULL_MODE_CONFIG, ConfigDef.Type.STRING, - DEFAULT_NULL_VALUE_MODE.name(), + NULL_VALUE_MODE_DEFAULT.name(), enumValidator(NullValueMode.values()), ConfigDef.Importance.MEDIUM, "Defines what to do with `null`s within Kafka values. Supported modes are:" @@ -277,9 +277,9 @@ public static ConfigDef getDefinition() { + " field. `null` values nested more than two levels are serialized like other" + " values and don't result in any DELETE commands.") .define( - CONFIG_ERROR_MODE, + ERROR_MODE_CONFIG, ConfigDef.Type.STRING, - DEFAULT_ERROR_MODE.name(), + ERROR_MODE_DEFAULT.name(), enumValidator(BigtableErrorMode.values()), ConfigDef.Importance.MEDIUM, "Specifies how to handle errors that result from writes, after retries. It is ignored" @@ -289,7 +289,7 @@ public static ConfigDef getDefinition() { + "\n- ignore - The connector does not log a warning but continues operating" + " normally.") .define( - CONFIG_TABLE_NAME_FORMAT, + TABLE_NAME_FORMAT_CONFIG, ConfigDef.Type.STRING, ConfigInterpolation.TOPIC_PLACEHOLDER, ConfigDef.CompositeValidator.of( @@ -302,7 +302,7 @@ public static ConfigDef getDefinition() { + ConfigInterpolation.TOPIC_PLACEHOLDER + "` for the topic `stats` will map to the table name `user_stats`.") .define( - CONFIG_ROW_KEY_DEFINITION, + ROW_KEY_DEFINITION_CONFIG, ConfigDef.Type.LIST, "", ConfigDef.Importance.MEDIUM, @@ -320,7 +320,7 @@ public static ConfigDef getDefinition() { + " your Row Key, consider configuring an SMT to add relevant fields to the Kafka" + " Record key.") .define( - CONFIG_ROW_KEY_DELIMITER, + ROW_KEY_DELIMITER_CONFIG, ConfigDef.Type.STRING, "", ConfigDef.Importance.LOW, @@ -328,7 +328,7 @@ public static ConfigDef getDefinition() { + " configuration is empty or unspecified, the key fields will be concatenated" + " together directly.") .define( - CONFIG_AUTO_CREATE_TABLES, + AUTO_CREATE_TABLES_CONFIG, ConfigDef.Type.BOOLEAN, false, new ConfigDef.NonNullValidator(), @@ -337,7 +337,7 @@ public static ConfigDef getDefinition() { + "\nWhen enabled, the records for which the auto-creation fails, are failed." + "\nRecreation of tables deleted by other Cloud Bigtable users is not supported.") .define( - CONFIG_AUTO_CREATE_COLUMN_FAMILIES, + AUTO_CREATE_COLUMN_FAMILIES_CONFIG, ConfigDef.Type.BOOLEAN, false, new ConfigDef.NonNullValidator(), @@ -351,7 +351,7 @@ public static ConfigDef getDefinition() { + "Recreation of column families deleted by other Cloud Bigtable users is not" + " supported.") .define( - CONFIG_DEFAULT_COLUMN_FAMILY, + DEFAULT_COLUMN_FAMILY_CONFIG, ConfigDef.Type.STRING, ConfigInterpolation.TOPIC_PLACEHOLDER, ConfigDef.Importance.MEDIUM, @@ -360,14 +360,14 @@ public static ConfigDef getDefinition() { + ConfigInterpolation.TOPIC_PLACEHOLDER + "` within the column family name to specify the originating topic name.") .define( - CONFIG_DEFAULT_COLUMN_QUALIFIER, + DEFAULT_COLUMN_QUALIFIER_CONFIG, ConfigDef.Type.STRING, "KAFKA_VALUE", ConfigDef.Importance.MEDIUM, "Any root-level values on the SinkRecord that aren't objects will be added to this" + " column within default column family. If empty, the value will be ignored.") .define( - CONFIG_RETRY_TIMEOUT_MILLIS, + RETRY_TIMEOUT_MILLIS_CONFIG, ConfigDef.Type.LONG, 90000, ConfigDef.Range.atLeast(0), @@ -393,15 +393,15 @@ private static void addErrorMessage( } public NullValueMode getNullValueMode() { - return getEnum(CONFIG_VALUE_NULL_MODE, NullValueMode::valueOf); + return getEnum(VALUE_NULL_MODE_CONFIG, NullValueMode::valueOf); } public BigtableErrorMode getBigtableErrorMode() { - return getEnum(CONFIG_ERROR_MODE, BigtableErrorMode::valueOf); + return getEnum(ERROR_MODE_CONFIG, BigtableErrorMode::valueOf); } public InsertMode getInsertMode() { - return getEnum(CONFIG_INSERT_MODE, InsertMode::valueOf); + return getEnum(INSERT_MODE_CONFIG, InsertMode::valueOf); } /** @@ -420,8 +420,8 @@ BigtableTableAdminClient getBigtableAdminClient(RetrySettings retrySettings) { BigtableTableAdminSettings.Builder adminSettingsBuilder = BigtableTableAdminSettings.newBuilder() - .setProjectId(getString(BigtableSinkTaskConfig.CONFIG_GCP_PROJECT_ID)) - .setInstanceId(getString(BigtableSinkTaskConfig.CONFIG_BIGTABLE_INSTANCE_ID)); + .setProjectId(getString(BigtableSinkTaskConfig.GCP_PROJECT_ID_CONFIG)) + .setInstanceId(getString(BigtableSinkTaskConfig.BIGTABLE_INSTANCE_ID_CONFIG)); if (credentialsProvider.isPresent()) { adminSettingsBuilder.setCredentialsProvider(credentialsProvider.get()); } else { @@ -451,14 +451,14 @@ public BigtableDataClient getBigtableDataClient() { BigtableDataSettings.Builder dataSettingsBuilder = BigtableDataSettings.newBuilder() - .setProjectId(getString(BigtableSinkTaskConfig.CONFIG_GCP_PROJECT_ID)) - .setInstanceId(getString(BigtableSinkTaskConfig.CONFIG_BIGTABLE_INSTANCE_ID)); + .setProjectId(getString(BigtableSinkTaskConfig.GCP_PROJECT_ID_CONFIG)) + .setInstanceId(getString(BigtableSinkTaskConfig.BIGTABLE_INSTANCE_ID_CONFIG)); if (credentialsProvider.isPresent()) { dataSettingsBuilder.setCredentialsProvider(credentialsProvider.get()); } else { // Use the default credential provider that utilizes Application Default Credentials. } - String appProfileId = getString(BigtableSinkTaskConfig.CONFIG_BIGTABLE_APP_PROFILE_ID); + String appProfileId = getString(BigtableSinkTaskConfig.BIGTABLE_APP_PROFILE_ID_CONFIG); if (appProfileId == null) { dataSettingsBuilder.setDefaultAppProfileId(); } else { @@ -514,7 +514,7 @@ protected RetrySettings getRetrySettings() { return RetrySettings.newBuilder() .setTotalTimeout( Duration.of( - getLong(BigtableSinkTaskConfig.CONFIG_RETRY_TIMEOUT_MILLIS), ChronoUnit.MILLIS)) + getLong(BigtableSinkTaskConfig.RETRY_TIMEOUT_MILLIS_CONFIG), ChronoUnit.MILLIS)) .build(); } @@ -547,8 +547,8 @@ private static ConfigDef.Validator enumValidator(Enum[] enumValues) { * BigtableSinkConfig#getDefinition()} otherwise. */ protected Optional getUserConfiguredBigtableCredentialsProvider() { - String credentialsJson = getString(BigtableSinkTaskConfig.CONFIG_GCP_CREDENTIALS_JSON); - String credentialsPath = getString(BigtableSinkTaskConfig.CONFIG_GCP_CREDENTIALS_PATH); + String credentialsJson = getString(BigtableSinkTaskConfig.GCP_CREDENTIALS_JSON_CONFIG); + String credentialsPath = getString(BigtableSinkTaskConfig.GCP_CREDENTIALS_PATH_CONFIG); byte[] credentials; if (!Utils.isBlank(credentialsJson)) { credentials = credentialsJson.getBytes(StandardCharsets.UTF_8); diff --git a/google-cloud-bigtable-kafka-connect-sink/src/main/java/com/google/cloud/kafka/connect/bigtable/config/BigtableSinkTaskConfig.java b/google-cloud-bigtable-kafka-connect-sink/src/main/java/com/google/cloud/kafka/connect/bigtable/config/BigtableSinkTaskConfig.java index 48129c32f2..4ba7ed2c6e 100644 --- a/google-cloud-bigtable-kafka-connect-sink/src/main/java/com/google/cloud/kafka/connect/bigtable/config/BigtableSinkTaskConfig.java +++ b/google-cloud-bigtable-kafka-connect-sink/src/main/java/com/google/cloud/kafka/connect/bigtable/config/BigtableSinkTaskConfig.java @@ -23,7 +23,7 @@ * com.google.cloud.kafka.connect.bigtable.BigtableSinkTask}. */ public class BigtableSinkTaskConfig extends BigtableSinkConfig { - public static String CONFIG_TASK_ID = "taskId"; + public static String TASK_ID_CONFIG = "taskId"; /** * The main constructor. @@ -41,7 +41,7 @@ public BigtableSinkTaskConfig(Map properties) { public static ConfigDef getDefinition() { return BigtableSinkConfig.getDefinition() .defineInternal( - CONFIG_TASK_ID, + TASK_ID_CONFIG, ConfigDef.Type.INT, ConfigDef.NO_DEFAULT_VALUE, ConfigDef.Importance.LOW); diff --git a/google-cloud-bigtable-kafka-connect-sink/src/main/java/com/google/cloud/kafka/connect/bigtable/mapping/KeyMapper.java b/google-cloud-bigtable-kafka-connect-sink/src/main/java/com/google/cloud/kafka/connect/bigtable/mapping/KeyMapper.java index 219e88468a..6070e27519 100644 --- a/google-cloud-bigtable-kafka-connect-sink/src/main/java/com/google/cloud/kafka/connect/bigtable/mapping/KeyMapper.java +++ b/google-cloud-bigtable-kafka-connect-sink/src/main/java/com/google/cloud/kafka/connect/bigtable/mapping/KeyMapper.java @@ -53,9 +53,9 @@ public class KeyMapper { * The main constructor. * * @param delimiter Delimiter in the mapping as per {@link - * com.google.cloud.kafka.connect.bigtable.config.BigtableSinkConfig#CONFIG_ROW_KEY_DELIMITER} + * com.google.cloud.kafka.connect.bigtable.config.BigtableSinkConfig#ROW_KEY_DELIMITER_CONFIG} * @param definition Definition of the mapping as per {@link - * com.google.cloud.kafka.connect.bigtable.config.BigtableSinkConfig#CONFIG_ROW_KEY_DEFINITION}. + * com.google.cloud.kafka.connect.bigtable.config.BigtableSinkConfig#ROW_KEY_DEFINITION_CONFIG}. */ public KeyMapper(String delimiter, List definition) { this.delimiter = delimiter.getBytes(StandardCharsets.UTF_8); diff --git a/google-cloud-bigtable-kafka-connect-sink/src/main/java/com/google/cloud/kafka/connect/bigtable/mapping/ValueMapper.java b/google-cloud-bigtable-kafka-connect-sink/src/main/java/com/google/cloud/kafka/connect/bigtable/mapping/ValueMapper.java index 3255e1c440..a990f22ef4 100644 --- a/google-cloud-bigtable-kafka-connect-sink/src/main/java/com/google/cloud/kafka/connect/bigtable/mapping/ValueMapper.java +++ b/google-cloud-bigtable-kafka-connect-sink/src/main/java/com/google/cloud/kafka/connect/bigtable/mapping/ValueMapper.java @@ -65,9 +65,9 @@ public class ValueMapper { * The main constructor. * * @param defaultColumnFamily Default column family as per {@link - * com.google.cloud.kafka.connect.bigtable.config.BigtableSinkConfig#CONFIG_DEFAULT_COLUMN_FAMILY}. + * com.google.cloud.kafka.connect.bigtable.config.BigtableSinkConfig#DEFAULT_COLUMN_FAMILY_CONFIG}. * @param defaultColumnQualifier Default column as per {@link - * com.google.cloud.kafka.connect.bigtable.config.BigtableSinkConfig#CONFIG_ROW_KEY_DELIMITER}. + * com.google.cloud.kafka.connect.bigtable.config.BigtableSinkConfig#ROW_KEY_DELIMITER_CONFIG}. */ public ValueMapper( String defaultColumnFamily, String defaultColumnQualifier, @Nonnull NullValueMode nullMode) { diff --git a/google-cloud-bigtable-kafka-connect-sink/src/test/java/com/google/cloud/kafka/connect/bigtable/BigtableSinkConnectorTest.java b/google-cloud-bigtable-kafka-connect-sink/src/test/java/com/google/cloud/kafka/connect/bigtable/BigtableSinkConnectorTest.java index 6b3141d80e..9d04c010d7 100644 --- a/google-cloud-bigtable-kafka-connect-sink/src/test/java/com/google/cloud/kafka/connect/bigtable/BigtableSinkConnectorTest.java +++ b/google-cloud-bigtable-kafka-connect-sink/src/test/java/com/google/cloud/kafka/connect/bigtable/BigtableSinkConnectorTest.java @@ -15,7 +15,7 @@ */ package com.google.cloud.kafka.connect.bigtable; -import static com.google.cloud.kafka.connect.bigtable.config.BigtableSinkTaskConfig.CONFIG_TASK_ID; +import static com.google.cloud.kafka.connect.bigtable.config.BigtableSinkTaskConfig.TASK_ID_CONFIG; import static org.junit.Assert.assertEquals; import static org.junit.Assert.assertNotNull; @@ -71,8 +71,8 @@ public void testTaskConfigs() { assertEquals(maxTasks, taskConfigs.size()); for (Integer i = 0; i < maxTasks; i++) { Map taskConfig = taskConfigs.get(i); - assertEquals(i.toString(), taskConfig.get(CONFIG_TASK_ID)); - taskConfig.remove(CONFIG_TASK_ID); + assertEquals(i.toString(), taskConfig.get(TASK_ID_CONFIG)); + taskConfig.remove(TASK_ID_CONFIG); assertEquals(connectorConfig, taskConfig); } } diff --git a/google-cloud-bigtable-kafka-connect-sink/src/test/java/com/google/cloud/kafka/connect/bigtable/BigtableSinkTaskTest.java b/google-cloud-bigtable-kafka-connect-sink/src/test/java/com/google/cloud/kafka/connect/bigtable/BigtableSinkTaskTest.java index eab9857aa4..ff2d573737 100644 --- a/google-cloud-bigtable-kafka-connect-sink/src/test/java/com/google/cloud/kafka/connect/bigtable/BigtableSinkTaskTest.java +++ b/google-cloud-bigtable-kafka-connect-sink/src/test/java/com/google/cloud/kafka/connect/bigtable/BigtableSinkTaskTest.java @@ -15,11 +15,11 @@ */ package com.google.cloud.kafka.connect.bigtable; -import static com.google.cloud.kafka.connect.bigtable.config.BigtableSinkConfig.CONFIG_AUTO_CREATE_COLUMN_FAMILIES; -import static com.google.cloud.kafka.connect.bigtable.config.BigtableSinkConfig.CONFIG_AUTO_CREATE_TABLES; -import static com.google.cloud.kafka.connect.bigtable.config.BigtableSinkConfig.CONFIG_ERROR_MODE; -import static com.google.cloud.kafka.connect.bigtable.config.BigtableSinkConfig.CONFIG_INSERT_MODE; -import static com.google.cloud.kafka.connect.bigtable.config.BigtableSinkConfig.CONFIG_TABLE_NAME_FORMAT; +import static com.google.cloud.kafka.connect.bigtable.config.BigtableSinkConfig.AUTO_CREATE_COLUMN_FAMILIES_CONFIG; +import static com.google.cloud.kafka.connect.bigtable.config.BigtableSinkConfig.AUTO_CREATE_TABLES_CONFIG; +import static com.google.cloud.kafka.connect.bigtable.config.BigtableSinkConfig.ERROR_MODE_CONFIG; +import static com.google.cloud.kafka.connect.bigtable.config.BigtableSinkConfig.INSERT_MODE_CONFIG; +import static com.google.cloud.kafka.connect.bigtable.config.BigtableSinkConfig.TABLE_NAME_FORMAT_CONFIG; import static com.google.cloud.kafka.connect.bigtable.util.FutureUtil.completedApiFuture; import static com.google.cloud.kafka.connect.bigtable.util.MockUtil.assertTotalNumberOfInvocations; import static org.junit.Assert.assertEquals; @@ -166,7 +166,7 @@ public void testGetTableName() { String tableFormat = "table"; SinkRecord record = new SinkRecord("topic", 1, null, null, null, null, 1); Map props = BasicPropertiesFactory.getTaskProps(); - props.put(CONFIG_TABLE_NAME_FORMAT, tableFormat); + props.put(TABLE_NAME_FORMAT_CONFIG, tableFormat); task = new TestBigtableSinkTask( new BigtableSinkTaskConfig(props), null, null, null, null, null, null); @@ -207,7 +207,7 @@ public void testErrorReporterWithDLQ() { @Test public void testErrorReporterNoDLQIgnoreMode() { Map props = BasicPropertiesFactory.getTaskProps(); - props.put(CONFIG_ERROR_MODE, BigtableErrorMode.IGNORE.name()); + props.put(ERROR_MODE_CONFIG, BigtableErrorMode.IGNORE.name()); BigtableSinkTaskConfig config = new BigtableSinkTaskConfig(props); doThrow(new NoSuchMethodError()).when(context).errantRecordReporter(); @@ -221,7 +221,7 @@ public void testErrorReporterNoDLQIgnoreMode() { @Test public void testErrorReporterNoDLQWarnMode() { Map props = BasicPropertiesFactory.getTaskProps(); - props.put(CONFIG_ERROR_MODE, BigtableErrorMode.WARN.name()); + props.put(ERROR_MODE_CONFIG, BigtableErrorMode.WARN.name()); BigtableSinkTaskConfig config = new BigtableSinkTaskConfig(props); doReturn(null).when(context).errantRecordReporter(); @@ -236,7 +236,7 @@ public void testErrorReporterNoDLQWarnMode() { @Test public void testErrorReporterNoDLQFailMode() { Map props = BasicPropertiesFactory.getTaskProps(); - props.put(CONFIG_ERROR_MODE, BigtableErrorMode.FAIL.name()); + props.put(ERROR_MODE_CONFIG, BigtableErrorMode.FAIL.name()); BigtableSinkTaskConfig config = new BigtableSinkTaskConfig(props); doReturn(null).when(context).errantRecordReporter(); @@ -470,7 +470,7 @@ public void testUpsertRows() { Map props = BasicPropertiesFactory.getTaskProps(); int maxBatchSize = 3; int totalRecords = 1000; - props.put(BigtableSinkTaskConfig.CONFIG_MAX_BATCH_SIZE, Integer.toString(maxBatchSize)); + props.put(BigtableSinkTaskConfig.MAX_BATCH_SIZE_CONFIG, Integer.toString(maxBatchSize)); BigtableSinkTaskConfig config = new BigtableSinkTaskConfig(props); task = spy(new TestBigtableSinkTask(config, null, null, null, null, null, null)); @@ -574,9 +574,9 @@ public void testPutBranches() { boolean useInsertMode = test.get(2); Map props = BasicPropertiesFactory.getTaskProps(); - props.put(CONFIG_AUTO_CREATE_TABLES, Boolean.toString(autoCreateTables)); - props.put(CONFIG_AUTO_CREATE_COLUMN_FAMILIES, Boolean.toString(autoCreateColumnFamilies)); - props.put(CONFIG_INSERT_MODE, (useInsertMode ? InsertMode.INSERT : InsertMode.UPSERT).name()); + props.put(AUTO_CREATE_TABLES_CONFIG, Boolean.toString(autoCreateTables)); + props.put(AUTO_CREATE_COLUMN_FAMILIES_CONFIG, Boolean.toString(autoCreateColumnFamilies)); + props.put(INSERT_MODE_CONFIG, (useInsertMode ? InsertMode.INSERT : InsertMode.UPSERT).name()); config = new BigtableSinkTaskConfig(props); byte[] rowKey = "rowKey".getBytes(StandardCharsets.UTF_8); diff --git a/google-cloud-bigtable-kafka-connect-sink/src/test/java/com/google/cloud/kafka/connect/bigtable/config/BigtableSinkConfigTest.java b/google-cloud-bigtable-kafka-connect-sink/src/test/java/com/google/cloud/kafka/connect/bigtable/config/BigtableSinkConfigTest.java index 2aeb80e483..b63ec6dacd 100644 --- a/google-cloud-bigtable-kafka-connect-sink/src/test/java/com/google/cloud/kafka/connect/bigtable/config/BigtableSinkConfigTest.java +++ b/google-cloud-bigtable-kafka-connect-sink/src/test/java/com/google/cloud/kafka/connect/bigtable/config/BigtableSinkConfigTest.java @@ -15,13 +15,13 @@ */ package com.google.cloud.kafka.connect.bigtable.config; -import static com.google.cloud.kafka.connect.bigtable.config.BigtableSinkConfig.CONFIG_AUTO_CREATE_COLUMN_FAMILIES; -import static com.google.cloud.kafka.connect.bigtable.config.BigtableSinkConfig.CONFIG_AUTO_CREATE_TABLES; -import static com.google.cloud.kafka.connect.bigtable.config.BigtableSinkConfig.CONFIG_ERROR_MODE; -import static com.google.cloud.kafka.connect.bigtable.config.BigtableSinkConfig.CONFIG_INSERT_MODE; -import static com.google.cloud.kafka.connect.bigtable.config.BigtableSinkConfig.CONFIG_MAX_BATCH_SIZE; -import static com.google.cloud.kafka.connect.bigtable.config.BigtableSinkConfig.CONFIG_TABLE_NAME_FORMAT; -import static com.google.cloud.kafka.connect.bigtable.config.BigtableSinkConfig.CONFIG_VALUE_NULL_MODE; +import static com.google.cloud.kafka.connect.bigtable.config.BigtableSinkConfig.AUTO_CREATE_COLUMN_FAMILIES_CONFIG; +import static com.google.cloud.kafka.connect.bigtable.config.BigtableSinkConfig.AUTO_CREATE_TABLES_CONFIG; +import static com.google.cloud.kafka.connect.bigtable.config.BigtableSinkConfig.ERROR_MODE_CONFIG; +import static com.google.cloud.kafka.connect.bigtable.config.BigtableSinkConfig.INSERT_MODE_CONFIG; +import static com.google.cloud.kafka.connect.bigtable.config.BigtableSinkConfig.MAX_BATCH_SIZE_CONFIG; +import static com.google.cloud.kafka.connect.bigtable.config.BigtableSinkConfig.TABLE_NAME_FORMAT_CONFIG; +import static com.google.cloud.kafka.connect.bigtable.config.BigtableSinkConfig.VALUE_NULL_MODE_CONFIG; import static java.util.Collections.emptyList; import static org.junit.Assert.assertEquals; import static org.junit.Assert.assertFalse; @@ -64,15 +64,15 @@ public void testBasicValidationFailure() { assertThrows(ConfigException.class, () -> new BigtableSinkConfig(new HashMap<>())); for (String configName : List.of( - CONFIG_TABLE_NAME_FORMAT, - CONFIG_AUTO_CREATE_TABLES, - CONFIG_AUTO_CREATE_COLUMN_FAMILIES)) { + TABLE_NAME_FORMAT_CONFIG, + AUTO_CREATE_TABLES_CONFIG, + AUTO_CREATE_COLUMN_FAMILIES_CONFIG)) { Map props = BasicPropertiesFactory.getSinkProps(); props.put(configName, null); assertThrows(ConfigException.class, () -> new BigtableSinkConfig(new HashMap<>())); } for (String configName : - List.of(CONFIG_INSERT_MODE, CONFIG_VALUE_NULL_MODE, CONFIG_ERROR_MODE)) { + List.of(INSERT_MODE_CONFIG, VALUE_NULL_MODE_CONFIG, ERROR_MODE_CONFIG)) { Map props = BasicPropertiesFactory.getSinkProps(); props.put(configName, "invalid"); assertThrows(ConfigException.class, () -> new BigtableSinkConfig(new HashMap<>())); @@ -82,16 +82,16 @@ public void testBasicValidationFailure() { @Test public void testDefaults() { BigtableSinkConfig config = new BigtableSinkConfig(BasicPropertiesFactory.getSinkProps()); - assertEquals(config.getString(CONFIG_INSERT_MODE), InsertMode.INSERT.name()); - assertEquals((long) config.getInt(CONFIG_MAX_BATCH_SIZE), 1); - assertEquals(config.getString(CONFIG_VALUE_NULL_MODE), NullValueMode.WRITE.name()); + assertEquals(config.getString(INSERT_MODE_CONFIG), InsertMode.INSERT.name()); + assertEquals((long) config.getInt(MAX_BATCH_SIZE_CONFIG), 1); + assertEquals(config.getString(VALUE_NULL_MODE_CONFIG), NullValueMode.WRITE.name()); } @Test public void testInsertModeOnlyAllowsMaxBatchSizeOf1() { Map props = BasicPropertiesFactory.getSinkProps(); - props.put(BigtableSinkConfig.CONFIG_INSERT_MODE, InsertMode.INSERT.name()); - props.put(BigtableSinkConfig.CONFIG_MAX_BATCH_SIZE, "2"); + props.put(BigtableSinkConfig.INSERT_MODE_CONFIG, InsertMode.INSERT.name()); + props.put(BigtableSinkConfig.MAX_BATCH_SIZE_CONFIG, "2"); BigtableSinkConfig config = new BigtableSinkConfig(props); assertFalse(configIsValid(config)); } @@ -99,8 +99,8 @@ public void testInsertModeOnlyAllowsMaxBatchSizeOf1() { @Test public void testMultipleCredentialsAreDisallowed() { Map props = BasicPropertiesFactory.getSinkProps(); - props.put(BigtableSinkConfig.CONFIG_GCP_CREDENTIALS_JSON, "nonempty"); - props.put(BigtableSinkConfig.CONFIG_GCP_CREDENTIALS_PATH, "nonempty"); + props.put(BigtableSinkConfig.GCP_CREDENTIALS_JSON_CONFIG, "nonempty"); + props.put(BigtableSinkConfig.GCP_CREDENTIALS_PATH_CONFIG, "nonempty"); BigtableSinkConfig config = new BigtableSinkConfig(props); assertFalse(configIsValid(config)); } @@ -108,8 +108,8 @@ public void testMultipleCredentialsAreDisallowed() { @Test public void testNullDeletionIsIncompatibleWithInsertMode() { Map props = BasicPropertiesFactory.getSinkProps(); - props.put(BigtableSinkConfig.CONFIG_INSERT_MODE, InsertMode.INSERT.name()); - props.put(CONFIG_VALUE_NULL_MODE, NullValueMode.DELETE.name()); + props.put(BigtableSinkConfig.INSERT_MODE_CONFIG, InsertMode.INSERT.name()); + props.put(VALUE_NULL_MODE_CONFIG, NullValueMode.DELETE.name()); BigtableSinkConfig config = new BigtableSinkConfig(props); assertFalse(configIsValid(config)); } @@ -131,9 +131,9 @@ public void testGetBigtableAdminClient() { @Test public void testEnumCaseInsensitivity() { Map props = BasicPropertiesFactory.getSinkProps(); - props.put(CONFIG_INSERT_MODE, "uPsErT"); - props.put(CONFIG_ERROR_MODE, "IGNORE"); - props.put(CONFIG_VALUE_NULL_MODE, "delete"); + props.put(INSERT_MODE_CONFIG, "uPsErT"); + props.put(ERROR_MODE_CONFIG, "IGNORE"); + props.put(VALUE_NULL_MODE_CONFIG, "delete"); BigtableSinkConfig config = new BigtableSinkConfig(props); } diff --git a/google-cloud-bigtable-kafka-connect-sink/src/test/java/com/google/cloud/kafka/connect/bigtable/integration/BaseIT.java b/google-cloud-bigtable-kafka-connect-sink/src/test/java/com/google/cloud/kafka/connect/bigtable/integration/BaseIT.java index 34d6b8abc9..e31dafb911 100644 --- a/google-cloud-bigtable-kafka-connect-sink/src/test/java/com/google/cloud/kafka/connect/bigtable/integration/BaseIT.java +++ b/google-cloud-bigtable-kafka-connect-sink/src/test/java/com/google/cloud/kafka/connect/bigtable/integration/BaseIT.java @@ -15,8 +15,8 @@ */ package com.google.cloud.kafka.connect.bigtable.integration; -import static com.google.cloud.kafka.connect.bigtable.config.BigtableSinkConfig.CONFIG_BIGTABLE_INSTANCE_ID; -import static com.google.cloud.kafka.connect.bigtable.config.BigtableSinkConfig.CONFIG_GCP_PROJECT_ID; +import static com.google.cloud.kafka.connect.bigtable.config.BigtableSinkConfig.BIGTABLE_INSTANCE_ID_CONFIG; +import static com.google.cloud.kafka.connect.bigtable.config.BigtableSinkConfig.GCP_PROJECT_ID_CONFIG; import static org.apache.kafka.connect.runtime.ConnectorConfig.CONNECTOR_CLASS_CONFIG; import static org.apache.kafka.connect.runtime.ConnectorConfig.TASKS_MAX_CONFIG; import static org.apache.kafka.connect.runtime.WorkerConfig.KEY_CONVERTER_CLASS_CONFIG; @@ -136,8 +136,8 @@ protected Map baseConnectorProps() { String.valueOf(maxKafkaMessageSizeBytes)); // TODO: get it from environment variables after migrating to kokoro. - result.put(CONFIG_GCP_PROJECT_ID, "todotodo"); - result.put(CONFIG_BIGTABLE_INSTANCE_ID, "todotodo"); + result.put(GCP_PROJECT_ID_CONFIG, "todotodo"); + result.put(BIGTABLE_INSTANCE_ID_CONFIG, "todotodo"); return result; } diff --git a/google-cloud-bigtable-kafka-connect-sink/src/test/java/com/google/cloud/kafka/connect/bigtable/integration/BasicIT.java b/google-cloud-bigtable-kafka-connect-sink/src/test/java/com/google/cloud/kafka/connect/bigtable/integration/BasicIT.java index fbf77c6bd3..d474f4b9ab 100644 --- a/google-cloud-bigtable-kafka-connect-sink/src/test/java/com/google/cloud/kafka/connect/bigtable/integration/BasicIT.java +++ b/google-cloud-bigtable-kafka-connect-sink/src/test/java/com/google/cloud/kafka/connect/bigtable/integration/BasicIT.java @@ -34,8 +34,8 @@ public class BasicIT extends BaseKafkaConnectBigtableIT { @Test public void testSimpleWrite() throws InterruptedException { Map props = baseConnectorProps(); - props.put(BigtableSinkConfig.CONFIG_AUTO_CREATE_TABLES, "true"); - props.put(BigtableSinkConfig.CONFIG_AUTO_CREATE_COLUMN_FAMILIES, "true"); + props.put(BigtableSinkConfig.AUTO_CREATE_TABLES_CONFIG, "true"); + props.put(BigtableSinkConfig.AUTO_CREATE_COLUMN_FAMILIES_CONFIG, "true"); String topic = startSingleTopicConnector(props); String key = "key"; diff --git a/google-cloud-bigtable-kafka-connect-sink/src/test/java/com/google/cloud/kafka/connect/bigtable/integration/ConfluentCompatibilityIT.java b/google-cloud-bigtable-kafka-connect-sink/src/test/java/com/google/cloud/kafka/connect/bigtable/integration/ConfluentCompatibilityIT.java index 96523db31a..8d821d0cde 100644 --- a/google-cloud-bigtable-kafka-connect-sink/src/test/java/com/google/cloud/kafka/connect/bigtable/integration/ConfluentCompatibilityIT.java +++ b/google-cloud-bigtable-kafka-connect-sink/src/test/java/com/google/cloud/kafka/connect/bigtable/integration/ConfluentCompatibilityIT.java @@ -140,10 +140,10 @@ public String startConnector() throws InterruptedException { + "." + AbstractKafkaSchemaSerDeConfig.SCHEMA_REGISTRY_URL_CONFIG, schemaRegistry.schemaRegistryUrl()); - connectorProps.put(BigtableSinkConfig.CONFIG_AUTO_CREATE_TABLES, "true"); - connectorProps.put(BigtableSinkConfig.CONFIG_AUTO_CREATE_COLUMN_FAMILIES, "true"); - connectorProps.put(BigtableSinkConfig.CONFIG_DEFAULT_COLUMN_FAMILY, COMPATIBILITY_TEST_TOPIC); - connectorProps.put(BigtableSinkConfig.CONFIG_ROW_KEY_DELIMITER, "#"); + connectorProps.put(BigtableSinkConfig.AUTO_CREATE_TABLES_CONFIG, "true"); + connectorProps.put(BigtableSinkConfig.AUTO_CREATE_COLUMN_FAMILIES_CONFIG, "true"); + connectorProps.put(BigtableSinkConfig.DEFAULT_COLUMN_FAMILY_CONFIG, COMPATIBILITY_TEST_TOPIC); + connectorProps.put(BigtableSinkConfig.ROW_KEY_DELIMITER_CONFIG, "#"); String topic = startSingleTopicConnector(connectorProps); connect .assertions() diff --git a/google-cloud-bigtable-kafka-connect-sink/src/test/java/com/google/cloud/kafka/connect/bigtable/integration/DifferentConvertersIT.java b/google-cloud-bigtable-kafka-connect-sink/src/test/java/com/google/cloud/kafka/connect/bigtable/integration/DifferentConvertersIT.java index d73188d26e..44152d60f1 100644 --- a/google-cloud-bigtable-kafka-connect-sink/src/test/java/com/google/cloud/kafka/connect/bigtable/integration/DifferentConvertersIT.java +++ b/google-cloud-bigtable-kafka-connect-sink/src/test/java/com/google/cloud/kafka/connect/bigtable/integration/DifferentConvertersIT.java @@ -169,8 +169,8 @@ public void testConverter() throws InterruptedException { protected Map connectorProps() { Map props = baseConnectorProps(); - props.put(BigtableSinkConfig.CONFIG_AUTO_CREATE_TABLES, "true"); - props.put(BigtableSinkConfig.CONFIG_AUTO_CREATE_COLUMN_FAMILIES, "true"); + props.put(BigtableSinkConfig.AUTO_CREATE_TABLES_CONFIG, "true"); + props.put(BigtableSinkConfig.AUTO_CREATE_COLUMN_FAMILIES_CONFIG, "true"); return props; } diff --git a/google-cloud-bigtable-kafka-connect-sink/src/test/java/com/google/cloud/kafka/connect/bigtable/integration/ErrorHandlingIT.java b/google-cloud-bigtable-kafka-connect-sink/src/test/java/com/google/cloud/kafka/connect/bigtable/integration/ErrorHandlingIT.java index bf181c70a8..e664f99273 100644 --- a/google-cloud-bigtable-kafka-connect-sink/src/test/java/com/google/cloud/kafka/connect/bigtable/integration/ErrorHandlingIT.java +++ b/google-cloud-bigtable-kafka-connect-sink/src/test/java/com/google/cloud/kafka/connect/bigtable/integration/ErrorHandlingIT.java @@ -59,7 +59,7 @@ public class ErrorHandlingIT extends BaseKafkaConnectBigtableIT { @Test public void testBigtableCredentialsAreCheckedOnStartup() { Map props = baseConnectorProps(); - props.put(BigtableSinkConfig.CONFIG_GCP_CREDENTIALS_JSON, "{}"); + props.put(BigtableSinkConfig.GCP_CREDENTIALS_JSON_CONFIG, "{}"); String testId = getTestCaseId(); assertThrows(Throwable.class, () -> connect.configureConnector(testId, props)); @@ -70,8 +70,8 @@ public void testBigtableCredentialsAreCheckedOnStartup() { public void testTooLargeData() throws InterruptedException, ExecutionException { String dlqTopic = createDlq(); Map props = baseConnectorProps(); - props.put(BigtableSinkConfig.CONFIG_AUTO_CREATE_TABLES, String.valueOf(true)); - props.put(BigtableSinkConfig.CONFIG_AUTO_CREATE_COLUMN_FAMILIES, String.valueOf(true)); + props.put(BigtableSinkConfig.AUTO_CREATE_TABLES_CONFIG, String.valueOf(true)); + props.put(BigtableSinkConfig.AUTO_CREATE_COLUMN_FAMILIES_CONFIG, String.valueOf(true)); props.put(ConnectorConfig.VALUE_CONVERTER_CLASS_CONFIG, ByteArrayConverter.class.getName()); configureDlq(props, dlqTopic); String testId = startSingleTopicConnector(props); @@ -90,8 +90,8 @@ public void testTooLargeData() throws InterruptedException, ExecutionException { public void testSecondInsertIntoARowCausesAnError() throws InterruptedException { String dlqTopic = createDlq(); Map props = baseConnectorProps(); - props.put(BigtableSinkConfig.CONFIG_AUTO_CREATE_TABLES, String.valueOf(true)); - props.put(BigtableSinkConfig.CONFIG_AUTO_CREATE_COLUMN_FAMILIES, String.valueOf(true)); + props.put(BigtableSinkConfig.AUTO_CREATE_TABLES_CONFIG, String.valueOf(true)); + props.put(BigtableSinkConfig.AUTO_CREATE_COLUMN_FAMILIES_CONFIG, String.valueOf(true)); configureDlq(props, dlqTopic); String testId = startSingleTopicConnector(props); @@ -121,9 +121,9 @@ public void testPartialBatchErrorWhenRelyingOnInputOrdering() throws Interrupted String dlqTopic = createDlq(); Map props = baseConnectorProps(); - props.put(BigtableSinkConfig.CONFIG_AUTO_CREATE_TABLES, String.valueOf(true)); - props.put(BigtableSinkConfig.CONFIG_AUTO_CREATE_COLUMN_FAMILIES, String.valueOf(true)); - props.put(BigtableSinkConfig.CONFIG_INSERT_MODE, InsertMode.INSERT.name()); + props.put(BigtableSinkConfig.AUTO_CREATE_TABLES_CONFIG, String.valueOf(true)); + props.put(BigtableSinkConfig.AUTO_CREATE_COLUMN_FAMILIES_CONFIG, String.valueOf(true)); + props.put(BigtableSinkConfig.INSERT_MODE_CONFIG, InsertMode.INSERT.name()); props.put( ConnectorConfig.CONNECTOR_CLIENT_CONSUMER_OVERRIDES_PREFIX + ConsumerConfig.MAX_POLL_RECORDS_CONFIG, @@ -190,10 +190,10 @@ public void testDeletingARowTwiceWorks() throws InterruptedException, ExecutionException, TimeoutException { String dlqTopic = createDlq(); Map props = baseConnectorProps(); - props.put(BigtableSinkConfig.CONFIG_AUTO_CREATE_TABLES, String.valueOf(true)); - props.put(BigtableSinkConfig.CONFIG_AUTO_CREATE_COLUMN_FAMILIES, String.valueOf(true)); - props.put(BigtableSinkConfig.CONFIG_VALUE_NULL_MODE, NullValueMode.DELETE.name()); - props.put(BigtableSinkConfig.CONFIG_INSERT_MODE, InsertMode.UPSERT.name()); + props.put(BigtableSinkConfig.AUTO_CREATE_TABLES_CONFIG, String.valueOf(true)); + props.put(BigtableSinkConfig.AUTO_CREATE_COLUMN_FAMILIES_CONFIG, String.valueOf(true)); + props.put(BigtableSinkConfig.VALUE_NULL_MODE_CONFIG, NullValueMode.DELETE.name()); + props.put(BigtableSinkConfig.INSERT_MODE_CONFIG, InsertMode.UPSERT.name()); configureDlq(props, dlqTopic); props.put(ConnectorConfig.VALUE_CONVERTER_CLASS_CONFIG, JsonConverter.class.getName()); props.put( @@ -228,10 +228,10 @@ public void testNonexistentCellDeletionWorks() throws InterruptedException, ExecutionException, TimeoutException { String dlqTopic = createDlq(); Map props = baseConnectorProps(); - props.put(BigtableSinkConfig.CONFIG_AUTO_CREATE_TABLES, String.valueOf(true)); - props.put(BigtableSinkConfig.CONFIG_AUTO_CREATE_COLUMN_FAMILIES, String.valueOf(true)); - props.put(BigtableSinkConfig.CONFIG_VALUE_NULL_MODE, NullValueMode.DELETE.name()); - props.put(BigtableSinkConfig.CONFIG_INSERT_MODE, InsertMode.UPSERT.name()); + props.put(BigtableSinkConfig.AUTO_CREATE_TABLES_CONFIG, String.valueOf(true)); + props.put(BigtableSinkConfig.AUTO_CREATE_COLUMN_FAMILIES_CONFIG, String.valueOf(true)); + props.put(BigtableSinkConfig.VALUE_NULL_MODE_CONFIG, NullValueMode.DELETE.name()); + props.put(BigtableSinkConfig.INSERT_MODE_CONFIG, InsertMode.UPSERT.name()); configureDlq(props, dlqTopic); props.put(ConnectorConfig.VALUE_CONVERTER_CLASS_CONFIG, JsonConverter.class.getName()); props.put( @@ -260,10 +260,10 @@ public void testNonexistentColumnFamilyDeletionWorks() throws ExecutionException, InterruptedException, TimeoutException { String dlqTopic = createDlq(); Map props = baseConnectorProps(); - props.put(BigtableSinkConfig.CONFIG_AUTO_CREATE_TABLES, String.valueOf(true)); - props.put(BigtableSinkConfig.CONFIG_AUTO_CREATE_COLUMN_FAMILIES, String.valueOf(true)); - props.put(BigtableSinkConfig.CONFIG_VALUE_NULL_MODE, NullValueMode.DELETE.name()); - props.put(BigtableSinkConfig.CONFIG_INSERT_MODE, InsertMode.UPSERT.name()); + props.put(BigtableSinkConfig.AUTO_CREATE_TABLES_CONFIG, String.valueOf(true)); + props.put(BigtableSinkConfig.AUTO_CREATE_COLUMN_FAMILIES_CONFIG, String.valueOf(true)); + props.put(BigtableSinkConfig.VALUE_NULL_MODE_CONFIG, NullValueMode.DELETE.name()); + props.put(BigtableSinkConfig.INSERT_MODE_CONFIG, InsertMode.UPSERT.name()); configureDlq(props, dlqTopic); props.put(ConnectorConfig.VALUE_CONVERTER_CLASS_CONFIG, JsonConverter.class.getName()); props.put( diff --git a/google-cloud-bigtable-kafka-connect-sink/src/test/java/com/google/cloud/kafka/connect/bigtable/integration/ErrorReportingIT.java b/google-cloud-bigtable-kafka-connect-sink/src/test/java/com/google/cloud/kafka/connect/bigtable/integration/ErrorReportingIT.java index 7df9a03bc2..3ca43dd473 100644 --- a/google-cloud-bigtable-kafka-connect-sink/src/test/java/com/google/cloud/kafka/connect/bigtable/integration/ErrorReportingIT.java +++ b/google-cloud-bigtable-kafka-connect-sink/src/test/java/com/google/cloud/kafka/connect/bigtable/integration/ErrorReportingIT.java @@ -27,7 +27,7 @@ public class ErrorReportingIT extends BaseKafkaConnectIT { @Test public void testErrorModeFail() throws InterruptedException { Map props = baseConnectorProps(); - props.put(BigtableSinkConfig.CONFIG_ERROR_MODE, BigtableErrorMode.FAIL.name()); + props.put(BigtableSinkConfig.ERROR_MODE_CONFIG, BigtableErrorMode.FAIL.name()); String testId = startSingleTopicConnector(props); connect.kafka().produce(testId, "key", "value"); @@ -40,7 +40,7 @@ public void testErrorModeFail() throws InterruptedException { @Test public void testErrorModeWarn() throws InterruptedException { Map props = baseConnectorProps(); - props.put(BigtableSinkConfig.CONFIG_ERROR_MODE, BigtableErrorMode.WARN.name()); + props.put(BigtableSinkConfig.ERROR_MODE_CONFIG, BigtableErrorMode.WARN.name()); String testId = startSingleTopicConnector(props); connect.kafka().produce(testId, "key", "value"); @@ -53,7 +53,7 @@ public void testErrorModeWarn() throws InterruptedException { @Test public void testErrorModeIgnore() throws InterruptedException { Map props = baseConnectorProps(); - props.put(BigtableSinkConfig.CONFIG_ERROR_MODE, BigtableErrorMode.IGNORE.name()); + props.put(BigtableSinkConfig.ERROR_MODE_CONFIG, BigtableErrorMode.IGNORE.name()); String testId = startSingleTopicConnector(props); connect.kafka().produce(testId, "key", "value"); @@ -67,7 +67,7 @@ public void testErrorModeIgnore() throws InterruptedException { public void testErrorModeDLQOverridesErrorMode() throws InterruptedException { String dlqTopic = createDlq(); Map props = baseConnectorProps(); - props.put(BigtableSinkConfig.CONFIG_ERROR_MODE, BigtableErrorMode.FAIL.name()); + props.put(BigtableSinkConfig.ERROR_MODE_CONFIG, BigtableErrorMode.FAIL.name()); configureDlq(props, dlqTopic); String key = "key"; diff --git a/google-cloud-bigtable-kafka-connect-sink/src/test/java/com/google/cloud/kafka/connect/bigtable/integration/InsertUpsertIT.java b/google-cloud-bigtable-kafka-connect-sink/src/test/java/com/google/cloud/kafka/connect/bigtable/integration/InsertUpsertIT.java index cb0a5960d5..855afa88d1 100644 --- a/google-cloud-bigtable-kafka-connect-sink/src/test/java/com/google/cloud/kafka/connect/bigtable/integration/InsertUpsertIT.java +++ b/google-cloud-bigtable-kafka-connect-sink/src/test/java/com/google/cloud/kafka/connect/bigtable/integration/InsertUpsertIT.java @@ -43,10 +43,10 @@ public class InsertUpsertIT extends BaseKafkaConnectBigtableIT { @Test public void testInsert() throws InterruptedException { Map props = baseConnectorProps(); - props.put(BigtableSinkConfig.CONFIG_AUTO_CREATE_TABLES, "true"); - props.put(BigtableSinkConfig.CONFIG_AUTO_CREATE_COLUMN_FAMILIES, "true"); - props.put(BigtableSinkConfig.CONFIG_INSERT_MODE, InsertMode.INSERT.name()); - props.put(BigtableSinkConfig.CONFIG_ERROR_MODE, BigtableErrorMode.IGNORE.name()); + props.put(BigtableSinkConfig.AUTO_CREATE_TABLES_CONFIG, "true"); + props.put(BigtableSinkConfig.AUTO_CREATE_COLUMN_FAMILIES_CONFIG, "true"); + props.put(BigtableSinkConfig.INSERT_MODE_CONFIG, InsertMode.INSERT.name()); + props.put(BigtableSinkConfig.ERROR_MODE_CONFIG, BigtableErrorMode.IGNORE.name()); String testId = startSingleTopicConnector(props); connect.kafka().produce(testId, KEY1, VALUE1); @@ -65,10 +65,10 @@ public void testInsert() throws InterruptedException { @Test public void testUpsert() throws InterruptedException { Map props = baseConnectorProps(); - props.put(BigtableSinkConfig.CONFIG_AUTO_CREATE_TABLES, "true"); - props.put(BigtableSinkConfig.CONFIG_AUTO_CREATE_COLUMN_FAMILIES, "true"); - props.put(BigtableSinkConfig.CONFIG_INSERT_MODE, InsertMode.UPSERT.name()); - props.put(BigtableSinkConfig.CONFIG_ERROR_MODE, BigtableErrorMode.IGNORE.name()); + props.put(BigtableSinkConfig.AUTO_CREATE_TABLES_CONFIG, "true"); + props.put(BigtableSinkConfig.AUTO_CREATE_COLUMN_FAMILIES_CONFIG, "true"); + props.put(BigtableSinkConfig.INSERT_MODE_CONFIG, InsertMode.UPSERT.name()); + props.put(BigtableSinkConfig.ERROR_MODE_CONFIG, BigtableErrorMode.IGNORE.name()); String testId = startSingleTopicConnector(props); connect.kafka().produce(testId, KEY1, VALUE1); diff --git a/google-cloud-bigtable-kafka-connect-sink/src/test/java/com/google/cloud/kafka/connect/bigtable/integration/NullHandlingIT.java b/google-cloud-bigtable-kafka-connect-sink/src/test/java/com/google/cloud/kafka/connect/bigtable/integration/NullHandlingIT.java index e4507de1db..1bb0751064 100644 --- a/google-cloud-bigtable-kafka-connect-sink/src/test/java/com/google/cloud/kafka/connect/bigtable/integration/NullHandlingIT.java +++ b/google-cloud-bigtable-kafka-connect-sink/src/test/java/com/google/cloud/kafka/connect/bigtable/integration/NullHandlingIT.java @@ -76,7 +76,7 @@ public class NullHandlingIT extends BaseKafkaConnectBigtableIT { @Test public void testIgnoreMode() throws InterruptedException { Map connectorProps = connectorProps(); - connectorProps.put(BigtableSinkConfig.CONFIG_VALUE_NULL_MODE, NullValueMode.IGNORE.name()); + connectorProps.put(BigtableSinkConfig.VALUE_NULL_MODE_CONFIG, NullValueMode.IGNORE.name()); String testId = startSingleTopicConnector(connectorProps); connect .assertions() @@ -132,9 +132,9 @@ public void testWriteMode() throws InterruptedException { ByteString.copyFrom(defaultColumnQualifier.getBytes(StandardCharsets.UTF_8)); Map connectorProps = connectorProps(); - connectorProps.put(BigtableSinkConfig.CONFIG_VALUE_NULL_MODE, NullValueMode.WRITE.name()); - connectorProps.put(BigtableSinkConfig.CONFIG_DEFAULT_COLUMN_FAMILY, defaultColumnFamily); - connectorProps.put(BigtableSinkConfig.CONFIG_DEFAULT_COLUMN_QUALIFIER, defaultColumnQualifier); + connectorProps.put(BigtableSinkConfig.VALUE_NULL_MODE_CONFIG, NullValueMode.WRITE.name()); + connectorProps.put(BigtableSinkConfig.DEFAULT_COLUMN_FAMILY_CONFIG, defaultColumnFamily); + connectorProps.put(BigtableSinkConfig.DEFAULT_COLUMN_QUALIFIER_CONFIG, defaultColumnQualifier); String testId = startSingleTopicConnector(connectorProps); connect .assertions() @@ -193,8 +193,8 @@ public void testWriteMode() throws InterruptedException { @Test public void testDeleteMode() throws InterruptedException { Map connectorProps = connectorProps(); - connectorProps.put(BigtableSinkConfig.CONFIG_VALUE_NULL_MODE, NullValueMode.DELETE.name()); - connectorProps.put(BigtableSinkConfig.CONFIG_INSERT_MODE, InsertMode.UPSERT.name()); + connectorProps.put(BigtableSinkConfig.VALUE_NULL_MODE_CONFIG, NullValueMode.DELETE.name()); + connectorProps.put(BigtableSinkConfig.INSERT_MODE_CONFIG, InsertMode.UPSERT.name()); String testId = startSingleTopicConnector(connectorProps); connect .assertions() @@ -327,8 +327,8 @@ public void testDeleteMode() throws InterruptedException { protected Map connectorProps() { Map props = super.baseConnectorProps(); - props.put(BigtableSinkConfig.CONFIG_AUTO_CREATE_TABLES, "true"); - props.put(BigtableSinkConfig.CONFIG_AUTO_CREATE_COLUMN_FAMILIES, "true"); + props.put(BigtableSinkConfig.AUTO_CREATE_TABLES_CONFIG, "true"); + props.put(BigtableSinkConfig.AUTO_CREATE_COLUMN_FAMILIES_CONFIG, "true"); // We use JsonConverter since it doesn't care about schemas, so we may use differently-shaped // data within a single test. props.put(ConnectorConfig.VALUE_CONVERTER_CLASS_CONFIG, JsonConverter.class.getName()); diff --git a/google-cloud-bigtable-kafka-connect-sink/src/test/java/com/google/cloud/kafka/connect/bigtable/integration/ResourceAutoCreationIT.java b/google-cloud-bigtable-kafka-connect-sink/src/test/java/com/google/cloud/kafka/connect/bigtable/integration/ResourceAutoCreationIT.java index d13a15a439..85df334d2c 100644 --- a/google-cloud-bigtable-kafka-connect-sink/src/test/java/com/google/cloud/kafka/connect/bigtable/integration/ResourceAutoCreationIT.java +++ b/google-cloud-bigtable-kafka-connect-sink/src/test/java/com/google/cloud/kafka/connect/bigtable/integration/ResourceAutoCreationIT.java @@ -131,7 +131,7 @@ public void testTableAutoCreationEnabledColumnFamilyAutoCreationDisabled() String dlqTopic = createDlq(); Map props = baseConnectorProps(); configureDlq(props, dlqTopic); - props.put(BigtableSinkTaskConfig.CONFIG_AUTO_CREATE_TABLES, String.valueOf(true)); + props.put(BigtableSinkTaskConfig.AUTO_CREATE_TABLES_CONFIG, String.valueOf(true)); props.put(ConnectorConfig.VALUE_CONVERTER_CLASS_CONFIG, JsonConverter.class.getName()); props.put( ConnectorConfig.VALUE_CONVERTER_CLASS_CONFIG @@ -157,7 +157,7 @@ public void testTableAutoCreationDisabledColumnFamilyAutoCreationEnabled() String dlqTopic = createDlq(); Map props = baseConnectorProps(); configureDlq(props, dlqTopic); - props.put(BigtableSinkTaskConfig.CONFIG_AUTO_CREATE_COLUMN_FAMILIES, String.valueOf(true)); + props.put(BigtableSinkTaskConfig.AUTO_CREATE_COLUMN_FAMILIES_CONFIG, String.valueOf(true)); props.put(ConnectorConfig.VALUE_CONVERTER_CLASS_CONFIG, JsonConverter.class.getName()); props.put( ConnectorConfig.VALUE_CONVERTER_CLASS_CONFIG @@ -186,8 +186,8 @@ public void testAutoTableAndColumnFamilyAutoCreationWhenReadingMultipleTopics() String dlqTopic = createDlq(); Map props = baseConnectorProps(); configureDlq(props, dlqTopic); - props.put(BigtableSinkConfig.CONFIG_AUTO_CREATE_TABLES, String.valueOf(true)); - props.put(BigtableSinkConfig.CONFIG_AUTO_CREATE_COLUMN_FAMILIES, String.valueOf(true)); + props.put(BigtableSinkConfig.AUTO_CREATE_TABLES_CONFIG, String.valueOf(true)); + props.put(BigtableSinkConfig.AUTO_CREATE_COLUMN_FAMILIES_CONFIG, String.valueOf(true)); Set topicSuffixes = Set.of("topic1", "topic2", "topic3", "topic4"); String testId = startMultipleTopicConnector(props, topicSuffixes); @@ -213,9 +213,9 @@ public void testCreationOfInvalidTable() throws InterruptedException { String dlqTopic = createDlq(); Map props = baseConnectorProps(); String invalidTableName = "T".repeat(10000); - props.put(BigtableSinkConfig.CONFIG_TABLE_NAME_FORMAT, invalidTableName); - props.put(BigtableSinkConfig.CONFIG_AUTO_CREATE_TABLES, String.valueOf(true)); - props.put(BigtableSinkConfig.CONFIG_RETRY_TIMEOUT_MILLIS, "10000"); + props.put(BigtableSinkConfig.TABLE_NAME_FORMAT_CONFIG, invalidTableName); + props.put(BigtableSinkConfig.AUTO_CREATE_TABLES_CONFIG, String.valueOf(true)); + props.put(BigtableSinkConfig.RETRY_TIMEOUT_MILLIS_CONFIG, "10000"); configureDlq(props, dlqTopic); String testId = startSingleTopicConnector(props); @@ -235,8 +235,8 @@ public void testCreationOfTooManyColumnFamilies() throws InterruptedException { String dlqTopic = createDlq(); Map props = baseConnectorProps(); - props.put(BigtableSinkConfig.CONFIG_AUTO_CREATE_TABLES, String.valueOf(true)); - props.put(BigtableSinkConfig.CONFIG_AUTO_CREATE_COLUMN_FAMILIES, String.valueOf(true)); + props.put(BigtableSinkConfig.AUTO_CREATE_TABLES_CONFIG, String.valueOf(true)); + props.put(BigtableSinkConfig.AUTO_CREATE_COLUMN_FAMILIES_CONFIG, String.valueOf(true)); props.put(ConnectorConfig.VALUE_CONVERTER_CLASS_CONFIG, JsonConverter.class.getName()); props.put( ConnectorConfig.VALUE_CONVERTER_CLASS_CONFIG @@ -274,10 +274,10 @@ public void testRowDeletionCreatesTableWhenAutoCreationEnabled() throws Interrup String dlqTopic = createDlq(); Map props = baseConnectorProps(); configureDlq(props, dlqTopic); - props.put(BigtableSinkConfig.CONFIG_AUTO_CREATE_TABLES, String.valueOf(true)); - props.put(BigtableSinkConfig.CONFIG_AUTO_CREATE_COLUMN_FAMILIES, String.valueOf(true)); - props.put(BigtableSinkConfig.CONFIG_VALUE_NULL_MODE, NullValueMode.DELETE.name()); - props.put(BigtableSinkConfig.CONFIG_INSERT_MODE, InsertMode.UPSERT.name()); + props.put(BigtableSinkConfig.AUTO_CREATE_TABLES_CONFIG, String.valueOf(true)); + props.put(BigtableSinkConfig.AUTO_CREATE_COLUMN_FAMILIES_CONFIG, String.valueOf(true)); + props.put(BigtableSinkConfig.VALUE_NULL_MODE_CONFIG, NullValueMode.DELETE.name()); + props.put(BigtableSinkConfig.INSERT_MODE_CONFIG, InsertMode.UPSERT.name()); props.put(ConnectorConfig.VALUE_CONVERTER_CLASS_CONFIG, JsonConverter.class.getName()); props.put( ConnectorConfig.VALUE_CONVERTER_CLASS_CONFIG @@ -374,8 +374,8 @@ public void testDeletionFailsWhenAutoCreationDisabled() throws InterruptedExcept String dlqTopic = createDlq(); Map props = baseConnectorProps(); configureDlq(props, dlqTopic); - props.put(BigtableSinkConfig.CONFIG_VALUE_NULL_MODE, NullValueMode.DELETE.name()); - props.put(BigtableSinkConfig.CONFIG_INSERT_MODE, InsertMode.UPSERT.name()); + props.put(BigtableSinkConfig.VALUE_NULL_MODE_CONFIG, NullValueMode.DELETE.name()); + props.put(BigtableSinkConfig.INSERT_MODE_CONFIG, InsertMode.UPSERT.name()); props.put(ConnectorConfig.VALUE_CONVERTER_CLASS_CONFIG, JsonConverter.class.getName()); props.put( ConnectorConfig.VALUE_CONVERTER_CLASS_CONFIG diff --git a/google-cloud-bigtable-kafka-connect-sink/src/test/java/com/google/cloud/kafka/connect/bigtable/util/BasicPropertiesFactory.java b/google-cloud-bigtable-kafka-connect-sink/src/test/java/com/google/cloud/kafka/connect/bigtable/util/BasicPropertiesFactory.java index 3cdc7a2a7d..ac16dde541 100644 --- a/google-cloud-bigtable-kafka-connect-sink/src/test/java/com/google/cloud/kafka/connect/bigtable/util/BasicPropertiesFactory.java +++ b/google-cloud-bigtable-kafka-connect-sink/src/test/java/com/google/cloud/kafka/connect/bigtable/util/BasicPropertiesFactory.java @@ -23,14 +23,14 @@ public class BasicPropertiesFactory { public static Map getSinkProps() { Map props = new HashMap<>(); - props.put(BigtableSinkConfig.CONFIG_GCP_PROJECT_ID, "project"); - props.put(BigtableSinkConfig.CONFIG_BIGTABLE_INSTANCE_ID, "instance"); + props.put(BigtableSinkConfig.GCP_PROJECT_ID_CONFIG, "project"); + props.put(BigtableSinkConfig.BIGTABLE_INSTANCE_ID_CONFIG, "instance"); return props; } public static Map getTaskProps() { Map props = getSinkProps(); - props.put(BigtableSinkTaskConfig.CONFIG_TASK_ID, "1"); + props.put(BigtableSinkTaskConfig.TASK_ID_CONFIG, "1"); return props; } } From 34b910196e5884cf0a5829fde2624b76f940e2fb Mon Sep 17 00:00:00 2001 From: Adam Czajkowski Date: Wed, 5 Feb 2025 17:10:55 +0100 Subject: [PATCH 34/76] Configure Admin API write retries --- .../bigtable/config/BigtableSinkConfig.java | 75 +++++++++++++------ .../config/BigtableSinkConfigTest.java | 4 +- 2 files changed, 56 insertions(+), 23 deletions(-) diff --git a/google-cloud-bigtable-kafka-connect-sink/src/main/java/com/google/cloud/kafka/connect/bigtable/config/BigtableSinkConfig.java b/google-cloud-bigtable-kafka-connect-sink/src/main/java/com/google/cloud/kafka/connect/bigtable/config/BigtableSinkConfig.java index d8275e3b01..b05acc6f1f 100644 --- a/google-cloud-bigtable-kafka-connect-sink/src/main/java/com/google/cloud/kafka/connect/bigtable/config/BigtableSinkConfig.java +++ b/google-cloud-bigtable-kafka-connect-sink/src/main/java/com/google/cloud/kafka/connect/bigtable/config/BigtableSinkConfig.java @@ -18,6 +18,7 @@ import com.google.api.gax.core.CredentialsProvider; import com.google.api.gax.core.FixedCredentialsProvider; import com.google.api.gax.retrying.RetrySettings; +import com.google.api.gax.rpc.StatusCode; import com.google.auth.oauth2.GoogleCredentials; import com.google.cloud.bigtable.admin.v2.BigtableTableAdminClient; import com.google.cloud.bigtable.admin.v2.BigtableTableAdminSettings; @@ -82,7 +83,11 @@ public class BigtableSinkConfig extends AbstractConfig { GCP_PROJECT_ID_CONFIG, BIGTABLE_INSTANCE_ID_CONFIG, BIGTABLE_APP_PROFILE_ID_CONFIG); - private static final int BIGTABLE_CREDENTIALS_CHECK_TIMEOUT_SECONDS = 2; + private static final Duration BIGTABLE_CREDENTIALS_CHECK_TIMEOUT = + Duration.of(2, ChronoUnit.SECONDS); + private static final Duration BIGTABLE_ADMIN_API_WRITE_RETRY_INITIAL_DELAY = + Duration.of(100, ChronoUnit.MILLIS); + private static final double BIGTABLE_CLIENT_RETRY_DELAY_MULTIPLIER = 1.5; protected BigtableSinkConfig(ConfigDef definition, Map properties) { super(definition, properties); @@ -409,12 +414,18 @@ public InsertMode getInsertMode() { * described in {@link BigtableSinkConfig#getDefinition()}. */ public BigtableTableAdminClient getBigtableAdminClient() { - RetrySettings retrySettings = getRetrySettings(); - return getBigtableAdminClient(retrySettings); + Duration totalTimeout = getTotalRetryTimeout(); + RetrySettings defaultRetrySettings = getRetrySettings(totalTimeout, Duration.ZERO); + // Retries of Admin API writes need to have a nontrivial initial delay to avoid hitting + // the rate limit, which is low (1000 requests per minute). + RetrySettings adminApiWriteRetrySettings = + getRetrySettings(totalTimeout, BIGTABLE_ADMIN_API_WRITE_RETRY_INITIAL_DELAY); + return getBigtableAdminClient(defaultRetrySettings, adminApiWriteRetrySettings); } @VisibleForTesting - BigtableTableAdminClient getBigtableAdminClient(RetrySettings retrySettings) { + BigtableTableAdminClient getBigtableAdminClient( + RetrySettings defaultRetrySettings, RetrySettings adminApiWriteRetrySettings) { Optional credentialsProvider = getUserConfiguredBigtableCredentialsProvider(); @@ -429,10 +440,25 @@ BigtableTableAdminClient getBigtableAdminClient(RetrySettings retrySettings) { } BigtableTableAdminStubSettings.Builder adminStubSettings = adminSettingsBuilder.stubSettings(); - adminStubSettings.createTableSettings().setRetrySettings(retrySettings); - adminStubSettings.modifyColumnFamiliesSettings().setRetrySettings(retrySettings); - adminStubSettings.listTablesSettings().setRetrySettings(retrySettings); - adminStubSettings.getTableSettings().setRetrySettings(retrySettings); + adminStubSettings.listTablesSettings().setRetrySettings(defaultRetrySettings); + adminStubSettings.getTableSettings().setRetrySettings(defaultRetrySettings); + adminStubSettings + .createTableSettings() + .setRetrySettings(adminApiWriteRetrySettings) + // Retry createTable() for status codes other admin operations retry by default as + // seen in BigtableTableAdminStubSettings. + .setRetryableCodes(StatusCode.Code.UNAVAILABLE, StatusCode.Code.DEADLINE_EXCEEDED); + adminStubSettings + .modifyColumnFamiliesSettings() + .setRetrySettings(adminApiWriteRetrySettings) + // Retry createTable() for status codes other admin operations retry by default as + // seen in BigtableTableAdminStubSettings and for FAILED_PRECONDITION which is + // returned when concurrent column family creation is detected. + .setRetryableCodes( + StatusCode.Code.UNAVAILABLE, + StatusCode.Code.DEADLINE_EXCEEDED, + StatusCode.Code.FAILED_PRECONDITION); + try { return BigtableTableAdminClient.create(adminSettingsBuilder.build()); } catch (IOException e) { @@ -445,7 +471,8 @@ BigtableTableAdminClient getBigtableAdminClient(RetrySettings retrySettings) { * in {@link BigtableSinkConfig#getDefinition()}. */ public BigtableDataClient getBigtableDataClient() { - RetrySettings retrySettings = getRetrySettings(); + Duration totalTimeout = getTotalRetryTimeout(); + RetrySettings retrySettings = getRetrySettings(totalTimeout, Duration.ZERO); Optional credentialsProvider = getUserConfiguredBigtableCredentialsProvider(); @@ -489,12 +516,8 @@ boolean isBigtableConfigurationValid() { BigtableTableAdminClient bigtable = null; try { RetrySettings retrySettings = - RetrySettings.newBuilder() - .setMaxAttempts(0) - .setTotalTimeout( - Duration.of(BIGTABLE_CREDENTIALS_CHECK_TIMEOUT_SECONDS, ChronoUnit.SECONDS)) - .build(); - bigtable = getBigtableAdminClient(retrySettings); + getRetrySettings(BIGTABLE_CREDENTIALS_CHECK_TIMEOUT, Duration.ZERO); + bigtable = getBigtableAdminClient(retrySettings, retrySettings); bigtable.listTables(); return true; } catch (Throwable t) { @@ -507,17 +530,27 @@ boolean isBigtableConfigurationValid() { } /** - * @return {@link RetrySettings} of Cloud Bigtable clients configured as described in {@link - * BigtableSinkConfig#getDefinition()}. + * @return {@link RetrySettings} of Cloud Bigtable clients configured with exponential backoff and + * specified timeout and retry delay. */ - protected RetrySettings getRetrySettings() { + protected RetrySettings getRetrySettings(Duration totalTimeout, Duration initialDelay) { return RetrySettings.newBuilder() - .setTotalTimeout( - Duration.of( - getLong(BigtableSinkTaskConfig.RETRY_TIMEOUT_MILLIS_CONFIG), ChronoUnit.MILLIS)) + .setTotalTimeout(totalTimeout) + .setInitialRetryDelay(initialDelay) + .setMaxRetryDelay(totalTimeout) + .setRetryDelayMultiplier(BIGTABLE_CLIENT_RETRY_DELAY_MULTIPLIER) .build(); } + /** + * @return Maximal time for Cloud Bigtable clients as described in {@link + * BigtableSinkConfig#getDefinition()}. + */ + private Duration getTotalRetryTimeout() { + return Duration.of( + getLong(BigtableSinkTaskConfig.RETRY_TIMEOUT_MILLIS_CONFIG), ChronoUnit.MILLIS); + } + /** * Extracts typed enum value from this object. * diff --git a/google-cloud-bigtable-kafka-connect-sink/src/test/java/com/google/cloud/kafka/connect/bigtable/config/BigtableSinkConfigTest.java b/google-cloud-bigtable-kafka-connect-sink/src/test/java/com/google/cloud/kafka/connect/bigtable/config/BigtableSinkConfigTest.java index b63ec6dacd..4a6926b755 100644 --- a/google-cloud-bigtable-kafka-connect-sink/src/test/java/com/google/cloud/kafka/connect/bigtable/config/BigtableSinkConfigTest.java +++ b/google-cloud-bigtable-kafka-connect-sink/src/test/java/com/google/cloud/kafka/connect/bigtable/config/BigtableSinkConfigTest.java @@ -143,7 +143,7 @@ public void testIsBigtableConfigurationValidBasicSuccess() { BigtableSinkConfig config = spy(new BigtableSinkConfig(props)); BigtableTableAdminClient bigtable = mock(BigtableTableAdminClient.class); doReturn(emptyList()).when(bigtable).listTables(); - doReturn(bigtable).when(config).getBigtableAdminClient(any()); + doReturn(bigtable).when(config).getBigtableAdminClient(any(), any()); assertTrue(config.isBigtableConfigurationValid()); verify(bigtable, times(1)).close(); } @@ -162,7 +162,7 @@ public void testIsBigtableConfigurationValidOperationError() { BigtableSinkConfig config = spy(new BigtableSinkConfig(props)); BigtableTableAdminClient bigtable = mock(BigtableTableAdminClient.class); doThrow(new RuntimeException()).when(bigtable).listTables(); - doReturn(bigtable).when(config).getBigtableAdminClient(any()); + doReturn(bigtable).when(config).getBigtableAdminClient(any(), any()); assertFalse(config.isBigtableConfigurationValid()); verify(bigtable, times(1)).close(); } From 7479ee083707021a8c5e1e35b895cb8917545402 Mon Sep 17 00:00:00 2001 From: Adam Czajkowski Date: Wed, 5 Feb 2025 17:35:13 +0100 Subject: [PATCH 35/76] Configure integration test logging --- .../pom.xml | 5 ++-- .../connect/bigtable/integration/BaseIT.java | 12 +++------- .../BaseKafkaConnectBigtableIT.java | 23 +++++++++++++++++++ .../integration/ResourceAutoCreationIT.java | 2 +- .../bigtable/util/SchemaRegistryTestUtil.java | 6 +++++ .../src/test/resources/log4j.properties | 9 ++++++++ 6 files changed, 45 insertions(+), 12 deletions(-) create mode 100644 google-cloud-bigtable-kafka-connect-sink/src/test/resources/log4j.properties diff --git a/google-cloud-bigtable-kafka-connect-sink/pom.xml b/google-cloud-bigtable-kafka-connect-sink/pom.xml index e2d4b27d0e..241a552df7 100644 --- a/google-cloud-bigtable-kafka-connect-sink/pom.xml +++ b/google-cloud-bigtable-kafka-connect-sink/pom.xml @@ -15,7 +15,7 @@ 3.6.1 2.12 2.6.1 - 1.7.36 + 2.0.16 2.14.2 5.14.2 @@ -81,8 +81,9 @@ org.slf4j - slf4j-simple + slf4j-reload4j ${slf4j.version} + test com.fasterxml.jackson.core diff --git a/google-cloud-bigtable-kafka-connect-sink/src/test/java/com/google/cloud/kafka/connect/bigtable/integration/BaseIT.java b/google-cloud-bigtable-kafka-connect-sink/src/test/java/com/google/cloud/kafka/connect/bigtable/integration/BaseIT.java index e31dafb911..83894f26cd 100644 --- a/google-cloud-bigtable-kafka-connect-sink/src/test/java/com/google/cloud/kafka/connect/bigtable/integration/BaseIT.java +++ b/google-cloud-bigtable-kafka-connect-sink/src/test/java/com/google/cloud/kafka/connect/bigtable/integration/BaseIT.java @@ -24,18 +24,14 @@ import com.google.cloud.bigtable.admin.v2.BigtableTableAdminClient; import com.google.cloud.bigtable.data.v2.BigtableDataClient; -import com.google.cloud.bigtable.data.v2.models.Query; -import com.google.cloud.bigtable.data.v2.models.Row; import com.google.cloud.kafka.connect.bigtable.BigtableSinkConnector; import com.google.cloud.kafka.connect.bigtable.config.BigtableSinkConfig; import com.google.cloud.kafka.connect.bigtable.util.TestId; -import com.google.protobuf.ByteString; import java.util.Collections; import java.util.HashMap; import java.util.Map; import java.util.Properties; import java.util.Set; -import java.util.stream.Collectors; import org.apache.commons.lang.StringUtils; import org.apache.kafka.clients.admin.Admin; import org.apache.kafka.clients.consumer.ConsumerConfig; @@ -101,6 +97,9 @@ protected void startConnect() { throw new RuntimeException(e); } kafkaAdminClient = connect.kafka().createAdminClient(); + logger.info( + "Started embedded Kafka Connect cluster using bootstrap servers: {}", + connect.kafka().bootstrapServers()); } protected void stopConnect() { @@ -150,11 +149,6 @@ protected BigtableTableAdminClient getBigtableAdminClient(Map co return new BigtableSinkConfig(configProps).getBigtableAdminClient(); } - protected Map readAllRows(BigtableDataClient bigtable, String table) { - Query query = Query.create(table); - return bigtable.readRows(query).stream().collect(Collectors.toMap(Row::getKey, r -> r)); - } - protected String getTestClassId() { return TestId.getTestClassId(this.getClass()); } diff --git a/google-cloud-bigtable-kafka-connect-sink/src/test/java/com/google/cloud/kafka/connect/bigtable/integration/BaseKafkaConnectBigtableIT.java b/google-cloud-bigtable-kafka-connect-sink/src/test/java/com/google/cloud/kafka/connect/bigtable/integration/BaseKafkaConnectBigtableIT.java index f379549973..c43c6639dc 100644 --- a/google-cloud-bigtable-kafka-connect-sink/src/test/java/com/google/cloud/kafka/connect/bigtable/integration/BaseKafkaConnectBigtableIT.java +++ b/google-cloud-bigtable-kafka-connect-sink/src/test/java/com/google/cloud/kafka/connect/bigtable/integration/BaseKafkaConnectBigtableIT.java @@ -19,9 +19,15 @@ import com.google.cloud.bigtable.admin.v2.BigtableTableAdminClient; import com.google.cloud.bigtable.data.v2.BigtableDataClient; +import com.google.cloud.bigtable.data.v2.models.Query; +import com.google.cloud.bigtable.data.v2.models.Row; +import com.google.protobuf.ByteString; import java.util.Map; +import java.util.stream.Collectors; import org.junit.After; import org.junit.Before; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; public abstract class BaseKafkaConnectBigtableIT extends BaseKafkaConnectIT { // Not copied from BigtableSinkConfig since it isn't present in its public API. @@ -29,6 +35,7 @@ public abstract class BaseKafkaConnectBigtableIT extends BaseKafkaConnectIT { protected BigtableDataClient bigtableData; protected BigtableTableAdminClient bigtableAdmin; + private final Logger logger = LoggerFactory.getLogger(BaseKafkaConnectBigtableIT.class); @Before public void setUpBigtable() { @@ -47,6 +54,22 @@ public void tearDownBigtable() { } } + public Map readAllRows(BigtableDataClient bigtable, String table) { + Boolean success = null; + try { + Query query = Query.create(table); + Map result = + bigtable.readRows(query).stream().collect(Collectors.toMap(Row::getKey, r -> r)); + success = true; + return result; + } catch (Throwable t) { + success = false; + throw t; + } finally { + logger.info("readAllRows({}): success={}", table, success); + } + } + public void waitUntilBigtableContainsNumberOfRows(String tableId, long numberOfRows) throws InterruptedException { waitForCondition( diff --git a/google-cloud-bigtable-kafka-connect-sink/src/test/java/com/google/cloud/kafka/connect/bigtable/integration/ResourceAutoCreationIT.java b/google-cloud-bigtable-kafka-connect-sink/src/test/java/com/google/cloud/kafka/connect/bigtable/integration/ResourceAutoCreationIT.java index 85df334d2c..7d85eb0ac9 100644 --- a/google-cloud-bigtable-kafka-connect-sink/src/test/java/com/google/cloud/kafka/connect/bigtable/integration/ResourceAutoCreationIT.java +++ b/google-cloud-bigtable-kafka-connect-sink/src/test/java/com/google/cloud/kafka/connect/bigtable/integration/ResourceAutoCreationIT.java @@ -228,7 +228,7 @@ public void testCreationOfInvalidTable() throws InterruptedException { assertConnectorAndAllTasksAreRunning(testId); } - @org.junit.Ignore // TODO: unignore. For now, the emulator does not cause an exception. + @org.junit.Ignore // TODO: unignore, disabled for now to avoid 429s. @Test public void testCreationOfTooManyColumnFamilies() throws InterruptedException { int numberOfColumnFamilies = 1000; diff --git a/google-cloud-bigtable-kafka-connect-sink/src/test/java/com/google/cloud/kafka/connect/bigtable/util/SchemaRegistryTestUtil.java b/google-cloud-bigtable-kafka-connect-sink/src/test/java/com/google/cloud/kafka/connect/bigtable/util/SchemaRegistryTestUtil.java index c954de94c6..85fd2de1ac 100644 --- a/google-cloud-bigtable-kafka-connect-sink/src/test/java/com/google/cloud/kafka/connect/bigtable/util/SchemaRegistryTestUtil.java +++ b/google-cloud-bigtable-kafka-connect-sink/src/test/java/com/google/cloud/kafka/connect/bigtable/util/SchemaRegistryTestUtil.java @@ -31,9 +31,12 @@ import java.net.ServerSocket; import java.util.Properties; import org.apache.kafka.test.TestUtils; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; public class SchemaRegistryTestUtil { private static long STARTUP_TIMEOUT_MILLIS = 10000L; + private final Logger logger = LoggerFactory.getLogger(SchemaRegistryTestUtil.class); protected String bootstrapServers; @@ -46,6 +49,7 @@ public SchemaRegistryTestUtil(String bootstrapServers) { } public void start() throws Exception { + logger.info("Starting embedded Schema Registry..."); int port = findAvailableOpenPort(); restApp = new RestApp( @@ -64,10 +68,12 @@ public void start() throws Exception { "Schema Registry start timed out."); schemaRegistryUrl = restApp.restServer.getURI().toString(); + logger.info("Started embedded Schema Registry using bootstrap servers: {}", bootstrapServers); } public void stop() throws Exception { if (restApp != null) { + logger.info("Stopping embedded Schema Registry..."); restApp.stop(); } } diff --git a/google-cloud-bigtable-kafka-connect-sink/src/test/resources/log4j.properties b/google-cloud-bigtable-kafka-connect-sink/src/test/resources/log4j.properties new file mode 100644 index 0000000000..0cc837d45b --- /dev/null +++ b/google-cloud-bigtable-kafka-connect-sink/src/test/resources/log4j.properties @@ -0,0 +1,9 @@ +log4j.rootLogger=INFO, stdout +log4j.appender.stdout=org.apache.log4j.ConsoleAppender +log4j.appender.stdout.layout=org.apache.log4j.PatternLayout +log4j.appender.stdout.layout.ConversionPattern=[%d] %p %m (%c:%L)%n + +log4j.logger.org.apache.zookeeper=WARN +log4j.logger.org.eclipse.jetty=WARN +log4j.logger.org.reflections=WARN +log4j.logger.com.google.cloud.kafka.connect.bigtable=TRACE From 35602df061269026babe23f7a1a5ca89597ce363 Mon Sep 17 00:00:00 2001 From: Adam Czajkowski Date: Thu, 6 Feb 2025 13:53:12 +0100 Subject: [PATCH 36/76] Integration test cleanup move code from BaseIT to BaseKafkaConnectIT get rid of protected and use public/private in base classes --- .../connect/bigtable/integration/BaseIT.java | 126 +----------------- .../BaseKafkaConnectBigtableIT.java | 12 +- ...eKafkaConnectBigtableSchemaRegistryIT.java | 2 +- .../integration/BaseKafkaConnectIT.java | 117 +++++++++++++++- .../integration/DifferentConvertersIT.java | 2 +- .../bigtable/integration/NullHandlingIT.java | 6 +- .../integration/ResourceAutoCreationIT.java | 2 +- 7 files changed, 135 insertions(+), 132 deletions(-) diff --git a/google-cloud-bigtable-kafka-connect-sink/src/test/java/com/google/cloud/kafka/connect/bigtable/integration/BaseIT.java b/google-cloud-bigtable-kafka-connect-sink/src/test/java/com/google/cloud/kafka/connect/bigtable/integration/BaseIT.java index 83894f26cd..581a128d08 100644 --- a/google-cloud-bigtable-kafka-connect-sink/src/test/java/com/google/cloud/kafka/connect/bigtable/integration/BaseIT.java +++ b/google-cloud-bigtable-kafka-connect-sink/src/test/java/com/google/cloud/kafka/connect/bigtable/integration/BaseIT.java @@ -27,96 +27,17 @@ import com.google.cloud.kafka.connect.bigtable.BigtableSinkConnector; import com.google.cloud.kafka.connect.bigtable.config.BigtableSinkConfig; import com.google.cloud.kafka.connect.bigtable.util.TestId; -import java.util.Collections; import java.util.HashMap; import java.util.Map; -import java.util.Properties; -import java.util.Set; -import org.apache.commons.lang.StringUtils; -import org.apache.kafka.clients.admin.Admin; -import org.apache.kafka.clients.consumer.ConsumerConfig; import org.apache.kafka.clients.producer.ProducerConfig; -import org.apache.kafka.common.utils.Utils; import org.apache.kafka.connect.runtime.ConnectorConfig; -import org.apache.kafka.connect.runtime.SinkConnectorConfig; -import org.apache.kafka.connect.runtime.WorkerConfig; -import org.apache.kafka.connect.runtime.isolation.PluginDiscoveryMode; import org.apache.kafka.connect.storage.StringConverter; -import org.apache.kafka.connect.util.clusters.EmbeddedConnectCluster; -import org.slf4j.Logger; -import org.slf4j.LoggerFactory; public abstract class BaseIT { - // https://cloud.google.com/bigtable/docs/reference/admin/rpc/google.bigtable.admin.v2#createtablerequest - public static int MAX_BIGTABLE_TABLE_NAME_LENGTH = 50; + public int numTasks = 1; + public int maxKafkaMessageSizeBytes = 300 * 1024 * 1024; - private final Logger logger = LoggerFactory.getLogger(BaseIT.class); - protected EmbeddedConnectCluster connect; - private Admin kafkaAdminClient; - protected int numWorkers = 1; - protected int numBrokers = 1; - protected int numTasks = 1; - protected int maxKafkaMessageSizeBytes = 300 * 1024 * 1024; - - protected void startConnect() { - logger.info("Starting embedded Kafka Connect cluster..."); - Map workerProps = new HashMap<>(); - workerProps.put(WorkerConfig.OFFSET_COMMIT_INTERVAL_MS_CONFIG, Long.toString(10000)); - workerProps.put(WorkerConfig.PLUGIN_DISCOVERY_CONFIG, PluginDiscoveryMode.HYBRID_WARN.name()); - - Properties brokerProps = new Properties(); - brokerProps.put("socket.request.max.bytes", maxKafkaMessageSizeBytes); - brokerProps.put("message.max.bytes", maxKafkaMessageSizeBytes); - brokerProps.put("auto.create.topics.enable", "false"); - brokerProps.put("delete.topic.enable", "true"); - - Map clientConfigs = new HashMap<>(); - clientConfigs.put( - ProducerConfig.MAX_REQUEST_SIZE_CONFIG, String.valueOf(maxKafkaMessageSizeBytes)); - clientConfigs.put( - ProducerConfig.BUFFER_MEMORY_CONFIG, String.valueOf(maxKafkaMessageSizeBytes)); - clientConfigs.put( - ConsumerConfig.MAX_PARTITION_FETCH_BYTES_CONFIG, String.valueOf(maxKafkaMessageSizeBytes)); - connect = - new EmbeddedConnectCluster.Builder() - .name("kcbt-connect-cluster-" + getTestClassId()) - .numWorkers(numWorkers) - .numBrokers(numBrokers) - .brokerProps(brokerProps) - .workerProps(workerProps) - .clientConfigs(clientConfigs) - .build(); - - // Start the clusters - connect.start(); - try { - connect - .assertions() - .assertAtLeastNumWorkersAreUp(1, "Initial group of workers did not start in time."); - } catch (InterruptedException e) { - throw new RuntimeException(e); - } - kafkaAdminClient = connect.kafka().createAdminClient(); - logger.info( - "Started embedded Kafka Connect cluster using bootstrap servers: {}", - connect.kafka().bootstrapServers()); - } - - protected void stopConnect() { - logger.info("Stopping embedded Kafka Connect cluster..."); - if (kafkaAdminClient != null) { - Utils.closeQuietly(kafkaAdminClient, "Admin client for embedded Kafka cluster"); - kafkaAdminClient = null; - } - - // Stop all Connect, Kafka and Zk threads. - if (connect != null) { - Utils.closeQuietly(connect::stop, "Embedded Connect, Kafka, and Zookeeper clusters"); - connect = null; - } - } - - protected Map baseConnectorProps() { + public Map baseConnectorProps() { Map result = new HashMap<>(); result.put(CONNECTOR_CLASS_CONFIG, BigtableSinkConnector.class.getCanonicalName()); @@ -141,52 +62,19 @@ protected Map baseConnectorProps() { return result; } - protected BigtableDataClient getBigtableDataClient(Map configProps) { + public BigtableDataClient getBigtableDataClient(Map configProps) { return new BigtableSinkConfig(configProps).getBigtableDataClient(); } - protected BigtableTableAdminClient getBigtableAdminClient(Map configProps) { + public BigtableTableAdminClient getBigtableAdminClient(Map configProps) { return new BigtableSinkConfig(configProps).getBigtableAdminClient(); } - protected String getTestClassId() { + public String getTestClassId() { return TestId.getTestClassId(this.getClass()); } - protected String getTestCaseId() { + public String getTestCaseId() { return TestId.getTestCaseId(this.getClass()); } - - protected String startSingleTopicConnector(Map configProps) - throws InterruptedException { - return startConnector(configProps, Collections.emptySet()); - } - - protected String startMultipleTopicConnector( - Map configProps, Set topicNameSuffixes) throws InterruptedException { - return startConnector(configProps, topicNameSuffixes); - } - - private String startConnector(Map configProps, Set topicNameSuffixes) - throws InterruptedException { - int longestSuffix = topicNameSuffixes.stream().mapToInt(String::length).max().orElse(0); - String id = - StringUtils.right( - getTestCaseId() + System.currentTimeMillis(), - MAX_BIGTABLE_TABLE_NAME_LENGTH - longestSuffix); - if (topicNameSuffixes.isEmpty()) { - configProps.put(SinkConnectorConfig.TOPICS_CONFIG, id); - connect.kafka().createTopic(id, numBrokers); - } else { - configProps.put(SinkConnectorConfig.TOPICS_REGEX_CONFIG, id + ".*"); - for (String suffix : topicNameSuffixes) { - connect.kafka().createTopic(id + suffix, numBrokers); - } - } - connect.configureConnector(id, configProps); - connect - .assertions() - .assertConnectorAndAtLeastNumTasksAreRunning(id, numTasks, "Connector start timeout"); - return id; - } } diff --git a/google-cloud-bigtable-kafka-connect-sink/src/test/java/com/google/cloud/kafka/connect/bigtable/integration/BaseKafkaConnectBigtableIT.java b/google-cloud-bigtable-kafka-connect-sink/src/test/java/com/google/cloud/kafka/connect/bigtable/integration/BaseKafkaConnectBigtableIT.java index c43c6639dc..33e1d61abf 100644 --- a/google-cloud-bigtable-kafka-connect-sink/src/test/java/com/google/cloud/kafka/connect/bigtable/integration/BaseKafkaConnectBigtableIT.java +++ b/google-cloud-bigtable-kafka-connect-sink/src/test/java/com/google/cloud/kafka/connect/bigtable/integration/BaseKafkaConnectBigtableIT.java @@ -33,10 +33,11 @@ public abstract class BaseKafkaConnectBigtableIT extends BaseKafkaConnectIT { // Not copied from BigtableSinkConfig since it isn't present in its public API. public static long DEFAULT_BIGTABLE_RETRY_TIMEOUT_MILLIS = 90000; - protected BigtableDataClient bigtableData; - protected BigtableTableAdminClient bigtableAdmin; private final Logger logger = LoggerFactory.getLogger(BaseKafkaConnectBigtableIT.class); + public BigtableDataClient bigtableData; + public BigtableTableAdminClient bigtableAdmin; + @Before public void setUpBigtable() { Map props = baseConnectorProps(); @@ -55,18 +56,17 @@ public void tearDownBigtable() { } public Map readAllRows(BigtableDataClient bigtable, String table) { - Boolean success = null; + Integer numRecords = null; try { Query query = Query.create(table); Map result = bigtable.readRows(query).stream().collect(Collectors.toMap(Row::getKey, r -> r)); - success = true; + numRecords = result.size(); return result; } catch (Throwable t) { - success = false; throw t; } finally { - logger.info("readAllRows({}): success={}", table, success); + logger.info("readAllRows({}): #records={}", table, numRecords); } } diff --git a/google-cloud-bigtable-kafka-connect-sink/src/test/java/com/google/cloud/kafka/connect/bigtable/integration/BaseKafkaConnectBigtableSchemaRegistryIT.java b/google-cloud-bigtable-kafka-connect-sink/src/test/java/com/google/cloud/kafka/connect/bigtable/integration/BaseKafkaConnectBigtableSchemaRegistryIT.java index b688c3e512..298c3480e2 100644 --- a/google-cloud-bigtable-kafka-connect-sink/src/test/java/com/google/cloud/kafka/connect/bigtable/integration/BaseKafkaConnectBigtableSchemaRegistryIT.java +++ b/google-cloud-bigtable-kafka-connect-sink/src/test/java/com/google/cloud/kafka/connect/bigtable/integration/BaseKafkaConnectBigtableSchemaRegistryIT.java @@ -20,7 +20,7 @@ import org.junit.Before; public abstract class BaseKafkaConnectBigtableSchemaRegistryIT extends BaseKafkaConnectBigtableIT { - protected SchemaRegistryTestUtil schemaRegistry; + public SchemaRegistryTestUtil schemaRegistry; @Before public void setUpSchemaRegistry() throws Exception { diff --git a/google-cloud-bigtable-kafka-connect-sink/src/test/java/com/google/cloud/kafka/connect/bigtable/integration/BaseKafkaConnectIT.java b/google-cloud-bigtable-kafka-connect-sink/src/test/java/com/google/cloud/kafka/connect/bigtable/integration/BaseKafkaConnectIT.java index fbd4580c81..d7493b09e1 100644 --- a/google-cloud-bigtable-kafka-connect-sink/src/test/java/com/google/cloud/kafka/connect/bigtable/integration/BaseKafkaConnectIT.java +++ b/google-cloud-bigtable-kafka-connect-sink/src/test/java/com/google/cloud/kafka/connect/bigtable/integration/BaseKafkaConnectIT.java @@ -25,28 +25,50 @@ import java.util.Arrays; import java.util.Collection; import java.util.Collections; +import java.util.HashMap; import java.util.List; import java.util.Map; import java.util.Optional; +import java.util.Properties; +import java.util.Set; import java.util.concurrent.ExecutionException; import java.util.concurrent.Future; import java.util.concurrent.TimeUnit; import java.util.concurrent.TimeoutException; +import org.apache.commons.lang.StringUtils; +import org.apache.kafka.clients.admin.Admin; +import org.apache.kafka.clients.consumer.ConsumerConfig; import org.apache.kafka.clients.consumer.ConsumerRecord; import org.apache.kafka.clients.consumer.ConsumerRecords; import org.apache.kafka.clients.producer.KafkaProducer; +import org.apache.kafka.clients.producer.ProducerConfig; import org.apache.kafka.clients.producer.ProducerRecord; import org.apache.kafka.clients.producer.RecordMetadata; +import org.apache.kafka.common.utils.Utils; import org.apache.kafka.connect.data.SchemaAndValue; import org.apache.kafka.connect.runtime.SinkConnectorConfig; +import org.apache.kafka.connect.runtime.WorkerConfig; import org.apache.kafka.connect.runtime.errors.DeadLetterQueueReporter; import org.apache.kafka.connect.runtime.errors.ToleranceType; +import org.apache.kafka.connect.runtime.isolation.PluginDiscoveryMode; import org.apache.kafka.connect.storage.Converter; +import org.apache.kafka.connect.util.clusters.EmbeddedConnectCluster; import org.junit.After; import org.junit.Before; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; public abstract class BaseKafkaConnectIT extends BaseIT { - private static final long PRODUCE_TIMEOUT_MILLIS = 15000L; + public static final long PRODUCE_TIMEOUT_MILLIS = 15000L; + // https://cloud.google.com/bigtable/docs/reference/admin/rpc/google.bigtable.admin.v2#createtablerequest + public static int MAX_BIGTABLE_TABLE_NAME_LENGTH = 50; + + private final Logger logger = LoggerFactory.getLogger(BaseKafkaConnectIT.class); + + public EmbeddedConnectCluster connect; + public Admin kafkaAdminClient; + public int numWorkers = 1; + public int numBrokers = 1; @Before public void setUpConnect() { @@ -58,6 +80,99 @@ public void tearDownConnect() { stopConnect(); } + public void startConnect() { + logger.info("Starting embedded Kafka Connect cluster..."); + Map workerProps = new HashMap<>(); + workerProps.put(WorkerConfig.OFFSET_COMMIT_INTERVAL_MS_CONFIG, Long.toString(10000)); + workerProps.put(WorkerConfig.PLUGIN_DISCOVERY_CONFIG, PluginDiscoveryMode.HYBRID_WARN.name()); + + Properties brokerProps = new Properties(); + brokerProps.put("socket.request.max.bytes", maxKafkaMessageSizeBytes); + brokerProps.put("message.max.bytes", maxKafkaMessageSizeBytes); + brokerProps.put("auto.create.topics.enable", "false"); + brokerProps.put("delete.topic.enable", "true"); + + Map clientConfigs = new HashMap<>(); + clientConfigs.put( + ProducerConfig.MAX_REQUEST_SIZE_CONFIG, String.valueOf(maxKafkaMessageSizeBytes)); + clientConfigs.put( + ProducerConfig.BUFFER_MEMORY_CONFIG, String.valueOf(maxKafkaMessageSizeBytes)); + clientConfigs.put( + ConsumerConfig.MAX_PARTITION_FETCH_BYTES_CONFIG, String.valueOf(maxKafkaMessageSizeBytes)); + connect = + new EmbeddedConnectCluster.Builder() + .name("kcbt-connect-cluster-" + getTestClassId()) + .numWorkers(numWorkers) + .numBrokers(numBrokers) + .brokerProps(brokerProps) + .workerProps(workerProps) + .clientConfigs(clientConfigs) + .build(); + + // Start the clusters + connect.start(); + try { + connect + .assertions() + .assertAtLeastNumWorkersAreUp(1, "Initial group of workers did not start in time."); + } catch (InterruptedException e) { + throw new RuntimeException(e); + } + kafkaAdminClient = connect.kafka().createAdminClient(); + logger.info( + "Started embedded Kafka Connect cluster using bootstrap servers: {}", + connect.kafka().bootstrapServers()); + } + + public void stopConnect() { + logger.info( + "Stopping embedded Kafka Connect cluster using bootstrap servers: {}", + connect.kafka().bootstrapServers()); + if (kafkaAdminClient != null) { + Utils.closeQuietly(kafkaAdminClient, "Admin client for embedded Kafka cluster"); + kafkaAdminClient = null; + } + + // Stop all Connect, Kafka and Zk threads. + if (connect != null) { + Utils.closeQuietly(connect::stop, "Embedded Connect, Kafka, and Zookeeper clusters"); + connect = null; + } + } + + public String startSingleTopicConnector(Map configProps) + throws InterruptedException { + return startConnector(configProps, Collections.emptySet()); + } + + public String startMultipleTopicConnector( + Map configProps, Set topicNameSuffixes) throws InterruptedException { + return startConnector(configProps, topicNameSuffixes); + } + + private String startConnector(Map configProps, Set topicNameSuffixes) + throws InterruptedException { + int longestSuffix = topicNameSuffixes.stream().mapToInt(String::length).max().orElse(0); + String id = + StringUtils.right( + getTestCaseId() + System.currentTimeMillis(), + MAX_BIGTABLE_TABLE_NAME_LENGTH - longestSuffix); + if (topicNameSuffixes.isEmpty()) { + configProps.put(SinkConnectorConfig.TOPICS_CONFIG, id); + connect.kafka().createTopic(id, numBrokers); + } else { + configProps.put(SinkConnectorConfig.TOPICS_REGEX_CONFIG, id + ".*"); + for (String suffix : topicNameSuffixes) { + connect.kafka().createTopic(id + suffix, numBrokers); + } + } + connect.configureConnector(id, configProps); + connect + .assertions() + .assertConnectorAndAtLeastNumTasksAreRunning(id, numTasks, "Connector start timeout"); + return id; + } + public KafkaProducer getKafkaProducer() { return connect.kafka().createProducer(Collections.emptyMap()); } diff --git a/google-cloud-bigtable-kafka-connect-sink/src/test/java/com/google/cloud/kafka/connect/bigtable/integration/DifferentConvertersIT.java b/google-cloud-bigtable-kafka-connect-sink/src/test/java/com/google/cloud/kafka/connect/bigtable/integration/DifferentConvertersIT.java index 44152d60f1..696e884a4d 100644 --- a/google-cloud-bigtable-kafka-connect-sink/src/test/java/com/google/cloud/kafka/connect/bigtable/integration/DifferentConvertersIT.java +++ b/google-cloud-bigtable-kafka-connect-sink/src/test/java/com/google/cloud/kafka/connect/bigtable/integration/DifferentConvertersIT.java @@ -167,7 +167,7 @@ public void testConverter() throws InterruptedException { waitUntilBigtableContainsNumberOfRows(topic, NUM_RECORDS); } - protected Map connectorProps() { + private Map connectorProps() { Map props = baseConnectorProps(); props.put(BigtableSinkConfig.AUTO_CREATE_TABLES_CONFIG, "true"); props.put(BigtableSinkConfig.AUTO_CREATE_COLUMN_FAMILIES_CONFIG, "true"); diff --git a/google-cloud-bigtable-kafka-connect-sink/src/test/java/com/google/cloud/kafka/connect/bigtable/integration/NullHandlingIT.java b/google-cloud-bigtable-kafka-connect-sink/src/test/java/com/google/cloud/kafka/connect/bigtable/integration/NullHandlingIT.java index 1bb0751064..b6ecd12b72 100644 --- a/google-cloud-bigtable-kafka-connect-sink/src/test/java/com/google/cloud/kafka/connect/bigtable/integration/NullHandlingIT.java +++ b/google-cloud-bigtable-kafka-connect-sink/src/test/java/com/google/cloud/kafka/connect/bigtable/integration/NullHandlingIT.java @@ -325,7 +325,7 @@ public void testDeleteMode() throws InterruptedException { new HashSet<>(rowDeletedColumn.getCells())); } - protected Map connectorProps() { + private Map connectorProps() { Map props = super.baseConnectorProps(); props.put(BigtableSinkConfig.AUTO_CREATE_TABLES_CONFIG, "true"); props.put(BigtableSinkConfig.AUTO_CREATE_COLUMN_FAMILIES_CONFIG, "true"); @@ -339,13 +339,13 @@ protected Map connectorProps() { return props; } - protected ByteString jsonifiedStructWithNullField() { + private ByteString jsonifiedStructWithNullField() { String expectedJson = String.format("{\"%s\":null}", NESTED_NULL_STRUCT_FIELD_NAME); byte[] expectedJsonBytes = expectedJson.getBytes(StandardCharsets.UTF_8); return ByteString.copyFrom(expectedJsonBytes); } - protected void assertCellContents( + private void assertCellContents( RowCell cell, String family, ByteString qualifier, ByteString value) { assertEquals(family, cell.getFamily()); assertEquals(qualifier, cell.getQualifier()); diff --git a/google-cloud-bigtable-kafka-connect-sink/src/test/java/com/google/cloud/kafka/connect/bigtable/integration/ResourceAutoCreationIT.java b/google-cloud-bigtable-kafka-connect-sink/src/test/java/com/google/cloud/kafka/connect/bigtable/integration/ResourceAutoCreationIT.java index 7d85eb0ac9..8dd40398bd 100644 --- a/google-cloud-bigtable-kafka-connect-sink/src/test/java/com/google/cloud/kafka/connect/bigtable/integration/ResourceAutoCreationIT.java +++ b/google-cloud-bigtable-kafka-connect-sink/src/test/java/com/google/cloud/kafka/connect/bigtable/integration/ResourceAutoCreationIT.java @@ -212,7 +212,7 @@ public void testAutoTableAndColumnFamilyAutoCreationWhenReadingMultipleTopics() public void testCreationOfInvalidTable() throws InterruptedException { String dlqTopic = createDlq(); Map props = baseConnectorProps(); - String invalidTableName = "T".repeat(10000); + String invalidTableName = "T".repeat(100); props.put(BigtableSinkConfig.TABLE_NAME_FORMAT_CONFIG, invalidTableName); props.put(BigtableSinkConfig.AUTO_CREATE_TABLES_CONFIG, String.valueOf(true)); props.put(BigtableSinkConfig.RETRY_TIMEOUT_MILLIS_CONFIG, "10000"); From d95971086a36bd4b02aa81ade7cb1214b57f0833 Mon Sep 17 00:00:00 2001 From: Adam Czajkowski Date: Thu, 6 Feb 2025 15:28:44 +0100 Subject: [PATCH 37/76] Tweak number of partitions for created topics --- .../connect/bigtable/integration/BaseKafkaConnectIT.java | 4 ++-- 1 file changed, 2 insertions(+), 2 deletions(-) diff --git a/google-cloud-bigtable-kafka-connect-sink/src/test/java/com/google/cloud/kafka/connect/bigtable/integration/BaseKafkaConnectIT.java b/google-cloud-bigtable-kafka-connect-sink/src/test/java/com/google/cloud/kafka/connect/bigtable/integration/BaseKafkaConnectIT.java index d7493b09e1..e9e9929498 100644 --- a/google-cloud-bigtable-kafka-connect-sink/src/test/java/com/google/cloud/kafka/connect/bigtable/integration/BaseKafkaConnectIT.java +++ b/google-cloud-bigtable-kafka-connect-sink/src/test/java/com/google/cloud/kafka/connect/bigtable/integration/BaseKafkaConnectIT.java @@ -159,11 +159,11 @@ private String startConnector(Map configProps, Set topic MAX_BIGTABLE_TABLE_NAME_LENGTH - longestSuffix); if (topicNameSuffixes.isEmpty()) { configProps.put(SinkConnectorConfig.TOPICS_CONFIG, id); - connect.kafka().createTopic(id, numBrokers); + connect.kafka().createTopic(id, numTasks); } else { configProps.put(SinkConnectorConfig.TOPICS_REGEX_CONFIG, id + ".*"); for (String suffix : topicNameSuffixes) { - connect.kafka().createTopic(id + suffix, numBrokers); + connect.kafka().createTopic(id + suffix, numTasks); } } connect.configureConnector(id, configProps); From e02bda021297a323ee9efea286af98fdddcf97a5 Mon Sep 17 00:00:00 2001 From: Adam Czajkowski Date: Thu, 6 Feb 2025 16:21:25 +0100 Subject: [PATCH 38/76] Integration test using multiple connector tasks --- .../integration/BaseDataGeneratorIT.java | 157 ++++++++++++++++++ .../integration/DifferentConvertersIT.java | 143 +--------------- .../integration/MultipleConnectorTasksIT.java | 73 ++++++++ 3 files changed, 233 insertions(+), 140 deletions(-) create mode 100644 google-cloud-bigtable-kafka-connect-sink/src/test/java/com/google/cloud/kafka/connect/bigtable/integration/BaseDataGeneratorIT.java create mode 100644 google-cloud-bigtable-kafka-connect-sink/src/test/java/com/google/cloud/kafka/connect/bigtable/integration/MultipleConnectorTasksIT.java diff --git a/google-cloud-bigtable-kafka-connect-sink/src/test/java/com/google/cloud/kafka/connect/bigtable/integration/BaseDataGeneratorIT.java b/google-cloud-bigtable-kafka-connect-sink/src/test/java/com/google/cloud/kafka/connect/bigtable/integration/BaseDataGeneratorIT.java new file mode 100644 index 0000000000..cef7ad855a --- /dev/null +++ b/google-cloud-bigtable-kafka-connect-sink/src/test/java/com/google/cloud/kafka/connect/bigtable/integration/BaseDataGeneratorIT.java @@ -0,0 +1,157 @@ +/* + * Copyright 2025 Google LLC + * + * Licensed under the Apache License, Version 2.0 (the "License"); + * you may not use this file except in compliance with the License. + * You may obtain a copy of the License at + * + * https://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ +package com.google.cloud.kafka.connect.bigtable.integration; + +/* + * This software contains code derived from the BigQuery Connector for Apache Kafka, + * Copyright Aiven Oy, which in turn contains code derived from the Confluent BigQuery + * Kafka Connector, Copyright Confluent, Inc, which in turn contains code derived from + * the WePay BigQuery Kafka Connector, Copyright WePay, Inc. + */ + +import java.math.BigDecimal; +import java.util.AbstractMap; +import java.util.ArrayList; +import java.util.List; +import java.util.Map; +import java.util.stream.Collectors; +import java.util.stream.LongStream; +import org.apache.kafka.connect.data.Date; +import org.apache.kafka.connect.data.Decimal; +import org.apache.kafka.connect.data.Schema; +import org.apache.kafka.connect.data.SchemaAndValue; +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; + +public class BaseDataGeneratorIT extends BaseKafkaConnectBigtableSchemaRegistryIT { + private static final Schema SUB_STRUCT_SCHEMA = + SchemaBuilder.struct() + .field("ssf1", Schema.INT64_SCHEMA) + .field("ssf2", Schema.BOOLEAN_SCHEMA) + .build(); + private static final Schema NESTED_STRUCT_SCHEMA = + SchemaBuilder.struct() + .field("sf1", Schema.STRING_SCHEMA) + .field("sf2", SUB_STRUCT_SCHEMA) + .field("sf3", Schema.FLOAT64_SCHEMA) + .build(); + private static final Schema PRIMITIVES_SCHEMA = + SchemaBuilder.struct() + .field("boolean_field", Schema.BOOLEAN_SCHEMA) + .field("float32_field", Schema.FLOAT32_SCHEMA) + .field("float64_field", Schema.FLOAT64_SCHEMA) + .field("int8_field", Schema.INT8_SCHEMA) + .field("int16_field", Schema.INT16_SCHEMA) + .field("int32_field", Schema.INT32_SCHEMA) + .field("int64_field", Schema.INT64_SCHEMA) + .field("string_field", Schema.STRING_SCHEMA); + private static final Schema LOGICALS_SCHEMA = + SchemaBuilder.struct() + // klf = "Kafka logical field" + .field("klf1", Timestamp.builder().optional().build()) + .field("klf2", Time.builder().optional().build()) + .field("klf3", Date.builder().optional().build()) + .field("klf4", Decimal.builder(5).optional().build()) + .build(); + private static final Schema ARRAY_SCHEMA = SchemaBuilder.array(Schema.STRING_SCHEMA); + private static final Schema KEY_SCHEMA = SchemaBuilder.INT64_SCHEMA; + private static final Schema VALUE_SCHEMA = + SchemaBuilder.struct() + .optional() + .field("f1", Schema.STRING_SCHEMA) + .field("f2", Schema.BOOLEAN_SCHEMA) + .field("f3", Schema.FLOAT64_SCHEMA) + .field("bytes_field", Schema.OPTIONAL_BYTES_SCHEMA) + .field("nested_field", NESTED_STRUCT_SCHEMA) + .field("primitives_field", PRIMITIVES_SCHEMA) + .field("logicals_field", LOGICALS_SCHEMA) + .field("array_field", ARRAY_SCHEMA) + .build(); + public long numRecords = 100L; + + public void populateKafkaTopic( + String topic, long numRecords, Converter keyConverter, Converter valueConverter) { + List> records = new ArrayList<>(); + for (long i = 0; i < numRecords; i++) { + Object key = i; + Object value = getValue(i); + records.add( + new AbstractMap.SimpleImmutableEntry<>( + new SchemaAndValue(KEY_SCHEMA, key), new SchemaAndValue(VALUE_SCHEMA, value))); + } + sendRecords(topic, records, keyConverter, valueConverter); + } + + public Struct getValue(long iteration) { + Struct primitivesStruct = new Struct(VALUE_SCHEMA.field("primitives_field").schema()); + primitivesStruct.put("boolean_field", iteration % 3 == 1); + primitivesStruct.put("float32_field", iteration * 1.5f); + primitivesStruct.put("float64_field", iteration * 0.5); + primitivesStruct.put("int8_field", (byte) (iteration % 10)); + primitivesStruct.put("int16_field", (short) (iteration % 30 + 1)); + primitivesStruct.put("int32_field", (int) (-1 * (iteration % 100))); + primitivesStruct.put("int64_field", iteration * 10); + primitivesStruct.put("string_field", Long.toString(iteration * 123)); + + Struct logicalsStruct = new Struct(VALUE_SCHEMA.field("logicals_field").schema()); + long timestampMs = 1707835187396L; + int msPerDay = 86400000; + int time = (int) (timestampMs % msPerDay); + int date = (int) (timestampMs / msPerDay); + Schema klf1Schema = logicalsStruct.schema().field("klf1").schema(); + java.util.Date klf1Value = Timestamp.toLogical(klf1Schema, timestampMs); + Schema klf2Schema = logicalsStruct.schema().field("klf2").schema(); + java.util.Date klf2Value = Time.toLogical(klf2Schema, time); + Schema klf3Schema = logicalsStruct.schema().field("klf3").schema(); + java.util.Date klf3Value = Date.toLogical(klf3Schema, date); + logicalsStruct + .put("klf1", klf1Value) + .put("klf2", klf2Value) + .put("klf3", klf3Value) + .put("klf4", BigDecimal.valueOf(6543).setScale(5)); + + Struct subStruct = + new Struct(VALUE_SCHEMA.field("nested_field").schema().field("sf2").schema()); + subStruct.put("ssf1", iteration / 2); + subStruct.put("ssf2", false); + + Struct nestedStruct = new Struct(VALUE_SCHEMA.field("nested_field").schema()); + nestedStruct.put("sf1", "sv1"); + nestedStruct.put("sf2", subStruct); + nestedStruct.put("sf3", iteration * 1.0); + + List arrayValue = + LongStream.of(iteration % 10) + .mapToObj(l -> "array element " + l) + .collect(Collectors.toList()); + + byte[] bytesValue = new byte[(int) iteration % 4]; + for (int i = 0; i < bytesValue.length; i++) bytesValue[i] = (byte) i; + + return new Struct(VALUE_SCHEMA) + .put("f1", "api" + iteration) + .put("f2", iteration % 2 == 0) + .put("f3", iteration * 0.01) + .put("bytes_field", bytesValue) + .put("nested_field", nestedStruct) + .put("primitives_field", primitivesStruct) + .put("logicals_field", logicalsStruct) + .put("array_field", arrayValue); + } +} diff --git a/google-cloud-bigtable-kafka-connect-sink/src/test/java/com/google/cloud/kafka/connect/bigtable/integration/DifferentConvertersIT.java b/google-cloud-bigtable-kafka-connect-sink/src/test/java/com/google/cloud/kafka/connect/bigtable/integration/DifferentConvertersIT.java index 696e884a4d..5d4549c6bb 100644 --- a/google-cloud-bigtable-kafka-connect-sink/src/test/java/com/google/cloud/kafka/connect/bigtable/integration/DifferentConvertersIT.java +++ b/google-cloud-bigtable-kafka-connect-sink/src/test/java/com/google/cloud/kafka/connect/bigtable/integration/DifferentConvertersIT.java @@ -15,37 +15,16 @@ */ package com.google.cloud.kafka.connect.bigtable.integration; -/* - * This software contains code derived from the BigQuery Connector for Apache Kafka, - * Copyright Aiven Oy, which in turn contains code derived from the Confluent BigQuery - * Kafka Connector, Copyright Confluent, Inc, which in turn contains code derived from - * the WePay BigQuery Kafka Connector, Copyright WePay, Inc. - */ - import com.google.cloud.kafka.connect.bigtable.config.BigtableSinkConfig; import io.confluent.connect.avro.AvroConverter; import io.confluent.connect.json.JsonSchemaConverter; import io.confluent.connect.protobuf.ProtobufConverter; import io.confluent.kafka.serializers.AbstractKafkaSchemaSerDeConfig; -import java.math.BigDecimal; -import java.util.AbstractMap; -import java.util.ArrayList; import java.util.Arrays; import java.util.Collection; import java.util.HashMap; -import java.util.List; import java.util.Map; import java.util.function.Supplier; -import java.util.stream.Collectors; -import java.util.stream.LongStream; -import org.apache.kafka.connect.data.Date; -import org.apache.kafka.connect.data.Decimal; -import org.apache.kafka.connect.data.Schema; -import org.apache.kafka.connect.data.SchemaAndValue; -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.json.JsonConverter; import org.apache.kafka.connect.json.JsonConverterConfig; import org.apache.kafka.connect.runtime.ConnectorConfig; @@ -55,52 +34,7 @@ import org.junit.runners.Parameterized; @RunWith(Parameterized.class) -public class DifferentConvertersIT extends BaseKafkaConnectBigtableSchemaRegistryIT { - private static final Schema SUB_STRUCT_SCHEMA = - SchemaBuilder.struct() - .field("ssf1", Schema.INT64_SCHEMA) - .field("ssf2", Schema.BOOLEAN_SCHEMA) - .build(); - private static final Schema NESTED_STRUCT_SCHEMA = - SchemaBuilder.struct() - .field("sf1", Schema.STRING_SCHEMA) - .field("sf2", SUB_STRUCT_SCHEMA) - .field("sf3", Schema.FLOAT64_SCHEMA) - .build(); - private static final Schema PRIMITIVES_SCHEMA = - SchemaBuilder.struct() - .field("boolean_field", Schema.BOOLEAN_SCHEMA) - .field("float32_field", Schema.FLOAT32_SCHEMA) - .field("float64_field", Schema.FLOAT64_SCHEMA) - .field("int8_field", Schema.INT8_SCHEMA) - .field("int16_field", Schema.INT16_SCHEMA) - .field("int32_field", Schema.INT32_SCHEMA) - .field("int64_field", Schema.INT64_SCHEMA) - .field("string_field", Schema.STRING_SCHEMA); - private static final Schema LOGICALS_SCHEMA = - SchemaBuilder.struct() - // klf = "Kafka logical field" - .field("klf1", Timestamp.builder().optional().build()) - .field("klf2", Time.builder().optional().build()) - .field("klf3", Date.builder().optional().build()) - .field("klf4", Decimal.builder(5).optional().build()) - .build(); - private static final Schema ARRAY_SCHEMA = SchemaBuilder.array(Schema.STRING_SCHEMA); - private static final Schema KEY_SCHEMA = SchemaBuilder.INT64_SCHEMA; - private static final Schema VALUE_SCHEMA = - SchemaBuilder.struct() - .optional() - .field("f1", Schema.STRING_SCHEMA) - .field("f2", Schema.BOOLEAN_SCHEMA) - .field("f3", Schema.FLOAT64_SCHEMA) - .field("bytes_field", Schema.OPTIONAL_BYTES_SCHEMA) - .field("nested_field", NESTED_STRUCT_SCHEMA) - .field("primitives_field", PRIMITIVES_SCHEMA) - .field("logicals_field", LOGICALS_SCHEMA) - .field("array_field", ARRAY_SCHEMA) - .build(); - private static final Long NUM_RECORDS = 100L; - +public class DifferentConvertersIT extends BaseDataGeneratorIT { private Supplier converterConstructor; private Map converterBaseConfig; private boolean converterUsesSchemaRegistry; @@ -162,9 +96,9 @@ public void testConverter() throws InterruptedException { connect .assertions() .assertConnectorAndAtLeastNumTasksAreRunning(topic, numTasks, "Connector start timeout"); - populateKafkaTopic(topic, NUM_RECORDS, keyConverter, valueConverter); + populateKafkaTopic(topic, numRecords, keyConverter, valueConverter); - waitUntilBigtableContainsNumberOfRows(topic, NUM_RECORDS); + waitUntilBigtableContainsNumberOfRows(topic, numRecords); } private Map connectorProps() { @@ -173,75 +107,4 @@ private Map connectorProps() { props.put(BigtableSinkConfig.AUTO_CREATE_COLUMN_FAMILIES_CONFIG, "true"); return props; } - - private void populateKafkaTopic( - String topic, long numRecords, Converter keyConverter, Converter valueConverter) { - List> records = new ArrayList<>(); - for (long i = 0; i < numRecords; i++) { - Object key = i; - Object value = getValue(i); - records.add( - new AbstractMap.SimpleImmutableEntry<>( - new SchemaAndValue(KEY_SCHEMA, key), new SchemaAndValue(VALUE_SCHEMA, value))); - } - sendRecords(topic, records, keyConverter, valueConverter); - } - - private Struct getValue(long iteration) { - Struct primitivesStruct = new Struct(VALUE_SCHEMA.field("primitives_field").schema()); - primitivesStruct.put("boolean_field", iteration % 3 == 1); - primitivesStruct.put("float32_field", iteration * 1.5f); - primitivesStruct.put("float64_field", iteration * 0.5); - primitivesStruct.put("int8_field", (byte) (iteration % 10)); - primitivesStruct.put("int16_field", (short) (iteration % 30 + 1)); - primitivesStruct.put("int32_field", (int) (-1 * (iteration % 100))); - primitivesStruct.put("int64_field", iteration * 10); - primitivesStruct.put("string_field", Long.toString(iteration * 123)); - - Struct logicalsStruct = new Struct(VALUE_SCHEMA.field("logicals_field").schema()); - long timestampMs = 1707835187396L; - int msPerDay = 86400000; - int time = (int) (timestampMs % msPerDay); - int date = (int) (timestampMs / msPerDay); - Schema klf1Schema = logicalsStruct.schema().field("klf1").schema(); - java.util.Date klf1Value = - org.apache.kafka.connect.data.Timestamp.toLogical(klf1Schema, timestampMs); - Schema klf2Schema = logicalsStruct.schema().field("klf2").schema(); - java.util.Date klf2Value = org.apache.kafka.connect.data.Time.toLogical(klf2Schema, time); - Schema klf3Schema = logicalsStruct.schema().field("klf3").schema(); - java.util.Date klf3Value = org.apache.kafka.connect.data.Date.toLogical(klf3Schema, date); - logicalsStruct - .put("klf1", klf1Value) - .put("klf2", klf2Value) - .put("klf3", klf3Value) - .put("klf4", BigDecimal.valueOf(6543).setScale(5)); - - Struct subStruct = - new Struct(VALUE_SCHEMA.field("nested_field").schema().field("sf2").schema()); - subStruct.put("ssf1", iteration / 2); - subStruct.put("ssf2", false); - - Struct nestedStruct = new Struct(VALUE_SCHEMA.field("nested_field").schema()); - nestedStruct.put("sf1", "sv1"); - nestedStruct.put("sf2", subStruct); - nestedStruct.put("sf3", iteration * 1.0); - - List arrayValue = - LongStream.of(iteration % 10) - .mapToObj(l -> "array element " + l) - .collect(Collectors.toList()); - - byte[] bytesValue = new byte[(int) iteration % 4]; - for (int i = 0; i < bytesValue.length; i++) bytesValue[i] = (byte) i; - - return new Struct(VALUE_SCHEMA) - .put("f1", "api" + iteration) - .put("f2", iteration % 2 == 0) - .put("f3", iteration * 0.01) - .put("bytes_field", bytesValue) - .put("nested_field", nestedStruct) - .put("primitives_field", primitivesStruct) - .put("logicals_field", logicalsStruct) - .put("array_field", arrayValue); - } } diff --git a/google-cloud-bigtable-kafka-connect-sink/src/test/java/com/google/cloud/kafka/connect/bigtable/integration/MultipleConnectorTasksIT.java b/google-cloud-bigtable-kafka-connect-sink/src/test/java/com/google/cloud/kafka/connect/bigtable/integration/MultipleConnectorTasksIT.java new file mode 100644 index 0000000000..936f028967 --- /dev/null +++ b/google-cloud-bigtable-kafka-connect-sink/src/test/java/com/google/cloud/kafka/connect/bigtable/integration/MultipleConnectorTasksIT.java @@ -0,0 +1,73 @@ +/* + * Copyright 2025 Google LLC + * + * Licensed under the Apache License, Version 2.0 (the "License"); + * you may not use this file except in compliance with the License. + * You may obtain a copy of the License at + * + * https://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ +package com.google.cloud.kafka.connect.bigtable.integration; + +import com.google.cloud.kafka.connect.bigtable.config.BigtableSinkConfig; +import com.google.cloud.kafka.connect.bigtable.config.InsertMode; +import io.confluent.connect.avro.AvroConverter; +import io.confluent.kafka.serializers.AbstractKafkaSchemaSerDeConfig; +import java.util.Map; +import org.apache.kafka.connect.runtime.ConnectorConfig; +import org.apache.kafka.connect.storage.Converter; +import org.junit.Test; +import org.junit.runner.RunWith; +import org.junit.runners.JUnit4; + +@RunWith(JUnit4.class) +public class MultipleConnectorTasksIT extends BaseDataGeneratorIT { + // Note that this test possibly could become flaky if Admin API write retry's initial delay is set + // to a too low value and retries of table/column family creation exhausted the quota that should + // be used for creation of different column families. + @Test + public void testMultipleTasks() throws InterruptedException { + numRecords = 1000L; + numTasks = 10; + + Map converterProps = + Map.of( + AbstractKafkaSchemaSerDeConfig.SCHEMA_REGISTRY_URL_CONFIG, + schemaRegistry.schemaRegistryUrl()); + Converter keyConverter = new AvroConverter(); + keyConverter.configure(converterProps, true); + Converter valueConverter = new AvroConverter(); + valueConverter.configure(converterProps, false); + + Map connectorProps = baseConnectorProps(); + connectorProps.put(BigtableSinkConfig.AUTO_CREATE_TABLES_CONFIG, "true"); + connectorProps.put(BigtableSinkConfig.AUTO_CREATE_COLUMN_FAMILIES_CONFIG, "true"); + connectorProps.put(BigtableSinkConfig.INSERT_MODE_CONFIG, InsertMode.UPSERT.name()); + + for (Map.Entry prop : converterProps.entrySet()) { + connectorProps.put( + ConnectorConfig.KEY_CONVERTER_CLASS_CONFIG + "." + prop.getKey(), prop.getValue()); + connectorProps.put( + ConnectorConfig.VALUE_CONVERTER_CLASS_CONFIG + "." + prop.getKey(), prop.getValue()); + } + connectorProps.put( + ConnectorConfig.KEY_CONVERTER_CLASS_CONFIG, keyConverter.getClass().getName()); + connectorProps.put( + ConnectorConfig.VALUE_CONVERTER_CLASS_CONFIG, valueConverter.getClass().getName()); + + String testId = startSingleTopicConnector(connectorProps); + connect + .assertions() + .assertConnectorAndAtLeastNumTasksAreRunning(testId, numTasks, "Connector start timeout"); + populateKafkaTopic(testId, numRecords, keyConverter, valueConverter); + + waitUntilBigtableContainsNumberOfRows(testId, numRecords); + assertConnectorAndAllTasksAreRunning(testId); + } +} From 5cc4143294c40d23ec6f48e431bcaf5d7ffae281 Mon Sep 17 00:00:00 2001 From: Adam Czajkowski Date: Fri, 7 Feb 2025 12:58:36 +0100 Subject: [PATCH 39/76] Remove a disabled test that nukes request quota --- .../integration/ResourceAutoCreationIT.java | 43 ------------------- 1 file changed, 43 deletions(-) diff --git a/google-cloud-bigtable-kafka-connect-sink/src/test/java/com/google/cloud/kafka/connect/bigtable/integration/ResourceAutoCreationIT.java b/google-cloud-bigtable-kafka-connect-sink/src/test/java/com/google/cloud/kafka/connect/bigtable/integration/ResourceAutoCreationIT.java index 8dd40398bd..118d458c5a 100644 --- a/google-cloud-bigtable-kafka-connect-sink/src/test/java/com/google/cloud/kafka/connect/bigtable/integration/ResourceAutoCreationIT.java +++ b/google-cloud-bigtable-kafka-connect-sink/src/test/java/com/google/cloud/kafka/connect/bigtable/integration/ResourceAutoCreationIT.java @@ -37,9 +37,7 @@ import java.util.Set; import java.util.concurrent.ExecutionException; import java.util.concurrent.TimeoutException; -import java.util.function.Function; import java.util.stream.Collectors; -import java.util.stream.IntStream; import java.util.stream.StreamSupport; import org.apache.kafka.clients.consumer.ConsumerRecords; import org.apache.kafka.connect.data.Schema; @@ -228,47 +226,6 @@ public void testCreationOfInvalidTable() throws InterruptedException { assertConnectorAndAllTasksAreRunning(testId); } - @org.junit.Ignore // TODO: unignore, disabled for now to avoid 429s. - @Test - public void testCreationOfTooManyColumnFamilies() throws InterruptedException { - int numberOfColumnFamilies = 1000; - - String dlqTopic = createDlq(); - Map props = baseConnectorProps(); - props.put(BigtableSinkConfig.AUTO_CREATE_TABLES_CONFIG, String.valueOf(true)); - props.put(BigtableSinkConfig.AUTO_CREATE_COLUMN_FAMILIES_CONFIG, String.valueOf(true)); - props.put(ConnectorConfig.VALUE_CONVERTER_CLASS_CONFIG, JsonConverter.class.getName()); - props.put( - ConnectorConfig.VALUE_CONVERTER_CLASS_CONFIG - + "." - + JsonConverterConfig.SCHEMAS_ENABLE_CONFIG, - String.valueOf(true)); - - configureDlq(props, dlqTopic); - String testId = startSingleTopicConnector(props); - - Struct innerStruct = - new Struct(SchemaBuilder.struct().field(COLUMN_QUALIFIER, Schema.INT32_SCHEMA)) - .put(COLUMN_QUALIFIER, 1); - - Function fieldNameGenerator = i -> "f" + i; - SchemaBuilder schemaBuilder = SchemaBuilder.struct(); - IntStream.range(0, numberOfColumnFamilies) - .forEach(i -> schemaBuilder.field(fieldNameGenerator.apply(i), innerStruct.schema())); - Struct struct = new Struct(schemaBuilder.build()); - IntStream.range(0, numberOfColumnFamilies) - .forEach(i -> struct.put(fieldNameGenerator.apply(i), innerStruct)); - - String value = jsonify(testId, struct.schema(), struct); - - assertThrows(Throwable.class, () -> bigtableAdmin.getTable(testId)); - connect.kafka().produce(testId, KEY1, value); - assertSingleDlqEntry(dlqTopic, KEY1, value, InvalidBigtableSchemaModificationException.class); - bigtableAdmin.getTable(testId); - - assertConnectorAndAllTasksAreRunning(testId); - } - @Test public void testRowDeletionCreatesTableWhenAutoCreationEnabled() throws InterruptedException { String dlqTopic = createDlq(); From 86e588a6514e9349342173674ec4fa56d818fe1f Mon Sep 17 00:00:00 2001 From: Adam Czajkowski Date: Tue, 11 Feb 2025 11:19:46 +0100 Subject: [PATCH 40/76] Use SchemaAndValue in KeyMapper#extractField(). --- .../connect/bigtable/mapping/KeyMapper.java | 30 ++++++++----------- 1 file changed, 13 insertions(+), 17 deletions(-) diff --git a/google-cloud-bigtable-kafka-connect-sink/src/main/java/com/google/cloud/kafka/connect/bigtable/mapping/KeyMapper.java b/google-cloud-bigtable-kafka-connect-sink/src/main/java/com/google/cloud/kafka/connect/bigtable/mapping/KeyMapper.java index 6070e27519..5e99927b19 100644 --- a/google-cloud-bigtable-kafka-connect-sink/src/main/java/com/google/cloud/kafka/connect/bigtable/mapping/KeyMapper.java +++ b/google-cloud-bigtable-kafka-connect-sink/src/main/java/com/google/cloud/kafka/connect/bigtable/mapping/KeyMapper.java @@ -80,12 +80,10 @@ public byte[] getKey(SchemaAndValue kafkaKeyAndSchema) { Object kafkaKey = kafkaKeyAndSchema.value(); Optional kafkaKeySchema = Optional.ofNullable(kafkaKeyAndSchema.schema()); ensureKeyElementIsNotNull(kafkaKey); + SchemaAndValue keySchemaAndValue = new SchemaAndValue(kafkaKeySchema.orElse(null), kafkaKey); Stream keyParts = this.getDefinition(kafkaKey).stream() - .map( - (d) -> - serializeTopLevelKeyElement( - extractField(kafkaKey, kafkaKeySchema, d.iterator()))); + .map((d) -> serializeTopLevelKeyElement(extractField(keySchemaAndValue, d.iterator()))); return concatenateByteArrays(new byte[0], keyParts, delimiter, new byte[0]); } @@ -96,8 +94,8 @@ public byte[] getKey(SchemaAndValue kafkaKeyAndSchema) { * @param kafkaKey {@link org.apache.kafka.connect.sink.SinkRecord SinkRecord's} key. * @return {@link List} containing {@link List Lists} of key fields that need to be retrieved and * concatenated to construct the Cloud Bigtable row key. - *

    See {@link KeyMapper#extractField(Object, Optional, Iterator)} for details on semantics - * of the inner list. + *

    See {@link KeyMapper#extractField(SchemaAndValue, Iterator)} for details on semantics of + * the inner list. */ private List> getDefinition(Object kafkaKey) { if (this.definition.isEmpty()) { @@ -134,27 +132,25 @@ private static Optional> getFieldsOfRootValue(Object kafkaKey) { /** * Extract possibly nested fields from the input value. * - * @param value {@link org.apache.kafka.connect.sink.SinkRecord SinkRecord's} key or some its - * child. - * @param schema A schema of {@code value}. + * @param keySchemaAndValue {@link org.apache.kafka.connect.sink.SinkRecord SinkRecord's} key or + * some its child with corresponding {@link Schema}. * @param fields Fields that need to be accessed before the target value is reached. * @return Extracted nested field. */ - private SchemaAndValue extractField( - Object value, Optional schema, Iterator fields) { + private SchemaAndValue extractField(SchemaAndValue keySchemaAndValue, Iterator fields) { + Object value = keySchemaAndValue.value(); + Optional schema = Optional.ofNullable(keySchemaAndValue.schema()); ensureKeyElementIsNotNull(value); LogicalTypeUtils.logIfLogicalTypeUnsupported(schema); if (!fields.hasNext()) { - return new SchemaAndValue(schema.orElse(null), value); + return keySchemaAndValue; } String field = fields.next(); if (value instanceof Struct) { - Struct struct = (Struct) value; // Note that getWithoutDefault() throws if such a field does not exist. - return extractField( - struct.getWithoutDefault(field), - SchemaUtils.maybeExtractFieldSchema(schema, field), - fields); + Object fieldValue = ((Struct) value).getWithoutDefault(field); + Schema fieldSchema = SchemaUtils.maybeExtractFieldSchema(schema, field).orElse(null); + return extractField(new SchemaAndValue(fieldSchema, fieldValue), fields); } else { throw new DataException( "Unexpected class `" From 3f851ffca0e2f14f25c89cbd82ea1d1edf212789 Mon Sep 17 00:00:00 2001 From: Adam Czajkowski Date: Tue, 11 Feb 2025 11:34:02 +0100 Subject: [PATCH 41/76] Implement hashCode() for SchemaUtils.SchemaAndField --- .../cloud/kafka/connect/bigtable/mapping/SchemaUtils.java | 5 +++++ 1 file changed, 5 insertions(+) diff --git a/google-cloud-bigtable-kafka-connect-sink/src/main/java/com/google/cloud/kafka/connect/bigtable/mapping/SchemaUtils.java b/google-cloud-bigtable-kafka-connect-sink/src/main/java/com/google/cloud/kafka/connect/bigtable/mapping/SchemaUtils.java index 83bff2a8af..b38ec39210 100644 --- a/google-cloud-bigtable-kafka-connect-sink/src/main/java/com/google/cloud/kafka/connect/bigtable/mapping/SchemaUtils.java +++ b/google-cloud-bigtable-kafka-connect-sink/src/main/java/com/google/cloud/kafka/connect/bigtable/mapping/SchemaUtils.java @@ -100,6 +100,11 @@ public SchemaAndField(Schema schema, String field) { this.field = field; } + @Override + public int hashCode() { + return Objects.hash(schema, field); + } + @Override public boolean equals(Object o) { if (o == null || getClass() != o.getClass()) return false; From a78b5e3761741f9bf6633fb6aafab8f24ea76dcd Mon Sep 17 00:00:00 2001 From: Adam Czajkowski Date: Tue, 11 Feb 2025 11:48:46 +0100 Subject: [PATCH 42/76] Assert against case insensivity of enums in BigtableSinkConfigTest#testEnumCaseInsensitivity --- .../kafka/connect/bigtable/config/BigtableSinkConfigTest.java | 3 +++ 1 file changed, 3 insertions(+) diff --git a/google-cloud-bigtable-kafka-connect-sink/src/test/java/com/google/cloud/kafka/connect/bigtable/config/BigtableSinkConfigTest.java b/google-cloud-bigtable-kafka-connect-sink/src/test/java/com/google/cloud/kafka/connect/bigtable/config/BigtableSinkConfigTest.java index 4a6926b755..b7e7590b9d 100644 --- a/google-cloud-bigtable-kafka-connect-sink/src/test/java/com/google/cloud/kafka/connect/bigtable/config/BigtableSinkConfigTest.java +++ b/google-cloud-bigtable-kafka-connect-sink/src/test/java/com/google/cloud/kafka/connect/bigtable/config/BigtableSinkConfigTest.java @@ -135,6 +135,9 @@ public void testEnumCaseInsensitivity() { props.put(ERROR_MODE_CONFIG, "IGNORE"); props.put(VALUE_NULL_MODE_CONFIG, "delete"); BigtableSinkConfig config = new BigtableSinkConfig(props); + assertEquals(InsertMode.UPSERT, config.getInsertMode()); + assertEquals(BigtableErrorMode.IGNORE, config.getBigtableErrorMode()); + assertEquals(NullValueMode.DELETE, config.getNullValueMode()); } @Test From a92ee68337e815322280647fd74557f65e16ce64 Mon Sep 17 00:00:00 2001 From: Adam Czajkowski Date: Tue, 11 Feb 2025 11:49:57 +0100 Subject: [PATCH 43/76] Fix typo in 'getStructhWithNullOnNthNestingLevel' --- .../bigtable/integration/NullHandlingIT.java | 16 ++++++++-------- .../bigtable/mapping/ValueMapperTest.java | 16 ++++++++-------- .../bigtable/util/NestedNullStructFactory.java | 2 +- 3 files changed, 17 insertions(+), 17 deletions(-) diff --git a/google-cloud-bigtable-kafka-connect-sink/src/test/java/com/google/cloud/kafka/connect/bigtable/integration/NullHandlingIT.java b/google-cloud-bigtable-kafka-connect-sink/src/test/java/com/google/cloud/kafka/connect/bigtable/integration/NullHandlingIT.java index b6ecd12b72..5b163be92b 100644 --- a/google-cloud-bigtable-kafka-connect-sink/src/test/java/com/google/cloud/kafka/connect/bigtable/integration/NullHandlingIT.java +++ b/google-cloud-bigtable-kafka-connect-sink/src/test/java/com/google/cloud/kafka/connect/bigtable/integration/NullHandlingIT.java @@ -17,7 +17,7 @@ import static com.google.cloud.kafka.connect.bigtable.util.NestedNullStructFactory.NESTED_NULL_STRUCT_FIELD_NAME; import static com.google.cloud.kafka.connect.bigtable.util.NestedNullStructFactory.NESTED_NULL_STRUCT_FIELD_NAME_BYTES; -import static com.google.cloud.kafka.connect.bigtable.util.NestedNullStructFactory.getStructhWithNullOnNthNestingLevel; +import static com.google.cloud.kafka.connect.bigtable.util.NestedNullStructFactory.getStructWithNullOnNthNestingLevel; import static org.junit.Assert.assertEquals; import static org.junit.Assert.assertTrue; @@ -90,17 +90,17 @@ public void testIgnoreMode() throws InterruptedException { new AbstractMap.SimpleImmutableEntry<>( new SchemaAndValue(Schema.STRING_SCHEMA, KEY1), new SchemaAndValue(SchemaBuilder.struct().optional().build(), null))); - Struct nested1 = getStructhWithNullOnNthNestingLevel(1); + Struct nested1 = getStructWithNullOnNthNestingLevel(1); records.add( new AbstractMap.SimpleImmutableEntry<>( new SchemaAndValue(Schema.STRING_SCHEMA, KEY2), new SchemaAndValue(nested1.schema(), nested1))); - Struct nested2 = getStructhWithNullOnNthNestingLevel(2); + Struct nested2 = getStructWithNullOnNthNestingLevel(2); records.add( new AbstractMap.SimpleImmutableEntry<>( new SchemaAndValue(Schema.STRING_SCHEMA, KEY3), new SchemaAndValue(nested2.schema(), nested2))); - Struct nested3 = getStructhWithNullOnNthNestingLevel(3); + Struct nested3 = getStructWithNullOnNthNestingLevel(3); records.add( new AbstractMap.SimpleImmutableEntry<>( new SchemaAndValue(Schema.STRING_SCHEMA, KEY4), @@ -145,17 +145,17 @@ public void testWriteMode() throws InterruptedException { new AbstractMap.SimpleImmutableEntry<>( new SchemaAndValue(Schema.STRING_SCHEMA, KEY1), new SchemaAndValue(SchemaBuilder.struct().optional().build(), null))); - Struct nested1 = getStructhWithNullOnNthNestingLevel(1); + Struct nested1 = getStructWithNullOnNthNestingLevel(1); records.add( new AbstractMap.SimpleImmutableEntry<>( new SchemaAndValue(Schema.STRING_SCHEMA, KEY2), new SchemaAndValue(nested1.schema(), nested1))); - Struct nested2 = getStructhWithNullOnNthNestingLevel(2); + Struct nested2 = getStructWithNullOnNthNestingLevel(2); records.add( new AbstractMap.SimpleImmutableEntry<>( new SchemaAndValue(Schema.STRING_SCHEMA, KEY3), new SchemaAndValue(nested2.schema(), nested2))); - Struct nested3 = getStructhWithNullOnNthNestingLevel(3); + Struct nested3 = getStructWithNullOnNthNestingLevel(3); records.add( new AbstractMap.SimpleImmutableEntry<>( new SchemaAndValue(Schema.STRING_SCHEMA, KEY4), @@ -225,7 +225,7 @@ public void testDeleteMode() throws InterruptedException { assertEquals(Set.of(KEY1_BYTES, KEY2_BYTES, KEY3_BYTES, KEY4_BYTES), rowsBefore.keySet()); ByteString keyAddedJsonification = KEY1_BYTES; - Struct nestedNullToBeJsonified = getStructhWithNullOnNthNestingLevel(3); + Struct nestedNullToBeJsonified = getStructWithNullOnNthNestingLevel(3); ByteString keyDeletedColumn = KEY2_BYTES; Map.Entry deletedColumn = diff --git a/google-cloud-bigtable-kafka-connect-sink/src/test/java/com/google/cloud/kafka/connect/bigtable/mapping/ValueMapperTest.java b/google-cloud-bigtable-kafka-connect-sink/src/test/java/com/google/cloud/kafka/connect/bigtable/mapping/ValueMapperTest.java index cd75fd93e4..6b353850fa 100644 --- a/google-cloud-bigtable-kafka-connect-sink/src/test/java/com/google/cloud/kafka/connect/bigtable/mapping/ValueMapperTest.java +++ b/google-cloud-bigtable-kafka-connect-sink/src/test/java/com/google/cloud/kafka/connect/bigtable/mapping/ValueMapperTest.java @@ -18,7 +18,7 @@ import static com.google.cloud.kafka.connect.bigtable.util.MockUtil.assertTotalNumberOfInvocations; import static com.google.cloud.kafka.connect.bigtable.util.NestedNullStructFactory.NESTED_NULL_STRUCT_FIELD_NAME; import static com.google.cloud.kafka.connect.bigtable.util.NestedNullStructFactory.NESTED_NULL_STRUCT_FIELD_NAME_BYTES; -import static com.google.cloud.kafka.connect.bigtable.util.NestedNullStructFactory.getStructhWithNullOnNthNestingLevel; +import static com.google.cloud.kafka.connect.bigtable.util.NestedNullStructFactory.getStructWithNullOnNthNestingLevel; import static org.junit.Assert.assertTrue; import static org.mockito.Mockito.spy; import static org.mockito.Mockito.times; @@ -705,7 +705,7 @@ public void testNullModeIgnoreNestedOnce() { ValueMapper mapper = new TestValueMapper(DEFAULT_COLUMN_FAMILY, DEFAULT_COLUMN, NullValueMode.IGNORE); MutationDataBuilder mutationDataBuilder = - getRecordMutationDataBuilder(mapper, getStructhWithNullOnNthNestingLevel(1)); + getRecordMutationDataBuilder(mapper, getStructWithNullOnNthNestingLevel(1)); assertTotalNumberOfInvocations(mutationDataBuilder, 0); assertTrue(mutationDataBuilder.maybeBuild(TARGET_TABLE_NAME, ROW_KEY).isEmpty()); } @@ -715,7 +715,7 @@ public void testNullModeIgnoreNestedTwice() { ValueMapper mapper = new TestValueMapper(DEFAULT_COLUMN_FAMILY, DEFAULT_COLUMN, NullValueMode.IGNORE); MutationDataBuilder mutationDataBuilder = - getRecordMutationDataBuilder(mapper, getStructhWithNullOnNthNestingLevel(2)); + getRecordMutationDataBuilder(mapper, getStructWithNullOnNthNestingLevel(2)); assertTotalNumberOfInvocations(mutationDataBuilder, 0); assertTrue(mutationDataBuilder.maybeBuild(TARGET_TABLE_NAME, ROW_KEY).isEmpty()); } @@ -736,7 +736,7 @@ public void testNullModeWriteNestedOnce() { ValueMapper mapper = new TestValueMapper(DEFAULT_COLUMN_FAMILY, DEFAULT_COLUMN, NullValueMode.WRITE); MutationDataBuilder mutationDataBuilder = - getRecordMutationDataBuilder(mapper, getStructhWithNullOnNthNestingLevel(1)); + getRecordMutationDataBuilder(mapper, getStructWithNullOnNthNestingLevel(1)); verify(mutationDataBuilder, times(1)) .setCell( DEFAULT_COLUMN_FAMILY, @@ -752,7 +752,7 @@ public void testNullModeWriteNestedTwice() { ValueMapper mapper = new TestValueMapper(DEFAULT_COLUMN_FAMILY, DEFAULT_COLUMN, NullValueMode.WRITE); MutationDataBuilder mutationDataBuilder = - getRecordMutationDataBuilder(mapper, getStructhWithNullOnNthNestingLevel(2)); + getRecordMutationDataBuilder(mapper, getStructWithNullOnNthNestingLevel(2)); verify(mutationDataBuilder, times(1)) .setCell( NESTED_NULL_STRUCT_FIELD_NAME, @@ -778,7 +778,7 @@ public void testNullModeDeleteNestedOnce() { ValueMapper mapper = new TestValueMapper(DEFAULT_COLUMN_FAMILY, DEFAULT_COLUMN, NullValueMode.DELETE); MutationDataBuilder mutationDataBuilder = - getRecordMutationDataBuilder(mapper, getStructhWithNullOnNthNestingLevel(1)); + getRecordMutationDataBuilder(mapper, getStructWithNullOnNthNestingLevel(1)); verify(mutationDataBuilder, times(1)).deleteFamily(NESTED_NULL_STRUCT_FIELD_NAME); assertTotalNumberOfInvocations(mutationDataBuilder, 1); assertTrue(mutationDataBuilder.maybeBuild(TARGET_TABLE_NAME, ROW_KEY).isPresent()); @@ -789,7 +789,7 @@ public void testNullModeDeleteNestedTwice() { ValueMapper mapper = new TestValueMapper(DEFAULT_COLUMN_FAMILY, DEFAULT_COLUMN, NullValueMode.DELETE); MutationDataBuilder mutationDataBuilder = - getRecordMutationDataBuilder(mapper, getStructhWithNullOnNthNestingLevel(2)); + getRecordMutationDataBuilder(mapper, getStructWithNullOnNthNestingLevel(2)); verify(mutationDataBuilder, times(1)) .deleteCells( NESTED_NULL_STRUCT_FIELD_NAME, NESTED_NULL_STRUCT_FIELD_NAME_BYTES, TIMESTAMP_RANGE); @@ -805,7 +805,7 @@ public void testNullModeNestedThrice() { ByteString expectedJsonificationBytes = ByteString.copyFrom(expectedJsonification.getBytes(StandardCharsets.UTF_8)); MutationDataBuilder mutationDataBuilder = - getRecordMutationDataBuilder(mapper, getStructhWithNullOnNthNestingLevel(3)); + getRecordMutationDataBuilder(mapper, getStructWithNullOnNthNestingLevel(3)); verify(mutationDataBuilder, times(1)) .setCell( NESTED_NULL_STRUCT_FIELD_NAME, diff --git a/google-cloud-bigtable-kafka-connect-sink/src/test/java/com/google/cloud/kafka/connect/bigtable/util/NestedNullStructFactory.java b/google-cloud-bigtable-kafka-connect-sink/src/test/java/com/google/cloud/kafka/connect/bigtable/util/NestedNullStructFactory.java index 1280ddb5a5..7c4fa79ce7 100644 --- a/google-cloud-bigtable-kafka-connect-sink/src/test/java/com/google/cloud/kafka/connect/bigtable/util/NestedNullStructFactory.java +++ b/google-cloud-bigtable-kafka-connect-sink/src/test/java/com/google/cloud/kafka/connect/bigtable/util/NestedNullStructFactory.java @@ -26,7 +26,7 @@ public class NestedNullStructFactory { public static final ByteString NESTED_NULL_STRUCT_FIELD_NAME_BYTES = ByteString.copyFrom(NESTED_NULL_STRUCT_FIELD_NAME.getBytes(StandardCharsets.UTF_8)); - public static Struct getStructhWithNullOnNthNestingLevel(int n) { + public static Struct getStructWithNullOnNthNestingLevel(int n) { assert n > 0; Schema schema = From d9ac4014cae1a7c510b0cd4f47f1078c9277da64 Mon Sep 17 00:00:00 2001 From: Adam Czajkowski Date: Tue, 11 Feb 2025 11:59:07 +0100 Subject: [PATCH 44/76] State explicitly why the operations in ErrorReportingIT fail --- .../connect/bigtable/integration/ErrorReportingIT.java | 8 ++++++++ 1 file changed, 8 insertions(+) diff --git a/google-cloud-bigtable-kafka-connect-sink/src/test/java/com/google/cloud/kafka/connect/bigtable/integration/ErrorReportingIT.java b/google-cloud-bigtable-kafka-connect-sink/src/test/java/com/google/cloud/kafka/connect/bigtable/integration/ErrorReportingIT.java index 3ca43dd473..86abedc051 100644 --- a/google-cloud-bigtable-kafka-connect-sink/src/test/java/com/google/cloud/kafka/connect/bigtable/integration/ErrorReportingIT.java +++ b/google-cloud-bigtable-kafka-connect-sink/src/test/java/com/google/cloud/kafka/connect/bigtable/integration/ErrorReportingIT.java @@ -24,10 +24,15 @@ @RunWith(JUnit4.class) public class ErrorReportingIT extends BaseKafkaConnectIT { + // The table auto creation is disabled by default, so all writes to this table + // are going to result in errors. + private static String NONEXISTENT_TABLE_NAME = "ThisTableDoesNotExist"; + @Test public void testErrorModeFail() throws InterruptedException { Map props = baseConnectorProps(); props.put(BigtableSinkConfig.ERROR_MODE_CONFIG, BigtableErrorMode.FAIL.name()); + props.put(BigtableSinkConfig.TABLE_NAME_FORMAT_CONFIG, NONEXISTENT_TABLE_NAME); String testId = startSingleTopicConnector(props); connect.kafka().produce(testId, "key", "value"); @@ -41,6 +46,7 @@ public void testErrorModeFail() throws InterruptedException { public void testErrorModeWarn() throws InterruptedException { Map props = baseConnectorProps(); props.put(BigtableSinkConfig.ERROR_MODE_CONFIG, BigtableErrorMode.WARN.name()); + props.put(BigtableSinkConfig.TABLE_NAME_FORMAT_CONFIG, NONEXISTENT_TABLE_NAME); String testId = startSingleTopicConnector(props); connect.kafka().produce(testId, "key", "value"); @@ -54,6 +60,7 @@ public void testErrorModeWarn() throws InterruptedException { public void testErrorModeIgnore() throws InterruptedException { Map props = baseConnectorProps(); props.put(BigtableSinkConfig.ERROR_MODE_CONFIG, BigtableErrorMode.IGNORE.name()); + props.put(BigtableSinkConfig.TABLE_NAME_FORMAT_CONFIG, NONEXISTENT_TABLE_NAME); String testId = startSingleTopicConnector(props); connect.kafka().produce(testId, "key", "value"); @@ -68,6 +75,7 @@ public void testErrorModeDLQOverridesErrorMode() throws InterruptedException { String dlqTopic = createDlq(); Map props = baseConnectorProps(); props.put(BigtableSinkConfig.ERROR_MODE_CONFIG, BigtableErrorMode.FAIL.name()); + props.put(BigtableSinkConfig.TABLE_NAME_FORMAT_CONFIG, NONEXISTENT_TABLE_NAME); configureDlq(props, dlqTopic); String key = "key"; From de359a89043f32ca5d446b7da2a406e37430e535 Mon Sep 17 00:00:00 2001 From: Adam Czajkowski Date: Tue, 11 Feb 2025 12:14:15 +0100 Subject: [PATCH 45/76] Improve InsertUpsertIT: check DLQ, assert against values --- .../bigtable/integration/InsertUpsertIT.java | 26 ++++++++++++++++--- 1 file changed, 23 insertions(+), 3 deletions(-) diff --git a/google-cloud-bigtable-kafka-connect-sink/src/test/java/com/google/cloud/kafka/connect/bigtable/integration/InsertUpsertIT.java b/google-cloud-bigtable-kafka-connect-sink/src/test/java/com/google/cloud/kafka/connect/bigtable/integration/InsertUpsertIT.java index 855afa88d1..70a932315b 100644 --- a/google-cloud-bigtable-kafka-connect-sink/src/test/java/com/google/cloud/kafka/connect/bigtable/integration/InsertUpsertIT.java +++ b/google-cloud-bigtable-kafka-connect-sink/src/test/java/com/google/cloud/kafka/connect/bigtable/integration/InsertUpsertIT.java @@ -18,12 +18,15 @@ import static org.junit.Assert.assertEquals; import com.google.cloud.bigtable.data.v2.models.Row; +import com.google.cloud.bigtable.data.v2.models.RowCell; import com.google.cloud.kafka.connect.bigtable.config.BigtableErrorMode; import com.google.cloud.kafka.connect.bigtable.config.BigtableSinkConfig; import com.google.cloud.kafka.connect.bigtable.config.InsertMode; import com.google.protobuf.ByteString; import java.nio.charset.StandardCharsets; import java.util.Map; +import java.util.Set; +import java.util.stream.Collectors; import org.junit.Test; import org.junit.runner.RunWith; import org.junit.runners.JUnit4; @@ -39,27 +42,40 @@ public class InsertUpsertIT extends BaseKafkaConnectBigtableIT { private static final String VALUE1 = "value1"; private static final String VALUE2 = "value2"; private static final String VALUE3 = "value3"; + private static final ByteString VALUE1_BYTES = + ByteString.copyFrom(VALUE1.getBytes(StandardCharsets.UTF_8)); + private static final ByteString VALUE2_BYTES = + ByteString.copyFrom(VALUE2.getBytes(StandardCharsets.UTF_8)); + private static final ByteString VALUE3_BYTES = + ByteString.copyFrom(VALUE3.getBytes(StandardCharsets.UTF_8)); @Test public void testInsert() throws InterruptedException { + String dlqTopic = createDlq(); Map props = baseConnectorProps(); props.put(BigtableSinkConfig.AUTO_CREATE_TABLES_CONFIG, "true"); props.put(BigtableSinkConfig.AUTO_CREATE_COLUMN_FAMILIES_CONFIG, "true"); props.put(BigtableSinkConfig.INSERT_MODE_CONFIG, InsertMode.INSERT.name()); props.put(BigtableSinkConfig.ERROR_MODE_CONFIG, BigtableErrorMode.IGNORE.name()); + configureDlq(props, dlqTopic); String testId = startSingleTopicConnector(props); connect.kafka().produce(testId, KEY1, VALUE1); + waitUntilBigtableContainsNumberOfRows(testId, 1); connect.kafka().produce(testId, KEY1, VALUE2); connect.kafka().produce(testId, KEY2, VALUE3); - waitUntilBigtableContainsNumberOfRows(testId, 2); + assertSingleDlqEntry(dlqTopic, KEY1, VALUE2, null); Map rows = readAllRows(bigtableData, testId); Row row1 = rows.get(KEY1_BYTES); Row row2 = rows.get(KEY2_BYTES); assertEquals(1, row1.getCells().size()); + assertEquals(VALUE1_BYTES, row1.getCells().get(0).getValue()); assertEquals(1, row2.getCells().size()); + assertEquals(VALUE3_BYTES, row2.getCells().get(0).getValue()); + + assertConnectorAndAllTasksAreRunning(testId); } @Test @@ -68,19 +84,23 @@ public void testUpsert() throws InterruptedException { props.put(BigtableSinkConfig.AUTO_CREATE_TABLES_CONFIG, "true"); props.put(BigtableSinkConfig.AUTO_CREATE_COLUMN_FAMILIES_CONFIG, "true"); props.put(BigtableSinkConfig.INSERT_MODE_CONFIG, InsertMode.UPSERT.name()); - props.put(BigtableSinkConfig.ERROR_MODE_CONFIG, BigtableErrorMode.IGNORE.name()); String testId = startSingleTopicConnector(props); connect.kafka().produce(testId, KEY1, VALUE1); + waitUntilBigtableContainsNumberOfRows(testId, 1); connect.kafka().produce(testId, KEY1, VALUE2); connect.kafka().produce(testId, KEY2, VALUE3); - waitUntilBigtableContainsNumberOfRows(testId, 2); Map rows = readAllRows(bigtableData, testId); Row row1 = rows.get(KEY1_BYTES); Row row2 = rows.get(KEY2_BYTES); assertEquals(2, row1.getCells().size()); + assertEquals( + Set.of(VALUE1_BYTES, VALUE2_BYTES), + row1.getCells().stream().map(RowCell::getValue).collect(Collectors.toSet())); assertEquals(1, row2.getCells().size()); + assertEquals(VALUE3_BYTES, row2.getCells().get(0).getValue()); + assertConnectorAndAllTasksAreRunning(testId); } } From 4987b714e8c53db3782363a23f4b890208e0fe7c Mon Sep 17 00:00:00 2001 From: Adam Czajkowski Date: Tue, 11 Feb 2025 12:17:02 +0100 Subject: [PATCH 46/76] Use MAX_BIGTABLE_TABLE_NAME_LENGTH in ResourceAutoCreationIT#testCreationOfInvalidTable() --- .../connect/bigtable/integration/ResourceAutoCreationIT.java | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/google-cloud-bigtable-kafka-connect-sink/src/test/java/com/google/cloud/kafka/connect/bigtable/integration/ResourceAutoCreationIT.java b/google-cloud-bigtable-kafka-connect-sink/src/test/java/com/google/cloud/kafka/connect/bigtable/integration/ResourceAutoCreationIT.java index 118d458c5a..d557b20179 100644 --- a/google-cloud-bigtable-kafka-connect-sink/src/test/java/com/google/cloud/kafka/connect/bigtable/integration/ResourceAutoCreationIT.java +++ b/google-cloud-bigtable-kafka-connect-sink/src/test/java/com/google/cloud/kafka/connect/bigtable/integration/ResourceAutoCreationIT.java @@ -210,7 +210,7 @@ public void testAutoTableAndColumnFamilyAutoCreationWhenReadingMultipleTopics() public void testCreationOfInvalidTable() throws InterruptedException { String dlqTopic = createDlq(); Map props = baseConnectorProps(); - String invalidTableName = "T".repeat(100); + String invalidTableName = "T".repeat(MAX_BIGTABLE_TABLE_NAME_LENGTH + 1); props.put(BigtableSinkConfig.TABLE_NAME_FORMAT_CONFIG, invalidTableName); props.put(BigtableSinkConfig.AUTO_CREATE_TABLES_CONFIG, String.valueOf(true)); props.put(BigtableSinkConfig.RETRY_TIMEOUT_MILLIS_CONFIG, "10000"); From 0e4ff3edff2ff5455142d874231ad38920341648 Mon Sep 17 00:00:00 2001 From: Adam Czajkowski Date: Tue, 11 Feb 2025 12:20:49 +0100 Subject: [PATCH 47/76] Explain why testDeletionFailsWhenAutoCreationDisabled is needed --- .../bigtable/integration/ResourceAutoCreationIT.java | 7 +++++++ 1 file changed, 7 insertions(+) diff --git a/google-cloud-bigtable-kafka-connect-sink/src/test/java/com/google/cloud/kafka/connect/bigtable/integration/ResourceAutoCreationIT.java b/google-cloud-bigtable-kafka-connect-sink/src/test/java/com/google/cloud/kafka/connect/bigtable/integration/ResourceAutoCreationIT.java index d557b20179..e82d364532 100644 --- a/google-cloud-bigtable-kafka-connect-sink/src/test/java/com/google/cloud/kafka/connect/bigtable/integration/ResourceAutoCreationIT.java +++ b/google-cloud-bigtable-kafka-connect-sink/src/test/java/com/google/cloud/kafka/connect/bigtable/integration/ResourceAutoCreationIT.java @@ -23,6 +23,7 @@ import com.google.cloud.bigtable.admin.v2.models.ColumnFamily; import com.google.cloud.bigtable.admin.v2.models.CreateTableRequest; import com.google.cloud.bigtable.admin.v2.models.ModifyColumnFamiliesRequest; +import com.google.cloud.bigtable.data.v2.models.Range; import com.google.cloud.kafka.connect.bigtable.config.BigtableSinkConfig; import com.google.cloud.kafka.connect.bigtable.config.BigtableSinkTaskConfig; import com.google.cloud.kafka.connect.bigtable.config.InsertMode; @@ -326,6 +327,12 @@ public void testColumnDeletionCreatesTableAndColumnFamilyWhenAutoCreationEnabled assertConnectorAndAllTasksAreRunning(testId); } + /** + * This test checks consequences of design choices described in comments in {@link + * com.google.cloud.kafka.connect.bigtable.mapping.MutationDataBuilder#deleteCells(String, + * ByteString, Range.TimestampRange)} and {@link + * com.google.cloud.kafka.connect.bigtable.mapping.MutationDataBuilder#deleteFamily(String)}. + */ @Test public void testDeletionFailsWhenAutoCreationDisabled() throws InterruptedException { String dlqTopic = createDlq(); From 74c284a608b6f7fc919e0af7b6e02f516826426a Mon Sep 17 00:00:00 2001 From: Adam Czajkowski Date: Tue, 11 Feb 2025 14:07:18 +0100 Subject: [PATCH 48/76] Readme for tests --- .../src/test/README.md | 114 ++++++++++++++++++ 1 file changed, 114 insertions(+) create mode 100644 google-cloud-bigtable-kafka-connect-sink/src/test/README.md diff --git a/google-cloud-bigtable-kafka-connect-sink/src/test/README.md b/google-cloud-bigtable-kafka-connect-sink/src/test/README.md new file mode 100644 index 0000000000..903a70d214 --- /dev/null +++ b/google-cloud-bigtable-kafka-connect-sink/src/test/README.md @@ -0,0 +1,114 @@ +# Running the tests +All the commands are to be run in the directory containing the sink's [pom.xml](../../pom.xml). + +## Unit tests +The unit test runner is [Maven Surefire Plugin](https://maven.apache.org/surefire/maven-surefire-plugin/). + +To run the test execute the following command: +```bash +mvn clean test +``` + +## Integration tests +The unit test runner is [Maven Failsafe Plugin](https://maven.apache.org/surefire/maven-failsafe-plugin/). + +### Prerequirements +The integration tests need a reachable Bigtable instance. +It might be either a real Cloud Bigtable or the emulator. +Note that some of the tests are broken on the emulator (because it handles some requests differently than Cloud Bigtable). + +To configure the Bigtable instance for tests, create either a real Cloud Bigtable instance, or an emulator instance: + +#### Cloud Bigtable +##### Create a Cloud Bigtable instance +It can be either created using either the [WebUI form](https://console.cloud.google.com/bigtable/create-instance) or terraform `google_bigtable_instance` resource, for example: +```terraform +resource "google_bigtable_instance" "bigtable" { + name = "kafka-connect-bigtable-sink-test" + deletion_protection = false + + cluster { + cluster_id = "kafka-connect-bigtable-sink-test-cluster" + num_nodes = 1 + storage_type = "HDD" + zone = "europe-central2-a" + } +} +``` + +##### [optional] Create Service Account with required permissions +This section is optional, you can skip it if you want to use Application Default Credentials. + +Create a service account and grant it Bigtable Administrator (`roles/bigtable.admin`) permissions (such wide permissions are needed for table and column family auto creation). + +##### Configure the integration tests to use the created instance +Ensure that the sink's [pom.xml](../../pom.xml) does **not** contain the following section in Failsafe's `` section: +```xml + + target/test-classes/fake_service_key.json + localhost:8086 + +``` + + +Replace the following TODO values with you GCP project ID and Cloud Bigtable instance ID in `BaseIT#baseConnectorProps()` function: +```java +result.put(GCP_PROJECT_ID_CONFIG, "todotodo"); +result.put(BIGTABLE_INSTANCE_ID_CONFIG, "todotodo"); +``` + +##### [optional] Configure the permissions for integration tests + +If you want to use Application Default Credentials, configure the machine (on a workstation, log in with `gcloud` into an account with Bigtable Administrator permissions to the instance created in one of the previous steps). + +Otherwise, you need to use service account's permissions: +- download a service account key. +- put that key or path to it to the properties in `BaseIT#baseConnectorProps()` with a key defined in `BigtableSinkConfig`. + +#### Emulator +Start the emulator using `gcloud` directly: +```bash +gcloud beta emulators bigtable start --host-port=127.0.0.1:8086 & +``` +or using Docker with compose plugin: +```yaml +services: + bigtable: + image: google/cloud-sdk:latest + ports: + - 127.0.0.1:8086:8086 + entrypoint: + - gcloud + - beta + - emulators + - bigtable + - start + - --host-port=0.0.0.0:8086 +``` +```bash +docker compose up -d +``` + +Ensure that the sink's [pom.xml](../../pom.xml) contains the following section in Failsafe's `` section: +```xml + + target/test-classes/fake_service_key.json + localhost:8086 + +``` + +### Assumptions +The integration tests assume that the Bigtable instance they use is empty at the start of the run. +The assumption is used to skip cleaning up the tables created by the tests. + +If the limit on number of tables in a single Cloud Bigtable instance starts causing problems for you, clean them up by running: +```bash +PROJECT= +INSTANCE= +cbt -project "$PROJECT" -instance "$INSTANCE" ls | xargs -P 0 -I {} cbt -project "$PROJECT" -instance "$INSTANCE" {} +``` + +### Command to run the integration tests +```bash +mvn clean integration-tests -DskipUnitTests +``` From 99e5567d9b4f0ff6a52308d64ae0d75d35ad0df0 Mon Sep 17 00:00:00 2001 From: Adam Czajkowski Date: Mon, 24 Feb 2025 19:33:43 +0100 Subject: [PATCH 49/76] Use Confluent's sink in compatibility tests directly through a .jar --- .../pom.xml | 31 +++ .../BaseKafkaConnectBigtableIT.java | 12 ++ .../integration/BaseKafkaConnectIT.java | 13 +- .../integration/ConfluentCompatibilityIT.java | 203 +++++++++++------- .../key_bytes/data.json | 1 + .../key_bytes/key-schema.json | 10 + .../key_bytes/value-schema.json | 1 + .../confluent_compat_topic/KAFKA_VALUE | 1 - .../confluent_compat_topic/KAFKA_VALUE | 1 - .../confluent_compat_topic/KAFKA_VALUE | 1 - .../confluent_compat_topic/KAFKA_VALUE | 1 - .../confluent_compat_topic/KAFKA_VALUE | 1 - .../confluent_compat_topic/KAFKA_VALUE | 1 - .../confluent_compat_topic/KAFKA_VALUE | 1 - .../key_root_primitives/data.json | 1 + .../key_root_primitives/key-schema.json | 1 + .../key_root_primitives/value-schema.json | 1 + .../key_union/data.json | 1 + .../key_union/key-schema.json | 1 + .../key_union/value-schema.json | 1 + .../value_bytes/data.json | 1 + .../value_bytes/key-schema.json | 1 + .../value_bytes/value-schema.json | 10 + .../containers1/confluent_compat_topic/array | 1 - .../containers1/confluent_compat_topic/map | 1 - .../containers1/struct/integer | Bin 4 -> 0 bytes .../value_logicals/data.json | 3 + .../value_logicals/key-schema.json | 1 + .../value_logicals/value-schema.json | 37 ++++ .../confluent_compat_topic/row | Bin 4 -> 0 bytes .../matryoshka-dolls1/inner/inner_int | 1 - .../matryoshka-dolls1/inner/inner_string | 1 - .../matryoshka-dolls1/middle/inner | 1 - .../matryoshka-dolls1/middle/middle_array | 1 - .../nestedlogicals1/outer/inner | 1 - .../nulls1/confluent_compat_topic/f1 | 1 - .../nulls1/confluent_compat_topic/f2 | 0 .../nulls1/confluent_compat_topic/f3 | Bin 4 -> 0 bytes .../nulls1/confluent_compat_topic/f4 | Bin 1 -> 0 bytes .../nulls1/confluent_compat_topic/row | Bin 4 -> 0 bytes .../nulls2/confluent_compat_topic/f1 | 1 - .../nulls2/confluent_compat_topic/f2 | 1 - .../nulls2/confluent_compat_topic/f3 | Bin 4 -> 0 bytes .../nulls2/confluent_compat_topic/f4 | 0 .../nulls2/confluent_compat_topic/row | Bin 4 -> 0 bytes .../nulls3/confluent_compat_topic/f1 | 1 - .../nulls3/confluent_compat_topic/f2 | 0 .../nulls3/confluent_compat_topic/f3 | 0 .../nulls3/confluent_compat_topic/f4 | 1 - .../nulls3/confluent_compat_topic/row | Bin 4 -> 0 bytes .../nulls4/confluent_compat_topic/f1 | 1 - .../nulls4/confluent_compat_topic/f2 | 1 - .../nulls4/confluent_compat_topic/f3 | 0 .../nulls4/confluent_compat_topic/f4 | 0 .../nulls4/confluent_compat_topic/row | Bin 4 -> 0 bytes .../confluent_compat_topic/boolean_prim | Bin 1 -> 0 bytes .../confluent_compat_topic/double_prim | 1 - .../confluent_compat_topic/float_prim | 1 - .../confluent_compat_topic/int_prim | Bin 4 -> 0 bytes .../confluent_compat_topic/long_prim | Bin 8 -> 0 bytes .../confluent_compat_topic/null_prim | 0 .../primitives1/confluent_compat_topic/row | Bin 4 -> 0 bytes .../confluent_compat_topic/string_prim | 1 - .../value_root_primitives/data.json | 1 + .../value_root_primitives/key-schema.json | 1 + .../value_root_primitives/value-schema.json | 1 + .../value_union/data.json | 1 + .../value_union/key-schema.json | 1 + .../value_union/value-schema.json | 1 + .../src/test/resources/log4j.properties | 1 + 70 files changed, 257 insertions(+), 104 deletions(-) create mode 100644 google-cloud-bigtable-kafka-connect-sink/src/test/resources/compatibility_test_cases/key_bytes/data.json create mode 100644 google-cloud-bigtable-kafka-connect-sink/src/test/resources/compatibility_test_cases/key_bytes/key-schema.json create mode 100644 google-cloud-bigtable-kafka-connect-sink/src/test/resources/compatibility_test_cases/key_bytes/value-schema.json delete mode 100644 google-cloud-bigtable-kafka-connect-sink/src/test/resources/compatibility_test_cases/key_containers/confluent_sink_output/{a=1, b=4, c=3, d=2}#[1, 2, 3, 4]#Struct{integer=1}/confluent_compat_topic/KAFKA_VALUE delete mode 100644 google-cloud-bigtable-kafka-connect-sink/src/test/resources/compatibility_test_cases/key_logicals/confluent_sink_output/Mon Jul 11 22:11:42 UTC 2016#Thu Jan 01 00:00:16 UTC 1970#Mon Jul 11 00:00:00 UTC 2016#1.089357896855742840E+68/confluent_compat_topic/KAFKA_VALUE delete mode 100644 google-cloud-bigtable-kafka-connect-sink/src/test/resources/compatibility_test_cases/key_logicals/confluent_sink_output/Thu Jan 01 00:00:00 UTC 1970#Thu Jan 01 00:00:00 UTC 1970#Thu Jan 01 00:00:00 UTC 1970#2.53635900E+58/confluent_compat_topic/KAFKA_VALUE delete mode 100644 google-cloud-bigtable-kafka-connect-sink/src/test/resources/compatibility_test_cases/key_logicals/confluent_sink_output/Thu Jan 01 11:40:00 UTC 1970#Thu Jan 01 00:00:04 UTC 1970#Thu Jul 02 00:00:00 UTC 1981#1.6622282361690E+63/confluent_compat_topic/KAFKA_VALUE delete mode 100644 google-cloud-bigtable-kafka-connect-sink/src/test/resources/compatibility_test_cases/key_matryoshkas/confluent_sink_output/1#Struct{middle_array=[42.0, 42.42, 42.4242],inner=Struct{inner_int=42,inner_string=42}}#Struct{inner_int=-42,inner_string=-42}/confluent_compat_topic/KAFKA_VALUE delete mode 100644 google-cloud-bigtable-kafka-connect-sink/src/test/resources/compatibility_test_cases/key_nestedlogicals/confluent_sink_output/Struct{inner=Struct{logical_timestamp=Thu Jan 01 00:00:04 UTC 1970,logical_time=Thu Jan 01 00:00:04 UTC 1970,logical_date=Thu Feb 12 00:00:00 UTC 1970,logical_decimal=1.089357896855742840E+68}}/confluent_compat_topic/KAFKA_VALUE delete mode 100644 google-cloud-bigtable-kafka-connect-sink/src/test/resources/compatibility_test_cases/key_primitives/confluent_sink_output/1#false#4242#42424242424242#42.42#4.242424242424242E7#forty-two/confluent_compat_topic/KAFKA_VALUE create mode 100644 google-cloud-bigtable-kafka-connect-sink/src/test/resources/compatibility_test_cases/key_root_primitives/data.json create mode 100644 google-cloud-bigtable-kafka-connect-sink/src/test/resources/compatibility_test_cases/key_root_primitives/key-schema.json create mode 100644 google-cloud-bigtable-kafka-connect-sink/src/test/resources/compatibility_test_cases/key_root_primitives/value-schema.json create mode 100644 google-cloud-bigtable-kafka-connect-sink/src/test/resources/compatibility_test_cases/key_union/data.json create mode 100644 google-cloud-bigtable-kafka-connect-sink/src/test/resources/compatibility_test_cases/key_union/key-schema.json create mode 100644 google-cloud-bigtable-kafka-connect-sink/src/test/resources/compatibility_test_cases/key_union/value-schema.json create mode 100644 google-cloud-bigtable-kafka-connect-sink/src/test/resources/compatibility_test_cases/value_bytes/data.json create mode 100644 google-cloud-bigtable-kafka-connect-sink/src/test/resources/compatibility_test_cases/value_bytes/key-schema.json create mode 100644 google-cloud-bigtable-kafka-connect-sink/src/test/resources/compatibility_test_cases/value_bytes/value-schema.json delete mode 100644 google-cloud-bigtable-kafka-connect-sink/src/test/resources/compatibility_test_cases/value_containers/confluent_sink_output/containers1/confluent_compat_topic/array delete mode 100644 google-cloud-bigtable-kafka-connect-sink/src/test/resources/compatibility_test_cases/value_containers/confluent_sink_output/containers1/confluent_compat_topic/map delete mode 100644 google-cloud-bigtable-kafka-connect-sink/src/test/resources/compatibility_test_cases/value_containers/confluent_sink_output/containers1/struct/integer create mode 100644 google-cloud-bigtable-kafka-connect-sink/src/test/resources/compatibility_test_cases/value_logicals/data.json create mode 100644 google-cloud-bigtable-kafka-connect-sink/src/test/resources/compatibility_test_cases/value_logicals/key-schema.json create mode 100644 google-cloud-bigtable-kafka-connect-sink/src/test/resources/compatibility_test_cases/value_logicals/value-schema.json delete mode 100644 google-cloud-bigtable-kafka-connect-sink/src/test/resources/compatibility_test_cases/value_matryoshkas/confluent_sink_output/matryoshka-dolls1/confluent_compat_topic/row delete mode 100644 google-cloud-bigtable-kafka-connect-sink/src/test/resources/compatibility_test_cases/value_matryoshkas/confluent_sink_output/matryoshka-dolls1/inner/inner_int delete mode 100644 google-cloud-bigtable-kafka-connect-sink/src/test/resources/compatibility_test_cases/value_matryoshkas/confluent_sink_output/matryoshka-dolls1/inner/inner_string delete mode 100644 google-cloud-bigtable-kafka-connect-sink/src/test/resources/compatibility_test_cases/value_matryoshkas/confluent_sink_output/matryoshka-dolls1/middle/inner delete mode 100644 google-cloud-bigtable-kafka-connect-sink/src/test/resources/compatibility_test_cases/value_matryoshkas/confluent_sink_output/matryoshka-dolls1/middle/middle_array delete mode 100644 google-cloud-bigtable-kafka-connect-sink/src/test/resources/compatibility_test_cases/value_nestedlogicals/confluent_sink_output/nestedlogicals1/outer/inner delete mode 100644 google-cloud-bigtable-kafka-connect-sink/src/test/resources/compatibility_test_cases/value_nulls/confluent_sink_output/nulls1/confluent_compat_topic/f1 delete mode 100644 google-cloud-bigtable-kafka-connect-sink/src/test/resources/compatibility_test_cases/value_nulls/confluent_sink_output/nulls1/confluent_compat_topic/f2 delete mode 100644 google-cloud-bigtable-kafka-connect-sink/src/test/resources/compatibility_test_cases/value_nulls/confluent_sink_output/nulls1/confluent_compat_topic/f3 delete mode 100644 google-cloud-bigtable-kafka-connect-sink/src/test/resources/compatibility_test_cases/value_nulls/confluent_sink_output/nulls1/confluent_compat_topic/f4 delete mode 100644 google-cloud-bigtable-kafka-connect-sink/src/test/resources/compatibility_test_cases/value_nulls/confluent_sink_output/nulls1/confluent_compat_topic/row delete mode 100644 google-cloud-bigtable-kafka-connect-sink/src/test/resources/compatibility_test_cases/value_nulls/confluent_sink_output/nulls2/confluent_compat_topic/f1 delete mode 100644 google-cloud-bigtable-kafka-connect-sink/src/test/resources/compatibility_test_cases/value_nulls/confluent_sink_output/nulls2/confluent_compat_topic/f2 delete mode 100644 google-cloud-bigtable-kafka-connect-sink/src/test/resources/compatibility_test_cases/value_nulls/confluent_sink_output/nulls2/confluent_compat_topic/f3 delete mode 100644 google-cloud-bigtable-kafka-connect-sink/src/test/resources/compatibility_test_cases/value_nulls/confluent_sink_output/nulls2/confluent_compat_topic/f4 delete mode 100644 google-cloud-bigtable-kafka-connect-sink/src/test/resources/compatibility_test_cases/value_nulls/confluent_sink_output/nulls2/confluent_compat_topic/row delete mode 100644 google-cloud-bigtable-kafka-connect-sink/src/test/resources/compatibility_test_cases/value_nulls/confluent_sink_output/nulls3/confluent_compat_topic/f1 delete mode 100644 google-cloud-bigtable-kafka-connect-sink/src/test/resources/compatibility_test_cases/value_nulls/confluent_sink_output/nulls3/confluent_compat_topic/f2 delete mode 100644 google-cloud-bigtable-kafka-connect-sink/src/test/resources/compatibility_test_cases/value_nulls/confluent_sink_output/nulls3/confluent_compat_topic/f3 delete mode 100644 google-cloud-bigtable-kafka-connect-sink/src/test/resources/compatibility_test_cases/value_nulls/confluent_sink_output/nulls3/confluent_compat_topic/f4 delete mode 100644 google-cloud-bigtable-kafka-connect-sink/src/test/resources/compatibility_test_cases/value_nulls/confluent_sink_output/nulls3/confluent_compat_topic/row delete mode 100644 google-cloud-bigtable-kafka-connect-sink/src/test/resources/compatibility_test_cases/value_nulls/confluent_sink_output/nulls4/confluent_compat_topic/f1 delete mode 100644 google-cloud-bigtable-kafka-connect-sink/src/test/resources/compatibility_test_cases/value_nulls/confluent_sink_output/nulls4/confluent_compat_topic/f2 delete mode 100644 google-cloud-bigtable-kafka-connect-sink/src/test/resources/compatibility_test_cases/value_nulls/confluent_sink_output/nulls4/confluent_compat_topic/f3 delete mode 100644 google-cloud-bigtable-kafka-connect-sink/src/test/resources/compatibility_test_cases/value_nulls/confluent_sink_output/nulls4/confluent_compat_topic/f4 delete mode 100644 google-cloud-bigtable-kafka-connect-sink/src/test/resources/compatibility_test_cases/value_nulls/confluent_sink_output/nulls4/confluent_compat_topic/row delete mode 100644 google-cloud-bigtable-kafka-connect-sink/src/test/resources/compatibility_test_cases/value_primitives/confluent_sink_output/primitives1/confluent_compat_topic/boolean_prim delete mode 100644 google-cloud-bigtable-kafka-connect-sink/src/test/resources/compatibility_test_cases/value_primitives/confluent_sink_output/primitives1/confluent_compat_topic/double_prim delete mode 100644 google-cloud-bigtable-kafka-connect-sink/src/test/resources/compatibility_test_cases/value_primitives/confluent_sink_output/primitives1/confluent_compat_topic/float_prim delete mode 100644 google-cloud-bigtable-kafka-connect-sink/src/test/resources/compatibility_test_cases/value_primitives/confluent_sink_output/primitives1/confluent_compat_topic/int_prim delete mode 100644 google-cloud-bigtable-kafka-connect-sink/src/test/resources/compatibility_test_cases/value_primitives/confluent_sink_output/primitives1/confluent_compat_topic/long_prim delete mode 100644 google-cloud-bigtable-kafka-connect-sink/src/test/resources/compatibility_test_cases/value_primitives/confluent_sink_output/primitives1/confluent_compat_topic/null_prim delete mode 100644 google-cloud-bigtable-kafka-connect-sink/src/test/resources/compatibility_test_cases/value_primitives/confluent_sink_output/primitives1/confluent_compat_topic/row delete mode 100644 google-cloud-bigtable-kafka-connect-sink/src/test/resources/compatibility_test_cases/value_primitives/confluent_sink_output/primitives1/confluent_compat_topic/string_prim create mode 100644 google-cloud-bigtable-kafka-connect-sink/src/test/resources/compatibility_test_cases/value_root_primitives/data.json create mode 100644 google-cloud-bigtable-kafka-connect-sink/src/test/resources/compatibility_test_cases/value_root_primitives/key-schema.json create mode 100644 google-cloud-bigtable-kafka-connect-sink/src/test/resources/compatibility_test_cases/value_root_primitives/value-schema.json create mode 100644 google-cloud-bigtable-kafka-connect-sink/src/test/resources/compatibility_test_cases/value_union/data.json create mode 100644 google-cloud-bigtable-kafka-connect-sink/src/test/resources/compatibility_test_cases/value_union/key-schema.json create mode 100644 google-cloud-bigtable-kafka-connect-sink/src/test/resources/compatibility_test_cases/value_union/value-schema.json diff --git a/google-cloud-bigtable-kafka-connect-sink/pom.xml b/google-cloud-bigtable-kafka-connect-sink/pom.xml index 241a552df7..5658d7b436 100644 --- a/google-cloud-bigtable-kafka-connect-sink/pom.xml +++ b/google-cloud-bigtable-kafka-connect-sink/pom.xml @@ -27,6 +27,7 @@ 1.19.2 3.11.2 0.8.12 + 3.1.0 3.5.2 3.5.2 3.4.2 @@ -35,6 +36,10 @@ 11 11 UTF-8 + + ${project.basedir}/integration_test_plugins + https://hub-downloads.confluent.io/api/plugins/confluentinc/kafka-connect-gcp-bigtable/versions/2.0.28/confluentinc-kafka-connect-gcp-bigtable-2.0.28.zip + confluentsink.zip @@ -290,6 +295,7 @@ target/test-classes/fake_service_key.json localhost:8086 + ${integration.test.plugin.path} @@ -325,6 +331,31 @@ + + org.apache.maven.plugins + maven-antrun-plugin + ${antrun.version} + + + download-and-unzip-confluent-sink + pre-integration-test + + run + + + + + + + + + + diff --git a/google-cloud-bigtable-kafka-connect-sink/src/test/java/com/google/cloud/kafka/connect/bigtable/integration/BaseKafkaConnectBigtableIT.java b/google-cloud-bigtable-kafka-connect-sink/src/test/java/com/google/cloud/kafka/connect/bigtable/integration/BaseKafkaConnectBigtableIT.java index 33e1d61abf..88fed5dc60 100644 --- a/google-cloud-bigtable-kafka-connect-sink/src/test/java/com/google/cloud/kafka/connect/bigtable/integration/BaseKafkaConnectBigtableIT.java +++ b/google-cloud-bigtable-kafka-connect-sink/src/test/java/com/google/cloud/kafka/connect/bigtable/integration/BaseKafkaConnectBigtableIT.java @@ -70,6 +70,10 @@ public Map readAllRows(BigtableDataClient bigtable, String tabl } } + public long cellCount(Map rows) { + return rows.values().stream().mapToLong(r -> r.getCells().size()).sum(); + } + public void waitUntilBigtableContainsNumberOfRows(String tableId, long numberOfRows) throws InterruptedException { waitForCondition( @@ -78,6 +82,14 @@ public void waitUntilBigtableContainsNumberOfRows(String tableId, long numberOfR "Records not consumed in time."); } + public void waitUntilBigtableContainsNumberOfCells(String tableId, long numberOfCells) + throws InterruptedException { + waitForCondition( + () -> cellCount(readAllRows(bigtableData, tableId)) == numberOfCells, + DEFAULT_BIGTABLE_RETRY_TIMEOUT_MILLIS, + "Records not consumed in time"); + } + public void waitUntilBigtableTableExists(String tableId) throws InterruptedException { waitForCondition( () -> { diff --git a/google-cloud-bigtable-kafka-connect-sink/src/test/java/com/google/cloud/kafka/connect/bigtable/integration/BaseKafkaConnectIT.java b/google-cloud-bigtable-kafka-connect-sink/src/test/java/com/google/cloud/kafka/connect/bigtable/integration/BaseKafkaConnectIT.java index e9e9929498..2b07add438 100644 --- a/google-cloud-bigtable-kafka-connect-sink/src/test/java/com/google/cloud/kafka/connect/bigtable/integration/BaseKafkaConnectIT.java +++ b/google-cloud-bigtable-kafka-connect-sink/src/test/java/com/google/cloud/kafka/connect/bigtable/integration/BaseKafkaConnectIT.java @@ -62,6 +62,7 @@ public abstract class BaseKafkaConnectIT extends BaseIT { public static final long PRODUCE_TIMEOUT_MILLIS = 15000L; // https://cloud.google.com/bigtable/docs/reference/admin/rpc/google.bigtable.admin.v2#createtablerequest public static int MAX_BIGTABLE_TABLE_NAME_LENGTH = 50; + public static final String PLUGIN_PATH_ENV_VAR_NAME = "INTEGRATION_TEST_PLUGINS_PATH"; private final Logger logger = LoggerFactory.getLogger(BaseKafkaConnectIT.class); @@ -82,9 +83,6 @@ public void tearDownConnect() { public void startConnect() { logger.info("Starting embedded Kafka Connect cluster..."); - Map workerProps = new HashMap<>(); - workerProps.put(WorkerConfig.OFFSET_COMMIT_INTERVAL_MS_CONFIG, Long.toString(10000)); - workerProps.put(WorkerConfig.PLUGIN_DISCOVERY_CONFIG, PluginDiscoveryMode.HYBRID_WARN.name()); Properties brokerProps = new Properties(); brokerProps.put("socket.request.max.bytes", maxKafkaMessageSizeBytes); @@ -105,7 +103,7 @@ public void startConnect() { .numWorkers(numWorkers) .numBrokers(numBrokers) .brokerProps(brokerProps) - .workerProps(workerProps) + .workerProps(workerProps()) .clientConfigs(clientConfigs) .build(); @@ -124,6 +122,13 @@ public void startConnect() { connect.kafka().bootstrapServers()); } + public Map workerProps() { + Map workerProps = new HashMap<>(); + workerProps.put(WorkerConfig.OFFSET_COMMIT_INTERVAL_MS_CONFIG, Long.toString(10000)); + workerProps.put(WorkerConfig.PLUGIN_DISCOVERY_CONFIG, PluginDiscoveryMode.HYBRID_WARN.name()); + return workerProps; + } + public void stopConnect() { logger.info( "Stopping embedded Kafka Connect cluster using bootstrap servers: {}", diff --git a/google-cloud-bigtable-kafka-connect-sink/src/test/java/com/google/cloud/kafka/connect/bigtable/integration/ConfluentCompatibilityIT.java b/google-cloud-bigtable-kafka-connect-sink/src/test/java/com/google/cloud/kafka/connect/bigtable/integration/ConfluentCompatibilityIT.java index 8d821d0cde..b123189a8f 100644 --- a/google-cloud-bigtable-kafka-connect-sink/src/test/java/com/google/cloud/kafka/connect/bigtable/integration/ConfluentCompatibilityIT.java +++ b/google-cloud-bigtable-kafka-connect-sink/src/test/java/com/google/cloud/kafka/connect/bigtable/integration/ConfluentCompatibilityIT.java @@ -22,10 +22,11 @@ * the WePay BigQuery Kafka Connector, Copyright WePay, Inc. */ +import static org.apache.kafka.connect.runtime.ConnectorConfig.CONNECTOR_CLASS_CONFIG; import static org.junit.Assert.assertEquals; -import static org.junit.Assert.assertFalse; -import static org.junit.Assert.assertTrue; +import static org.junit.Assert.assertNotEquals; +import com.google.cloud.bigtable.admin.v2.models.CreateTableRequest; import com.google.cloud.bigtable.data.v2.models.Row; import com.google.cloud.bigtable.data.v2.models.RowCell; import com.google.cloud.kafka.connect.bigtable.config.BigtableSinkConfig; @@ -35,27 +36,22 @@ import io.confluent.kafka.serializers.AbstractKafkaSchemaSerDeConfig; import java.io.IOException; import java.io.InputStream; -import java.net.URISyntaxException; import java.nio.charset.StandardCharsets; -import java.nio.file.Files; -import java.nio.file.Path; -import java.nio.file.Paths; -import java.util.AbstractMap; import java.util.Arrays; import java.util.Collection; import java.util.HashMap; import java.util.List; import java.util.Map; +import java.util.Objects; import java.util.Properties; -import java.util.Set; import java.util.concurrent.ExecutionException; import java.util.concurrent.TimeUnit; import java.util.concurrent.TimeoutException; -import java.util.stream.Collectors; import kafka.common.MessageReader; import org.apache.kafka.clients.producer.Producer; import org.apache.kafka.clients.producer.ProducerRecord; import org.apache.kafka.connect.runtime.ConnectorConfig; +import org.apache.kafka.connect.runtime.WorkerConfig; import org.junit.Test; import org.junit.runner.RunWith; import org.junit.runners.Parameterized; @@ -63,8 +59,6 @@ @RunWith(Parameterized.class) public class ConfluentCompatibilityIT extends BaseKafkaConnectBigtableSchemaRegistryIT { public static String TEST_CASES_DIR = "compatibility_test_cases"; - // Needed since the default column family is the topic's name in the default configuration. - public static String COMPATIBILITY_TEST_TOPIC = "confluent_compat_topic"; private String testCase; private Compatibility compatibility; @@ -78,6 +72,9 @@ public ConfluentCompatibilityIT(String testCase, Compatibility compatibility) { public static Collection testCases() { return Arrays.asList( new Object[][] { + // Confluent serializes bytes in keys incorrectly. For details, see the comments + // in KeyMapper#serializeKeyElement(). + {"key_bytes", Compatibility.KEY_MISMATCH}, {"key_containers", Compatibility.FULL}, // We serialize `Date`-based logical types differently. For details, see the comments // in KeyMapper, especially the ones in serializeKeyElement(). @@ -87,47 +84,105 @@ public static Collection testCases() { // in KeyMapper, especially the ones in serializeKeyElement(). {"key_nestedlogicals", Compatibility.KEY_MISMATCH}, {"key_primitives", Compatibility.FULL}, + {"key_root_primitives", Compatibility.FULL}, + {"key_union", Compatibility.FULL}, + // Confluent connector fails with an invalid class cast. + {"value_bytes", Compatibility.CONFLUENT_BROKEN}, {"value_containers", Compatibility.FULL}, + // Confluent connector fails with an invalid class cast. + {"value_logicals", Compatibility.CONFLUENT_BROKEN}, {"value_matryoshkas", Compatibility.FULL}, {"value_nestedlogicals", Compatibility.FULL}, {"value_nulls", Compatibility.FULL}, {"value_primitives", Compatibility.FULL}, + {"value_root_primitives", Compatibility.FULL}, + {"value_union", Compatibility.FULL}, }); } @Test - public void testCasesUsingSchemaRegistry() - throws InterruptedException, URISyntaxException, IOException { - String testId = startConnector(); - populateTopic(testId); - Map, ByteString>> expected = getExpectedOutput(); - waitUntilBigtableContainsNumberOfRows(testId, expected.size()); - Map allRows = readAllRows(bigtableData, testId); + public void testCasesUsingSchemaRegistry() throws InterruptedException, IOException { + String confluentTestId = startConfluentConnector(); + String googleTestId = startThisConnector(confluentTestId); + assertNotEquals(confluentTestId, googleTestId); + + bigtableAdmin.createTable(CreateTableRequest.of(confluentTestId)); + bigtableAdmin.createTable(CreateTableRequest.of(googleTestId)); + + populateTopic(confluentTestId); + populateTopic(googleTestId); + + long expectedRows = getInputSize(); + waitUntilBigtableContainsNumberOfRows(googleTestId, expectedRows); + Map allGoogleRows = readAllRows(bigtableData, googleTestId); + assertEquals(expectedRows, allGoogleRows.size()); + + Map allConfluentRows = null; switch (compatibility) { case FULL: - assertRowsMatch(expected, allRows.values()); + case KEY_MISMATCH: + // Done like this because Confluent sink seems to write rows cell-by-cell rather than + // atomically. + waitUntilBigtableContainsNumberOfCells(confluentTestId, cellCount(allGoogleRows)); + allConfluentRows = readAllRows(bigtableData, confluentTestId); + assertEquals(expectedRows, allConfluentRows.size()); + break; + case CONFLUENT_BROKEN: + break; + } + switch (compatibility) { + case FULL: + assertRowsAreTheSame(allConfluentRows, allGoogleRows); break; case KEY_MISMATCH: - assertEquals(expected.size(), allRows.size()); - assertTrue(allRows.values().stream().allMatch(r -> r.getCells().size() == 1)); - Set allValues = - allRows.values().stream() - .map(r -> r.getCells().get(0).getValue()) - .collect(Collectors.toSet()); - Set allExpectedValues = - expected.values().stream() - .flatMap(m -> m.values().stream()) - .collect(Collectors.toSet()); - assertEquals(allExpectedValues, allValues); + assertEquals(rowToValues(allConfluentRows.values()), rowToValues(allGoogleRows.values())); + break; + case CONFLUENT_BROKEN: break; } connect .assertions() - .assertConnectorAndExactlyNumTasksAreRunning(testId, numTasks, "Some task failed."); + .assertConnectorAndExactlyNumTasksAreRunning(googleTestId, numTasks, "Some task failed."); + switch (compatibility) { + case FULL: + case KEY_MISMATCH: + connect + .assertions() + .assertConnectorAndExactlyNumTasksAreRunning( + confluentTestId, numTasks, "Some Google connector task failed."); + break; + case CONFLUENT_BROKEN: + connect + .assertions() + .assertConnectorIsRunningAndTasksHaveFailed( + confluentTestId, + numTasks, + "Confluent sink should've been broken for this test case."); + break; + } } - public String startConnector() throws InterruptedException { + public String startConfluentConnector() throws InterruptedException { Map connectorProps = baseConnectorProps(); + connectorProps.put( + CONNECTOR_CLASS_CONFIG, "io.confluent.connect.gcp.bigtable.BigtableSinkConnector"); + connectorProps.put("confluent.license", ""); + connectorProps.put("confluent.topic.bootstrap.servers", connect.kafka().bootstrapServers()); + connectorProps.put("confluent.topic.replication.factor", "1"); + // TODO: fix it when transitioning to kokoro. + connectorProps.put( + BigtableSinkConfig.GCP_CREDENTIALS_PATH_CONFIG, + Objects.requireNonNull(System.getenv("GOOGLE_APPLICATION_CREDENTIALS"))); + return startConnector(connectorProps); + } + + public String startThisConnector(String confluentConnectorId) throws InterruptedException { + Map connectorProps = baseConnectorProps(); + connectorProps.put(BigtableSinkConfig.DEFAULT_COLUMN_FAMILY_CONFIG, confluentConnectorId); + return startConnector(connectorProps); + } + + public String startConnector(Map connectorProps) throws InterruptedException { connectorProps.put(ConnectorConfig.KEY_CONVERTER_CLASS_CONFIG, AvroConverter.class.getName()); connectorProps.put( ConnectorConfig.KEY_CONVERTER_CLASS_CONFIG @@ -140,9 +195,8 @@ public String startConnector() throws InterruptedException { + "." + AbstractKafkaSchemaSerDeConfig.SCHEMA_REGISTRY_URL_CONFIG, schemaRegistry.schemaRegistryUrl()); - connectorProps.put(BigtableSinkConfig.AUTO_CREATE_TABLES_CONFIG, "true"); + connectorProps.put(BigtableSinkConfig.AUTO_CREATE_TABLES_CONFIG, "false"); connectorProps.put(BigtableSinkConfig.AUTO_CREATE_COLUMN_FAMILIES_CONFIG, "true"); - connectorProps.put(BigtableSinkConfig.DEFAULT_COLUMN_FAMILY_CONFIG, COMPATIBILITY_TEST_TOPIC); connectorProps.put(BigtableSinkConfig.ROW_KEY_DELIMITER_CONFIG, "#"); String topic = startSingleTopicConnector(connectorProps); connect @@ -151,6 +205,15 @@ public String startConnector() throws InterruptedException { return topic; } + @Override + public Map workerProps() { + Map props = super.workerProps(); + String pluginPath = Objects.requireNonNull(System.getenv(PLUGIN_PATH_ENV_VAR_NAME)); + // Enabling embedded Kafka Connect to use the Confluent's sink. + props.put(WorkerConfig.PLUGIN_PATH_CONFIG, pluginPath); + return props; + } + public void populateTopic(String topic) throws IOException { String keySchema = readStringResource(getTestCaseDir() + "/key-schema.json"); String valueSchema = readStringResource(getTestCaseDir() + "/value-schema.json"); @@ -191,57 +254,45 @@ private String getTestCaseDir() { return String.format("%s/%s", TEST_CASES_DIR, testCase); } - private Map, ByteString>> getExpectedOutput() - throws URISyntaxException, IOException { - Map, ByteString>> result = new HashMap<>(); - - String expectedOutputDir = getTestCaseDir() + "/confluent_sink_output"; - List outputPaths = - Files.find( - Paths.get(getClassLoader().getResource(expectedOutputDir).toURI()), - 3, - (path, attr) -> attr.isRegularFile()) - .collect(Collectors.toList()); - - for (Path outputPath : outputPaths) { - String[] outputPathParts = outputPath.toString().split("/"); - int outputPathPartsLength = outputPathParts.length; - String row = outputPathParts[outputPathPartsLength - 3]; - String columnFamily = outputPathParts[outputPathPartsLength - 2]; - String columnQualifier = outputPathParts[outputPathPartsLength - 1]; - byte[] value = Files.readAllBytes(outputPath); - - result.putIfAbsent(row, new HashMap<>()); - Map.Entry familyAndQualifier = - new AbstractMap.SimpleImmutableEntry<>(columnFamily, columnQualifier); - assertFalse(result.get(row).containsKey(familyAndQualifier)); - result.get(row).put(familyAndQualifier, ByteString.copyFrom(value)); - } + private long getInputSize() throws IOException { + String data = readStringResource(getTestCaseDir() + "/data.json"); + return Arrays.stream(data.split("\n")).filter(s -> !s.trim().isEmpty()).count(); + } - return result; + private void assertRowsAreTheSame(Map expected, Map actual) { + assertEquals(expected.keySet(), actual.keySet()); + for (Row expectedRow : expected.values()) { + Row actualRow = actual.get(expectedRow.getKey()); + + List expectedCells = expectedRow.getCells(); + assertEquals(expectedCells.size(), actualRow.getCells().size()); + for (RowCell expectedCell : expectedCells) { + ByteString expectedValue = expectedCell.getValue(); + List actualCells = + actualRow.getCells(expectedCell.getFamily(), expectedCell.getQualifier()); + assertEquals(1, actualCells.size()); + ByteString actualValue = actualCells.get(0).getValue(); + assertEquals(expectedValue, actualValue); + } + } } - private void assertRowsMatch( - Map, ByteString>> expected, - Collection bigtableRows) { - assertEquals(expected.size(), bigtableRows.size()); - for (Row row : bigtableRows) { - String key = new String(row.getKey().toByteArray(), StandardCharsets.UTF_8); - Map, ByteString> bigtableRow = new HashMap<>(); - for (RowCell cell : row.getCells()) { - Map.Entry familyAndQualifier = - new AbstractMap.SimpleImmutableEntry<>( - cell.getFamily(), - new String(cell.getQualifier().toByteArray(), StandardCharsets.UTF_8)); - assertFalse(bigtableRow.containsKey(familyAndQualifier)); - bigtableRow.put(familyAndQualifier, cell.getValue()); + private Map>> rowToValues(Collection rows) { + Map>> result = new HashMap<>(); + for (Row r : rows) { + for (RowCell c : r.getCells()) { + result + .computeIfAbsent(c.getFamily(), ignored -> new HashMap<>()) + .computeIfAbsent(c.getQualifier(), ignored -> new HashMap<>()) + .merge(c.getValue(), 1L, Long::sum); } - assertEquals(expected.get(key), bigtableRow); } + return result; } public enum Compatibility { FULL, KEY_MISMATCH, + CONFLUENT_BROKEN, } } diff --git a/google-cloud-bigtable-kafka-connect-sink/src/test/resources/compatibility_test_cases/key_bytes/data.json b/google-cloud-bigtable-kafka-connect-sink/src/test/resources/compatibility_test_cases/key_bytes/data.json new file mode 100644 index 0000000000..b6bc90fb2a --- /dev/null +++ b/google-cloud-bigtable-kafka-connect-sink/src/test/resources/compatibility_test_cases/key_bytes/data.json @@ -0,0 +1 @@ +{"bytes_prim":"\u0000\u000f\u001e\u002d\u003c\u004b\u005a\u0069\u0078"} "bytes1" diff --git a/google-cloud-bigtable-kafka-connect-sink/src/test/resources/compatibility_test_cases/key_bytes/key-schema.json b/google-cloud-bigtable-kafka-connect-sink/src/test/resources/compatibility_test_cases/key_bytes/key-schema.json new file mode 100644 index 0000000000..5aa1f9fa99 --- /dev/null +++ b/google-cloud-bigtable-kafka-connect-sink/src/test/resources/compatibility_test_cases/key_bytes/key-schema.json @@ -0,0 +1,10 @@ +{ + "type": "record", + "name": "primitives", + "fields": [ + { + "name": "bytes_prim", + "type": "bytes" + } + ] +} \ No newline at end of file diff --git a/google-cloud-bigtable-kafka-connect-sink/src/test/resources/compatibility_test_cases/key_bytes/value-schema.json b/google-cloud-bigtable-kafka-connect-sink/src/test/resources/compatibility_test_cases/key_bytes/value-schema.json new file mode 100644 index 0000000000..4db187ebb3 --- /dev/null +++ b/google-cloud-bigtable-kafka-connect-sink/src/test/resources/compatibility_test_cases/key_bytes/value-schema.json @@ -0,0 +1 @@ +{"type":"string"} \ No newline at end of file diff --git a/google-cloud-bigtable-kafka-connect-sink/src/test/resources/compatibility_test_cases/key_containers/confluent_sink_output/{a=1, b=4, c=3, d=2}#[1, 2, 3, 4]#Struct{integer=1}/confluent_compat_topic/KAFKA_VALUE b/google-cloud-bigtable-kafka-connect-sink/src/test/resources/compatibility_test_cases/key_containers/confluent_sink_output/{a=1, b=4, c=3, d=2}#[1, 2, 3, 4]#Struct{integer=1}/confluent_compat_topic/KAFKA_VALUE deleted file mode 100644 index 499149dde4..0000000000 --- a/google-cloud-bigtable-kafka-connect-sink/src/test/resources/compatibility_test_cases/key_containers/confluent_sink_output/{a=1, b=4, c=3, d=2}#[1, 2, 3, 4]#Struct{integer=1}/confluent_compat_topic/KAFKA_VALUE +++ /dev/null @@ -1 +0,0 @@ -containers1 \ No newline at end of file diff --git a/google-cloud-bigtable-kafka-connect-sink/src/test/resources/compatibility_test_cases/key_logicals/confluent_sink_output/Mon Jul 11 22:11:42 UTC 2016#Thu Jan 01 00:00:16 UTC 1970#Mon Jul 11 00:00:00 UTC 2016#1.089357896855742840E+68/confluent_compat_topic/KAFKA_VALUE b/google-cloud-bigtable-kafka-connect-sink/src/test/resources/compatibility_test_cases/key_logicals/confluent_sink_output/Mon Jul 11 22:11:42 UTC 2016#Thu Jan 01 00:00:16 UTC 1970#Mon Jul 11 00:00:00 UTC 2016#1.089357896855742840E+68/confluent_compat_topic/KAFKA_VALUE deleted file mode 100644 index d4d78296cb..0000000000 --- a/google-cloud-bigtable-kafka-connect-sink/src/test/resources/compatibility_test_cases/key_logicals/confluent_sink_output/Mon Jul 11 22:11:42 UTC 2016#Thu Jan 01 00:00:16 UTC 1970#Mon Jul 11 00:00:00 UTC 2016#1.089357896855742840E+68/confluent_compat_topic/KAFKA_VALUE +++ /dev/null @@ -1 +0,0 @@ -logical3 \ No newline at end of file diff --git a/google-cloud-bigtable-kafka-connect-sink/src/test/resources/compatibility_test_cases/key_logicals/confluent_sink_output/Thu Jan 01 00:00:00 UTC 1970#Thu Jan 01 00:00:00 UTC 1970#Thu Jan 01 00:00:00 UTC 1970#2.53635900E+58/confluent_compat_topic/KAFKA_VALUE b/google-cloud-bigtable-kafka-connect-sink/src/test/resources/compatibility_test_cases/key_logicals/confluent_sink_output/Thu Jan 01 00:00:00 UTC 1970#Thu Jan 01 00:00:00 UTC 1970#Thu Jan 01 00:00:00 UTC 1970#2.53635900E+58/confluent_compat_topic/KAFKA_VALUE deleted file mode 100644 index 987b96f9e7..0000000000 --- a/google-cloud-bigtable-kafka-connect-sink/src/test/resources/compatibility_test_cases/key_logicals/confluent_sink_output/Thu Jan 01 00:00:00 UTC 1970#Thu Jan 01 00:00:00 UTC 1970#Thu Jan 01 00:00:00 UTC 1970#2.53635900E+58/confluent_compat_topic/KAFKA_VALUE +++ /dev/null @@ -1 +0,0 @@ -logical1 \ No newline at end of file diff --git a/google-cloud-bigtable-kafka-connect-sink/src/test/resources/compatibility_test_cases/key_logicals/confluent_sink_output/Thu Jan 01 11:40:00 UTC 1970#Thu Jan 01 00:00:04 UTC 1970#Thu Jul 02 00:00:00 UTC 1981#1.6622282361690E+63/confluent_compat_topic/KAFKA_VALUE b/google-cloud-bigtable-kafka-connect-sink/src/test/resources/compatibility_test_cases/key_logicals/confluent_sink_output/Thu Jan 01 11:40:00 UTC 1970#Thu Jan 01 00:00:04 UTC 1970#Thu Jul 02 00:00:00 UTC 1981#1.6622282361690E+63/confluent_compat_topic/KAFKA_VALUE deleted file mode 100644 index cf0c11b93f..0000000000 --- a/google-cloud-bigtable-kafka-connect-sink/src/test/resources/compatibility_test_cases/key_logicals/confluent_sink_output/Thu Jan 01 11:40:00 UTC 1970#Thu Jan 01 00:00:04 UTC 1970#Thu Jul 02 00:00:00 UTC 1981#1.6622282361690E+63/confluent_compat_topic/KAFKA_VALUE +++ /dev/null @@ -1 +0,0 @@ -logical2 \ No newline at end of file diff --git a/google-cloud-bigtable-kafka-connect-sink/src/test/resources/compatibility_test_cases/key_matryoshkas/confluent_sink_output/1#Struct{middle_array=[42.0, 42.42, 42.4242],inner=Struct{inner_int=42,inner_string=42}}#Struct{inner_int=-42,inner_string=-42}/confluent_compat_topic/KAFKA_VALUE b/google-cloud-bigtable-kafka-connect-sink/src/test/resources/compatibility_test_cases/key_matryoshkas/confluent_sink_output/1#Struct{middle_array=[42.0, 42.42, 42.4242],inner=Struct{inner_int=42,inner_string=42}}#Struct{inner_int=-42,inner_string=-42}/confluent_compat_topic/KAFKA_VALUE deleted file mode 100644 index 98b3558661..0000000000 --- a/google-cloud-bigtable-kafka-connect-sink/src/test/resources/compatibility_test_cases/key_matryoshkas/confluent_sink_output/1#Struct{middle_array=[42.0, 42.42, 42.4242],inner=Struct{inner_int=42,inner_string=42}}#Struct{inner_int=-42,inner_string=-42}/confluent_compat_topic/KAFKA_VALUE +++ /dev/null @@ -1 +0,0 @@ -matryoshka-dolls1 \ No newline at end of file diff --git a/google-cloud-bigtable-kafka-connect-sink/src/test/resources/compatibility_test_cases/key_nestedlogicals/confluent_sink_output/Struct{inner=Struct{logical_timestamp=Thu Jan 01 00:00:04 UTC 1970,logical_time=Thu Jan 01 00:00:04 UTC 1970,logical_date=Thu Feb 12 00:00:00 UTC 1970,logical_decimal=1.089357896855742840E+68}}/confluent_compat_topic/KAFKA_VALUE b/google-cloud-bigtable-kafka-connect-sink/src/test/resources/compatibility_test_cases/key_nestedlogicals/confluent_sink_output/Struct{inner=Struct{logical_timestamp=Thu Jan 01 00:00:04 UTC 1970,logical_time=Thu Jan 01 00:00:04 UTC 1970,logical_date=Thu Feb 12 00:00:00 UTC 1970,logical_decimal=1.089357896855742840E+68}}/confluent_compat_topic/KAFKA_VALUE deleted file mode 100644 index 41d3f3f489..0000000000 --- a/google-cloud-bigtable-kafka-connect-sink/src/test/resources/compatibility_test_cases/key_nestedlogicals/confluent_sink_output/Struct{inner=Struct{logical_timestamp=Thu Jan 01 00:00:04 UTC 1970,logical_time=Thu Jan 01 00:00:04 UTC 1970,logical_date=Thu Feb 12 00:00:00 UTC 1970,logical_decimal=1.089357896855742840E+68}}/confluent_compat_topic/KAFKA_VALUE +++ /dev/null @@ -1 +0,0 @@ -nestedlogicals1 \ No newline at end of file diff --git a/google-cloud-bigtable-kafka-connect-sink/src/test/resources/compatibility_test_cases/key_primitives/confluent_sink_output/1#false#4242#42424242424242#42.42#4.242424242424242E7#forty-two/confluent_compat_topic/KAFKA_VALUE b/google-cloud-bigtable-kafka-connect-sink/src/test/resources/compatibility_test_cases/key_primitives/confluent_sink_output/1#false#4242#42424242424242#42.42#4.242424242424242E7#forty-two/confluent_compat_topic/KAFKA_VALUE deleted file mode 100644 index e3cf798959..0000000000 --- a/google-cloud-bigtable-kafka-connect-sink/src/test/resources/compatibility_test_cases/key_primitives/confluent_sink_output/1#false#4242#42424242424242#42.42#4.242424242424242E7#forty-two/confluent_compat_topic/KAFKA_VALUE +++ /dev/null @@ -1 +0,0 @@ -primitives1 \ No newline at end of file diff --git a/google-cloud-bigtable-kafka-connect-sink/src/test/resources/compatibility_test_cases/key_root_primitives/data.json b/google-cloud-bigtable-kafka-connect-sink/src/test/resources/compatibility_test_cases/key_root_primitives/data.json new file mode 100644 index 0000000000..f7fdfb1626 --- /dev/null +++ b/google-cloud-bigtable-kafka-connect-sink/src/test/resources/compatibility_test_cases/key_root_primitives/data.json @@ -0,0 +1 @@ +3 "rootPrimitives1" diff --git a/google-cloud-bigtable-kafka-connect-sink/src/test/resources/compatibility_test_cases/key_root_primitives/key-schema.json b/google-cloud-bigtable-kafka-connect-sink/src/test/resources/compatibility_test_cases/key_root_primitives/key-schema.json new file mode 100644 index 0000000000..f08c4dfdee --- /dev/null +++ b/google-cloud-bigtable-kafka-connect-sink/src/test/resources/compatibility_test_cases/key_root_primitives/key-schema.json @@ -0,0 +1 @@ +"int" \ No newline at end of file diff --git a/google-cloud-bigtable-kafka-connect-sink/src/test/resources/compatibility_test_cases/key_root_primitives/value-schema.json b/google-cloud-bigtable-kafka-connect-sink/src/test/resources/compatibility_test_cases/key_root_primitives/value-schema.json new file mode 100644 index 0000000000..4db187ebb3 --- /dev/null +++ b/google-cloud-bigtable-kafka-connect-sink/src/test/resources/compatibility_test_cases/key_root_primitives/value-schema.json @@ -0,0 +1 @@ +{"type":"string"} \ No newline at end of file diff --git a/google-cloud-bigtable-kafka-connect-sink/src/test/resources/compatibility_test_cases/key_union/data.json b/google-cloud-bigtable-kafka-connect-sink/src/test/resources/compatibility_test_cases/key_union/data.json new file mode 100644 index 0000000000..d8297119ff --- /dev/null +++ b/google-cloud-bigtable-kafka-connect-sink/src/test/resources/compatibility_test_cases/key_union/data.json @@ -0,0 +1 @@ +{"int":3} "union1" diff --git a/google-cloud-bigtable-kafka-connect-sink/src/test/resources/compatibility_test_cases/key_union/key-schema.json b/google-cloud-bigtable-kafka-connect-sink/src/test/resources/compatibility_test_cases/key_union/key-schema.json new file mode 100644 index 0000000000..2fe380d7c3 --- /dev/null +++ b/google-cloud-bigtable-kafka-connect-sink/src/test/resources/compatibility_test_cases/key_union/key-schema.json @@ -0,0 +1 @@ +["int"] \ No newline at end of file diff --git a/google-cloud-bigtable-kafka-connect-sink/src/test/resources/compatibility_test_cases/key_union/value-schema.json b/google-cloud-bigtable-kafka-connect-sink/src/test/resources/compatibility_test_cases/key_union/value-schema.json new file mode 100644 index 0000000000..4db187ebb3 --- /dev/null +++ b/google-cloud-bigtable-kafka-connect-sink/src/test/resources/compatibility_test_cases/key_union/value-schema.json @@ -0,0 +1 @@ +{"type":"string"} \ No newline at end of file diff --git a/google-cloud-bigtable-kafka-connect-sink/src/test/resources/compatibility_test_cases/value_bytes/data.json b/google-cloud-bigtable-kafka-connect-sink/src/test/resources/compatibility_test_cases/value_bytes/data.json new file mode 100644 index 0000000000..14d99f30e2 --- /dev/null +++ b/google-cloud-bigtable-kafka-connect-sink/src/test/resources/compatibility_test_cases/value_bytes/data.json @@ -0,0 +1 @@ +"bytes1" {"bytes_prim":"\u0000\u000f\u001e\u002d\u003c\u004b\u005a\u0069\u0078"} diff --git a/google-cloud-bigtable-kafka-connect-sink/src/test/resources/compatibility_test_cases/value_bytes/key-schema.json b/google-cloud-bigtable-kafka-connect-sink/src/test/resources/compatibility_test_cases/value_bytes/key-schema.json new file mode 100644 index 0000000000..4db187ebb3 --- /dev/null +++ b/google-cloud-bigtable-kafka-connect-sink/src/test/resources/compatibility_test_cases/value_bytes/key-schema.json @@ -0,0 +1 @@ +{"type":"string"} \ No newline at end of file diff --git a/google-cloud-bigtable-kafka-connect-sink/src/test/resources/compatibility_test_cases/value_bytes/value-schema.json b/google-cloud-bigtable-kafka-connect-sink/src/test/resources/compatibility_test_cases/value_bytes/value-schema.json new file mode 100644 index 0000000000..5aa1f9fa99 --- /dev/null +++ b/google-cloud-bigtable-kafka-connect-sink/src/test/resources/compatibility_test_cases/value_bytes/value-schema.json @@ -0,0 +1,10 @@ +{ + "type": "record", + "name": "primitives", + "fields": [ + { + "name": "bytes_prim", + "type": "bytes" + } + ] +} \ No newline at end of file diff --git a/google-cloud-bigtable-kafka-connect-sink/src/test/resources/compatibility_test_cases/value_containers/confluent_sink_output/containers1/confluent_compat_topic/array b/google-cloud-bigtable-kafka-connect-sink/src/test/resources/compatibility_test_cases/value_containers/confluent_sink_output/containers1/confluent_compat_topic/array deleted file mode 100644 index fde6c1d745..0000000000 --- a/google-cloud-bigtable-kafka-connect-sink/src/test/resources/compatibility_test_cases/value_containers/confluent_sink_output/containers1/confluent_compat_topic/array +++ /dev/null @@ -1 +0,0 @@ -[1,2,3,4] \ No newline at end of file diff --git a/google-cloud-bigtable-kafka-connect-sink/src/test/resources/compatibility_test_cases/value_containers/confluent_sink_output/containers1/confluent_compat_topic/map b/google-cloud-bigtable-kafka-connect-sink/src/test/resources/compatibility_test_cases/value_containers/confluent_sink_output/containers1/confluent_compat_topic/map deleted file mode 100644 index 5aed479781..0000000000 --- a/google-cloud-bigtable-kafka-connect-sink/src/test/resources/compatibility_test_cases/value_containers/confluent_sink_output/containers1/confluent_compat_topic/map +++ /dev/null @@ -1 +0,0 @@ -{"a":1,"b":4,"c":3,"d":2} \ No newline at end of file diff --git a/google-cloud-bigtable-kafka-connect-sink/src/test/resources/compatibility_test_cases/value_containers/confluent_sink_output/containers1/struct/integer b/google-cloud-bigtable-kafka-connect-sink/src/test/resources/compatibility_test_cases/value_containers/confluent_sink_output/containers1/struct/integer deleted file mode 100644 index 720d64f4baafc33efdf971f02084aca5f25b34a5..0000000000000000000000000000000000000000 GIT binary patch literal 0 HcmV?d00001 literal 4 LcmZQzU|<9Q00jU7 diff --git a/google-cloud-bigtable-kafka-connect-sink/src/test/resources/compatibility_test_cases/value_logicals/data.json b/google-cloud-bigtable-kafka-connect-sink/src/test/resources/compatibility_test_cases/value_logicals/data.json new file mode 100644 index 0000000000..ac030ae8ad --- /dev/null +++ b/google-cloud-bigtable-kafka-connect-sink/src/test/resources/compatibility_test_cases/value_logicals/data.json @@ -0,0 +1,3 @@ +"logical1" {"logical_timestamp":0,"logical_time":0,"logical_date":0,"logical_decimal":"\u0000\u000f\u001e\u002d\u003c"} +"logical2" {"logical_timestamp":42000000,"logical_time":4200,"logical_date":4200,"logical_decimal":"\u0000\u000f\u001e\u002d\u003c\u004b\u005a"} +"logical3" {"logical_timestamp":1468275102000,"logical_time":16993,"logical_date":16993,"logical_decimal":"\u0000\u000f\u001e\u002d\u003c\u004b\u005a\u0069\u0078"} diff --git a/google-cloud-bigtable-kafka-connect-sink/src/test/resources/compatibility_test_cases/value_logicals/key-schema.json b/google-cloud-bigtable-kafka-connect-sink/src/test/resources/compatibility_test_cases/value_logicals/key-schema.json new file mode 100644 index 0000000000..4db187ebb3 --- /dev/null +++ b/google-cloud-bigtable-kafka-connect-sink/src/test/resources/compatibility_test_cases/value_logicals/key-schema.json @@ -0,0 +1 @@ +{"type":"string"} \ No newline at end of file diff --git a/google-cloud-bigtable-kafka-connect-sink/src/test/resources/compatibility_test_cases/value_logicals/value-schema.json b/google-cloud-bigtable-kafka-connect-sink/src/test/resources/compatibility_test_cases/value_logicals/value-schema.json new file mode 100644 index 0000000000..1e9fafaa00 --- /dev/null +++ b/google-cloud-bigtable-kafka-connect-sink/src/test/resources/compatibility_test_cases/value_logicals/value-schema.json @@ -0,0 +1,37 @@ +{ + "type": "record", + "name": "logicals", + "fields": [ + { + "name": "logical_timestamp", + "type": { + "type": "long", + "connect.name": "org.apache.kafka.connect.data.Timestamp" + } + }, + { + "name": "logical_time", + "type": { + "type": "int", + "connect.name": "org.apache.kafka.connect.data.Time" + } + }, + { + "name": "logical_date", + "type": { + "type": "int", + "connect.name": "org.apache.kafka.connect.data.Date" + } + }, + { + "name": "logical_decimal", + "type": { + "type": "bytes", + "connect.name": "org.apache.kafka.connect.data.Decimal", + "connect.parameters": { + "scale": "-50" + } + } + } + ] +} \ No newline at end of file diff --git a/google-cloud-bigtable-kafka-connect-sink/src/test/resources/compatibility_test_cases/value_matryoshkas/confluent_sink_output/matryoshka-dolls1/confluent_compat_topic/row b/google-cloud-bigtable-kafka-connect-sink/src/test/resources/compatibility_test_cases/value_matryoshkas/confluent_sink_output/matryoshka-dolls1/confluent_compat_topic/row deleted file mode 100644 index 720d64f4baafc33efdf971f02084aca5f25b34a5..0000000000000000000000000000000000000000 GIT binary patch literal 0 HcmV?d00001 literal 4 LcmZQzU|<9Q00jU7 diff --git a/google-cloud-bigtable-kafka-connect-sink/src/test/resources/compatibility_test_cases/value_matryoshkas/confluent_sink_output/matryoshka-dolls1/inner/inner_int b/google-cloud-bigtable-kafka-connect-sink/src/test/resources/compatibility_test_cases/value_matryoshkas/confluent_sink_output/matryoshka-dolls1/inner/inner_int deleted file mode 100644 index a7697b31d3..0000000000 --- a/google-cloud-bigtable-kafka-connect-sink/src/test/resources/compatibility_test_cases/value_matryoshkas/confluent_sink_output/matryoshka-dolls1/inner/inner_int +++ /dev/null @@ -1 +0,0 @@ -ÿÿÿÖ \ No newline at end of file diff --git a/google-cloud-bigtable-kafka-connect-sink/src/test/resources/compatibility_test_cases/value_matryoshkas/confluent_sink_output/matryoshka-dolls1/inner/inner_string b/google-cloud-bigtable-kafka-connect-sink/src/test/resources/compatibility_test_cases/value_matryoshkas/confluent_sink_output/matryoshka-dolls1/inner/inner_string deleted file mode 100644 index 67f7ad0566..0000000000 --- a/google-cloud-bigtable-kafka-connect-sink/src/test/resources/compatibility_test_cases/value_matryoshkas/confluent_sink_output/matryoshka-dolls1/inner/inner_string +++ /dev/null @@ -1 +0,0 @@ --42 \ No newline at end of file diff --git a/google-cloud-bigtable-kafka-connect-sink/src/test/resources/compatibility_test_cases/value_matryoshkas/confluent_sink_output/matryoshka-dolls1/middle/inner b/google-cloud-bigtable-kafka-connect-sink/src/test/resources/compatibility_test_cases/value_matryoshkas/confluent_sink_output/matryoshka-dolls1/middle/inner deleted file mode 100644 index 164adac427..0000000000 --- a/google-cloud-bigtable-kafka-connect-sink/src/test/resources/compatibility_test_cases/value_matryoshkas/confluent_sink_output/matryoshka-dolls1/middle/inner +++ /dev/null @@ -1 +0,0 @@ -{"inner_int":42,"inner_string":"42"} \ No newline at end of file diff --git a/google-cloud-bigtable-kafka-connect-sink/src/test/resources/compatibility_test_cases/value_matryoshkas/confluent_sink_output/matryoshka-dolls1/middle/middle_array b/google-cloud-bigtable-kafka-connect-sink/src/test/resources/compatibility_test_cases/value_matryoshkas/confluent_sink_output/matryoshka-dolls1/middle/middle_array deleted file mode 100644 index 12c84e81da..0000000000 --- a/google-cloud-bigtable-kafka-connect-sink/src/test/resources/compatibility_test_cases/value_matryoshkas/confluent_sink_output/matryoshka-dolls1/middle/middle_array +++ /dev/null @@ -1 +0,0 @@ -[42.0,42.42,42.4242] \ No newline at end of file diff --git a/google-cloud-bigtable-kafka-connect-sink/src/test/resources/compatibility_test_cases/value_nestedlogicals/confluent_sink_output/nestedlogicals1/outer/inner b/google-cloud-bigtable-kafka-connect-sink/src/test/resources/compatibility_test_cases/value_nestedlogicals/confluent_sink_output/nestedlogicals1/outer/inner deleted file mode 100644 index 9eae9a1459..0000000000 --- a/google-cloud-bigtable-kafka-connect-sink/src/test/resources/compatibility_test_cases/value_nestedlogicals/confluent_sink_output/nestedlogicals1/outer/inner +++ /dev/null @@ -1 +0,0 @@ -{"logical_timestamp":4242,"logical_time":4242,"logical_date":42,"logical_decimal":"Dx4tPEtaaXg="} \ No newline at end of file diff --git a/google-cloud-bigtable-kafka-connect-sink/src/test/resources/compatibility_test_cases/value_nulls/confluent_sink_output/nulls1/confluent_compat_topic/f1 b/google-cloud-bigtable-kafka-connect-sink/src/test/resources/compatibility_test_cases/value_nulls/confluent_sink_output/nulls1/confluent_compat_topic/f1 deleted file mode 100644 index 890469e582..0000000000 --- a/google-cloud-bigtable-kafka-connect-sink/src/test/resources/compatibility_test_cases/value_nulls/confluent_sink_output/nulls1/confluent_compat_topic/f1 +++ /dev/null @@ -1 +0,0 @@ -Required string \ No newline at end of file diff --git a/google-cloud-bigtable-kafka-connect-sink/src/test/resources/compatibility_test_cases/value_nulls/confluent_sink_output/nulls1/confluent_compat_topic/f2 b/google-cloud-bigtable-kafka-connect-sink/src/test/resources/compatibility_test_cases/value_nulls/confluent_sink_output/nulls1/confluent_compat_topic/f2 deleted file mode 100644 index e69de29bb2..0000000000 diff --git a/google-cloud-bigtable-kafka-connect-sink/src/test/resources/compatibility_test_cases/value_nulls/confluent_sink_output/nulls1/confluent_compat_topic/f3 b/google-cloud-bigtable-kafka-connect-sink/src/test/resources/compatibility_test_cases/value_nulls/confluent_sink_output/nulls1/confluent_compat_topic/f3 deleted file mode 100644 index 3fe959d6e15fb9443346556002dbd4a82afc72bf..0000000000000000000000000000000000000000 GIT binary patch literal 0 HcmV?d00001 literal 4 LcmZQzV9){p04@M4 diff --git a/google-cloud-bigtable-kafka-connect-sink/src/test/resources/compatibility_test_cases/value_nulls/confluent_sink_output/nulls1/confluent_compat_topic/f4 b/google-cloud-bigtable-kafka-connect-sink/src/test/resources/compatibility_test_cases/value_nulls/confluent_sink_output/nulls1/confluent_compat_topic/f4 deleted file mode 100644 index f76dd238ade08917e6712764a16a22005a50573d..0000000000000000000000000000000000000000 GIT binary patch literal 0 HcmV?d00001 literal 1 IcmZPo000310RR91 diff --git a/google-cloud-bigtable-kafka-connect-sink/src/test/resources/compatibility_test_cases/value_nulls/confluent_sink_output/nulls1/confluent_compat_topic/row b/google-cloud-bigtable-kafka-connect-sink/src/test/resources/compatibility_test_cases/value_nulls/confluent_sink_output/nulls1/confluent_compat_topic/row deleted file mode 100644 index 720d64f4baafc33efdf971f02084aca5f25b34a5..0000000000000000000000000000000000000000 GIT binary patch literal 0 HcmV?d00001 literal 4 LcmZQzU|<9Q00jU7 diff --git a/google-cloud-bigtable-kafka-connect-sink/src/test/resources/compatibility_test_cases/value_nulls/confluent_sink_output/nulls2/confluent_compat_topic/f1 b/google-cloud-bigtable-kafka-connect-sink/src/test/resources/compatibility_test_cases/value_nulls/confluent_sink_output/nulls2/confluent_compat_topic/f1 deleted file mode 100644 index 890469e582..0000000000 --- a/google-cloud-bigtable-kafka-connect-sink/src/test/resources/compatibility_test_cases/value_nulls/confluent_sink_output/nulls2/confluent_compat_topic/f1 +++ /dev/null @@ -1 +0,0 @@ -Required string \ No newline at end of file diff --git a/google-cloud-bigtable-kafka-connect-sink/src/test/resources/compatibility_test_cases/value_nulls/confluent_sink_output/nulls2/confluent_compat_topic/f2 b/google-cloud-bigtable-kafka-connect-sink/src/test/resources/compatibility_test_cases/value_nulls/confluent_sink_output/nulls2/confluent_compat_topic/f2 deleted file mode 100644 index 3d7d0e812e..0000000000 --- a/google-cloud-bigtable-kafka-connect-sink/src/test/resources/compatibility_test_cases/value_nulls/confluent_sink_output/nulls2/confluent_compat_topic/f2 +++ /dev/null @@ -1 +0,0 @@ -Optional string \ No newline at end of file diff --git a/google-cloud-bigtable-kafka-connect-sink/src/test/resources/compatibility_test_cases/value_nulls/confluent_sink_output/nulls2/confluent_compat_topic/f3 b/google-cloud-bigtable-kafka-connect-sink/src/test/resources/compatibility_test_cases/value_nulls/confluent_sink_output/nulls2/confluent_compat_topic/f3 deleted file mode 100644 index 189346e0e0b1279b4d5aed0ecdc6aa0062ee274a..0000000000000000000000000000000000000000 GIT binary patch literal 0 HcmV?d00001 literal 4 LcmZQzV2A_&09^oD diff --git a/google-cloud-bigtable-kafka-connect-sink/src/test/resources/compatibility_test_cases/value_nulls/confluent_sink_output/nulls2/confluent_compat_topic/f4 b/google-cloud-bigtable-kafka-connect-sink/src/test/resources/compatibility_test_cases/value_nulls/confluent_sink_output/nulls2/confluent_compat_topic/f4 deleted file mode 100644 index e69de29bb2..0000000000 diff --git a/google-cloud-bigtable-kafka-connect-sink/src/test/resources/compatibility_test_cases/value_nulls/confluent_sink_output/nulls2/confluent_compat_topic/row b/google-cloud-bigtable-kafka-connect-sink/src/test/resources/compatibility_test_cases/value_nulls/confluent_sink_output/nulls2/confluent_compat_topic/row deleted file mode 100644 index b6a8ef3e7ca7c398cd8f65bb1e21a23c0d251536..0000000000000000000000000000000000000000 GIT binary patch literal 0 HcmV?d00001 literal 4 LcmZQzU|<3O00sa9 diff --git a/google-cloud-bigtable-kafka-connect-sink/src/test/resources/compatibility_test_cases/value_nulls/confluent_sink_output/nulls3/confluent_compat_topic/f1 b/google-cloud-bigtable-kafka-connect-sink/src/test/resources/compatibility_test_cases/value_nulls/confluent_sink_output/nulls3/confluent_compat_topic/f1 deleted file mode 100644 index 890469e582..0000000000 --- a/google-cloud-bigtable-kafka-connect-sink/src/test/resources/compatibility_test_cases/value_nulls/confluent_sink_output/nulls3/confluent_compat_topic/f1 +++ /dev/null @@ -1 +0,0 @@ -Required string \ No newline at end of file diff --git a/google-cloud-bigtable-kafka-connect-sink/src/test/resources/compatibility_test_cases/value_nulls/confluent_sink_output/nulls3/confluent_compat_topic/f2 b/google-cloud-bigtable-kafka-connect-sink/src/test/resources/compatibility_test_cases/value_nulls/confluent_sink_output/nulls3/confluent_compat_topic/f2 deleted file mode 100644 index e69de29bb2..0000000000 diff --git a/google-cloud-bigtable-kafka-connect-sink/src/test/resources/compatibility_test_cases/value_nulls/confluent_sink_output/nulls3/confluent_compat_topic/f3 b/google-cloud-bigtable-kafka-connect-sink/src/test/resources/compatibility_test_cases/value_nulls/confluent_sink_output/nulls3/confluent_compat_topic/f3 deleted file mode 100644 index e69de29bb2..0000000000 diff --git a/google-cloud-bigtable-kafka-connect-sink/src/test/resources/compatibility_test_cases/value_nulls/confluent_sink_output/nulls3/confluent_compat_topic/f4 b/google-cloud-bigtable-kafka-connect-sink/src/test/resources/compatibility_test_cases/value_nulls/confluent_sink_output/nulls3/confluent_compat_topic/f4 deleted file mode 100644 index ce542efaa5..0000000000 --- a/google-cloud-bigtable-kafka-connect-sink/src/test/resources/compatibility_test_cases/value_nulls/confluent_sink_output/nulls3/confluent_compat_topic/f4 +++ /dev/null @@ -1 +0,0 @@ -ÿ \ No newline at end of file diff --git a/google-cloud-bigtable-kafka-connect-sink/src/test/resources/compatibility_test_cases/value_nulls/confluent_sink_output/nulls3/confluent_compat_topic/row b/google-cloud-bigtable-kafka-connect-sink/src/test/resources/compatibility_test_cases/value_nulls/confluent_sink_output/nulls3/confluent_compat_topic/row deleted file mode 100644 index fde1ac19d2b083530bcab4cb4fd2dcaa285234ab..0000000000000000000000000000000000000000 GIT binary patch literal 0 HcmV?d00001 literal 4 LcmZQzU| Date: Tue, 25 Feb 2025 13:31:08 +0100 Subject: [PATCH 50/76] Warn against latency unpredictability when using table and/or column family auto creation --- .../bigtable/config/BigtableSinkConfig.java | 15 +++++++++++---- 1 file changed, 11 insertions(+), 4 deletions(-) diff --git a/google-cloud-bigtable-kafka-connect-sink/src/main/java/com/google/cloud/kafka/connect/bigtable/config/BigtableSinkConfig.java b/google-cloud-bigtable-kafka-connect-sink/src/main/java/com/google/cloud/kafka/connect/bigtable/config/BigtableSinkConfig.java index b05acc6f1f..75628296cd 100644 --- a/google-cloud-bigtable-kafka-connect-sink/src/main/java/com/google/cloud/kafka/connect/bigtable/config/BigtableSinkConfig.java +++ b/google-cloud-bigtable-kafka-connect-sink/src/main/java/com/google/cloud/kafka/connect/bigtable/config/BigtableSinkConfig.java @@ -338,9 +338,12 @@ public static ConfigDef getDefinition() { false, new ConfigDef.NonNullValidator(), ConfigDef.Importance.MEDIUM, - "Whether to automatically create the destination table if it is found to be missing." - + "\nWhen enabled, the records for which the auto-creation fails, are failed." - + "\nRecreation of tables deleted by other Cloud Bigtable users is not supported.") + "Whether to automatically create the destination table if it is found to be missing.\n" + + "When enabled, the records for which the auto-creation fails, are failed.\n" + + "Recreation of tables deleted by other Cloud Bigtable users is not supported.\n" + + "Note that table auto-creation is slow (multiple seconds). It may slow down not" + + " only the records targeting nonexistent tables, but also other records batched" + + " with them. To facilitate predictable latency leave this option disabled.") .define( AUTO_CREATE_COLUMN_FAMILIES_CONFIG, ConfigDef.Type.BOOLEAN, @@ -354,7 +357,11 @@ public static ConfigDef getDefinition() { + "When enabled, column families will be created also for deletions of nonexistent" + " column families and cells within them.\n" + "Recreation of column families deleted by other Cloud Bigtable users is not" - + " supported.") + + " supported.\n" + + "Note that column family auto-creation is slow. It may slow down" + + " not only the records targeting nonexistent column families, but also other" + + " records batched with them. To facilitate predictable latency leave this option" + + " disabled.") .define( DEFAULT_COLUMN_FAMILY_CONFIG, ConfigDef.Type.STRING, From 751a13ef2bafa3272c577766ba2d7325dfbf61b1 Mon Sep 17 00:00:00 2001 From: Adam Czajkowski Date: Tue, 25 Feb 2025 13:32:43 +0100 Subject: [PATCH 51/76] Download and normalize strimzi 0.45.0 with namespace 'kafka' configured curl "https://strimzi.io/install/latest?namespace=kafka" | yq -P 'sort_keys(..)' | sponge strimzi-cluster-operator-0.45.0.yaml --- .../strimzi-cluster-operator-0.45.0.yaml | 20285 ++++++++++++++++ 1 file changed, 20285 insertions(+) create mode 100644 google-cloud-bigtable-kafka-connect-sink/performance/kubernetes/strimzi-cluster-operator-0.45.0.yaml diff --git a/google-cloud-bigtable-kafka-connect-sink/performance/kubernetes/strimzi-cluster-operator-0.45.0.yaml b/google-cloud-bigtable-kafka-connect-sink/performance/kubernetes/strimzi-cluster-operator-0.45.0.yaml new file mode 100644 index 0000000000..041020365b --- /dev/null +++ b/google-cloud-bigtable-kafka-connect-sink/performance/kubernetes/strimzi-cluster-operator-0.45.0.yaml @@ -0,0 +1,20285 @@ + +--- +apiVersion: apiextensions.k8s.io/v1 +kind: CustomResourceDefinition +metadata: + labels: + app: strimzi + strimzi.io/crd-install: "true" + name: kafkamirrormaker2s.kafka.strimzi.io +spec: + conversion: + strategy: None + group: kafka.strimzi.io + names: + categories: + - strimzi + kind: KafkaMirrorMaker2 + listKind: KafkaMirrorMaker2List + plural: kafkamirrormaker2s + shortNames: + - kmm2 + singular: kafkamirrormaker2 + scope: Namespaced + versions: + - additionalPrinterColumns: + - description: The desired number of Kafka MirrorMaker 2 replicas + jsonPath: .spec.replicas + name: Desired replicas + type: integer + - description: The state of the custom resource + jsonPath: .status.conditions[?(@.type=="Ready")].status + name: Ready + type: string + name: v1beta2 + schema: + openAPIV3Schema: + properties: + apiVersion: + description: 'APIVersion defines the versioned schema of this representation of an object. Servers should convert recognized schemas to the latest internal value, and may reject unrecognized values. More info: https://git.k8s.io/community/contributors/devel/sig-architecture/api-conventions.md#resources' + type: string + kind: + description: 'Kind is a string value representing the REST resource this object represents. Servers may infer this from the endpoint the client submits requests to. Cannot be updated. In CamelCase. More info: https://git.k8s.io/community/contributors/devel/sig-architecture/api-conventions.md#types-kinds' + type: string + metadata: + type: object + spec: + description: The specification of the Kafka MirrorMaker 2 cluster. + properties: + clientRackInitImage: + description: The image of the init container used for initializing the `client.rack`. + type: string + clusters: + description: Kafka clusters for mirroring. + items: + properties: + alias: + description: Alias used to reference the Kafka cluster. + pattern: ^[a-zA-Z0-9\._\-]{1,100}$ + type: string + authentication: + description: Authentication configuration for connecting to the cluster. + properties: + accessToken: + description: Link to Kubernetes Secret containing the access token which was obtained from the authorization server. + properties: + key: + description: The key under which the secret value is stored in the Kubernetes Secret. + type: string + secretName: + description: The name of the Kubernetes Secret containing the secret value. + type: string + required: + - key + - secretName + type: object + accessTokenIsJwt: + description: Configure whether access token should be treated as JWT. This should be set to `false` if the authorization server returns opaque tokens. Defaults to `true`. + type: boolean + accessTokenLocation: + description: Path to the token file containing an access token to be used for authentication. + type: string + audience: + description: OAuth audience to use when authenticating against the authorization server. Some authorization servers require the audience to be explicitly set. The possible values depend on how the authorization server is configured. By default, `audience` is not specified when performing the token endpoint request. + type: string + certificateAndKey: + description: Reference to the `Secret` which holds the certificate and private key pair. + properties: + certificate: + description: The name of the file certificate in the Secret. + type: string + key: + description: The name of the private key in the Secret. + type: string + secretName: + description: The name of the Secret containing the certificate. + type: string + required: + - secretName + - certificate + - key + type: object + clientAssertion: + description: Link to Kubernetes secret containing the client assertion which was manually configured for the client. + properties: + key: + description: The key under which the secret value is stored in the Kubernetes Secret. + type: string + secretName: + description: The name of the Kubernetes Secret containing the secret value. + type: string + required: + - key + - secretName + type: object + clientAssertionLocation: + description: Path to the file containing the client assertion to be used for authentication. + type: string + clientAssertionType: + description: The client assertion type. If not set, and either `clientAssertion` or `clientAssertionLocation` is configured, this value defaults to `urn:ietf:params:oauth:client-assertion-type:jwt-bearer`. + type: string + clientId: + description: OAuth Client ID which the Kafka client can use to authenticate against the OAuth server and use the token endpoint URI. + type: string + clientSecret: + description: Link to Kubernetes Secret containing the OAuth client secret which the Kafka client can use to authenticate against the OAuth server and use the token endpoint URI. + properties: + key: + description: The key under which the secret value is stored in the Kubernetes Secret. + type: string + secretName: + description: The name of the Kubernetes Secret containing the secret value. + type: string + required: + - key + - secretName + type: object + connectTimeoutSeconds: + description: The connect timeout in seconds when connecting to authorization server. If not set, the effective connect timeout is 60 seconds. + type: integer + disableTlsHostnameVerification: + description: Enable or disable TLS hostname verification. Default value is `false`. + type: boolean + enableMetrics: + description: Enable or disable OAuth metrics. Default value is `false`. + type: boolean + httpRetries: + description: The maximum number of retries to attempt if an initial HTTP request fails. If not set, the default is to not attempt any retries. + type: integer + httpRetryPauseMs: + description: The pause to take before retrying a failed HTTP request. If not set, the default is to not pause at all but to immediately repeat a request. + type: integer + includeAcceptHeader: + description: Whether the Accept header should be set in requests to the authorization servers. The default value is `true`. + type: boolean + maxTokenExpirySeconds: + description: Set or limit time-to-live of the access tokens to the specified number of seconds. This should be set if the authorization server returns opaque tokens. + type: integer + passwordSecret: + description: Reference to the `Secret` which holds the password. + properties: + password: + description: The name of the key in the Secret under which the password is stored. + type: string + secretName: + description: The name of the Secret containing the password. + type: string + required: + - secretName + - password + type: object + readTimeoutSeconds: + description: The read timeout in seconds when connecting to authorization server. If not set, the effective read timeout is 60 seconds. + type: integer + refreshToken: + description: Link to Kubernetes Secret containing the refresh token which can be used to obtain access token from the authorization server. + properties: + key: + description: The key under which the secret value is stored in the Kubernetes Secret. + type: string + secretName: + description: The name of the Kubernetes Secret containing the secret value. + type: string + required: + - key + - secretName + type: object + saslExtensions: + additionalProperties: + type: string + description: SASL extensions parameters. + type: object + scope: + description: OAuth scope to use when authenticating against the authorization server. Some authorization servers require this to be set. The possible values depend on how authorization server is configured. By default `scope` is not specified when doing the token endpoint request. + type: string + tlsTrustedCertificates: + description: Trusted certificates for TLS connection to the OAuth server. + items: + oneOf: + - properties: + certificate: {} + required: + - certificate + - properties: + pattern: {} + required: + - pattern + properties: + certificate: + description: The name of the file certificate in the secret. + type: string + pattern: + description: Pattern for the certificate files in the secret. Use the link:https://en.wikipedia.org/wiki/Glob_(programming)[_glob syntax_] for the pattern. All files in the secret that match the pattern are used. + type: string + secretName: + description: The name of the Secret containing the certificate. + type: string + required: + - secretName + type: object + type: array + tokenEndpointUri: + description: Authorization server token endpoint URI. + type: string + type: + description: Authentication type. Currently the supported types are `tls`, `scram-sha-256`, `scram-sha-512`, `plain`, and 'oauth'. `scram-sha-256` and `scram-sha-512` types use SASL SCRAM-SHA-256 and SASL SCRAM-SHA-512 Authentication, respectively. `plain` type uses SASL PLAIN Authentication. `oauth` type uses SASL OAUTHBEARER Authentication. The `tls` type uses TLS Client Authentication. The `tls` type is supported only over TLS connections. + enum: + - tls + - scram-sha-256 + - scram-sha-512 + - plain + - oauth + type: string + username: + description: Username used for the authentication. + type: string + required: + - type + type: object + bootstrapServers: + description: A comma-separated list of `host:port` pairs for establishing the connection to the Kafka cluster. + type: string + config: + description: 'The MirrorMaker 2 cluster config. Properties with the following prefixes cannot be set: ssl., sasl., security., listeners, plugin.path, rest., bootstrap.servers, consumer.interceptor.classes, producer.interceptor.classes (with the exception of: ssl.endpoint.identification.algorithm, ssl.cipher.suites, ssl.protocol, ssl.enabled.protocols).' + type: object + x-kubernetes-preserve-unknown-fields: true + tls: + description: TLS configuration for connecting MirrorMaker 2 connectors to a cluster. + properties: + trustedCertificates: + description: Trusted certificates for TLS connection. + items: + oneOf: + - properties: + certificate: {} + required: + - certificate + - properties: + pattern: {} + required: + - pattern + properties: + certificate: + description: The name of the file certificate in the secret. + type: string + pattern: + description: Pattern for the certificate files in the secret. Use the link:https://en.wikipedia.org/wiki/Glob_(programming)[_glob syntax_] for the pattern. All files in the secret that match the pattern are used. + type: string + secretName: + description: The name of the Secret containing the certificate. + type: string + required: + - secretName + type: object + type: array + type: object + required: + - alias + - bootstrapServers + type: object + type: array + connectCluster: + description: The cluster alias used for Kafka Connect. The value must match the alias of the *target* Kafka cluster as specified in the `spec.clusters` configuration. The target Kafka cluster is used by the underlying Kafka Connect framework for its internal topics. + type: string + externalConfiguration: + description: Pass data from Secrets or ConfigMaps to the Kafka Connect pods and use them to configure connectors. + properties: + env: + description: Makes data from a Secret or ConfigMap available in the Kafka Connect pods as environment variables. + items: + properties: + name: + description: Name of the environment variable which will be passed to the Kafka Connect pods. The name of the environment variable cannot start with `KAFKA_` or `STRIMZI_`. + type: string + valueFrom: + description: Value of the environment variable which will be passed to the Kafka Connect pods. It can be passed either as a reference to Secret or ConfigMap field. The field has to specify exactly one Secret or ConfigMap. + oneOf: + - properties: + secretKeyRef: {} + required: + - secretKeyRef + - properties: + configMapKeyRef: {} + required: + - configMapKeyRef + properties: + configMapKeyRef: + description: Reference to a key in a ConfigMap. + properties: + key: + type: string + name: + type: string + optional: + type: boolean + type: object + secretKeyRef: + description: Reference to a key in a Secret. + properties: + key: + type: string + name: + type: string + optional: + type: boolean + type: object + type: object + required: + - name + - valueFrom + type: object + type: array + volumes: + description: Makes data from a Secret or ConfigMap available in the Kafka Connect pods as volumes. + items: + oneOf: + - properties: + secret: {} + required: + - secret + - properties: + configMap: {} + required: + - configMap + properties: + configMap: + description: Reference to a key in a ConfigMap. Exactly one Secret or ConfigMap has to be specified. + properties: + defaultMode: + type: integer + items: + items: + properties: + key: + type: string + mode: + type: integer + path: + type: string + type: object + type: array + name: + type: string + optional: + type: boolean + type: object + name: + description: Name of the volume which will be added to the Kafka Connect pods. + type: string + secret: + description: Reference to a key in a Secret. Exactly one Secret or ConfigMap has to be specified. + properties: + defaultMode: + type: integer + items: + items: + properties: + key: + type: string + mode: + type: integer + path: + type: string + type: object + type: array + optional: + type: boolean + secretName: + type: string + type: object + required: + - name + type: object + type: array + type: object + image: + description: The container image used for Kafka Connect pods. If no image name is explicitly specified, it is determined based on the `spec.version` configuration. The image names are specifically mapped to corresponding versions in the Cluster Operator configuration. + type: string + jmxOptions: + description: JMX Options. + properties: + authentication: + description: Authentication configuration for connecting to the JMX port. + properties: + type: + description: Authentication type. Currently the only supported types are `password`.`password` type creates a username and protected port with no TLS. + enum: + - password + type: string + required: + - type + type: object + type: object + jvmOptions: + description: JVM Options for pods. + properties: + -XX: + additionalProperties: + type: string + description: A map of -XX options to the JVM. + type: object + -Xms: + description: -Xms option to to the JVM. + pattern: ^[0-9]+[mMgG]?$ + type: string + -Xmx: + description: -Xmx option to to the JVM. + pattern: ^[0-9]+[mMgG]?$ + type: string + gcLoggingEnabled: + description: Specifies whether the Garbage Collection logging is enabled. The default is false. + type: boolean + javaSystemProperties: + description: A map of additional system properties which will be passed using the `-D` option to the JVM. + items: + properties: + name: + description: The system property name. + type: string + value: + description: The system property value. + type: string + type: object + type: array + type: object + livenessProbe: + description: Pod liveness checking. + properties: + failureThreshold: + description: Minimum consecutive failures for the probe to be considered failed after having succeeded. Defaults to 3. Minimum value is 1. + minimum: 1 + type: integer + initialDelaySeconds: + description: The initial delay before first the health is first checked. Default to 15 seconds. Minimum value is 0. + minimum: 0 + type: integer + periodSeconds: + description: How often (in seconds) to perform the probe. Default to 10 seconds. Minimum value is 1. + minimum: 1 + type: integer + successThreshold: + description: Minimum consecutive successes for the probe to be considered successful after having failed. Defaults to 1. Must be 1 for liveness. Minimum value is 1. + minimum: 1 + type: integer + timeoutSeconds: + description: The timeout for each attempted health check. Default to 5 seconds. Minimum value is 1. + minimum: 1 + type: integer + type: object + logging: + description: Logging configuration for Kafka Connect. + properties: + loggers: + additionalProperties: + type: string + description: A Map from logger name to logger level. + type: object + type: + description: Logging type, must be either 'inline' or 'external'. + enum: + - inline + - external + type: string + valueFrom: + description: '`ConfigMap` entry where the logging configuration is stored. ' + properties: + configMapKeyRef: + description: Reference to the key in the ConfigMap containing the configuration. + properties: + key: + type: string + name: + type: string + optional: + type: boolean + type: object + type: object + required: + - type + type: object + metricsConfig: + description: Metrics configuration. + properties: + type: + description: Metrics type. Only 'jmxPrometheusExporter' supported currently. + enum: + - jmxPrometheusExporter + type: string + valueFrom: + description: 'ConfigMap entry where the Prometheus JMX Exporter configuration is stored. ' + properties: + configMapKeyRef: + description: Reference to the key in the ConfigMap containing the configuration. + properties: + key: + type: string + name: + type: string + optional: + type: boolean + type: object + type: object + required: + - type + - valueFrom + type: object + mirrors: + description: Configuration of the MirrorMaker 2 connectors. + items: + properties: + checkpointConnector: + description: The specification of the Kafka MirrorMaker 2 checkpoint connector. + properties: + alterOffsets: + description: Configuration for altering offsets. + properties: + fromConfigMap: + description: Reference to the ConfigMap where the new offsets are stored. + properties: + name: + type: string + type: object + required: + - fromConfigMap + type: object + autoRestart: + description: Automatic restart of connector and tasks configuration. + properties: + enabled: + description: Whether automatic restart for failed connectors and tasks should be enabled or disabled. + type: boolean + maxRestarts: + description: The maximum number of connector restarts that the operator will try. If the connector remains in a failed state after reaching this limit, it must be restarted manually by the user. Defaults to an unlimited number of restarts. + type: integer + type: object + config: + description: 'The Kafka Connector configuration. The following properties cannot be set: name, connector.class, tasks.max.' + type: object + x-kubernetes-preserve-unknown-fields: true + listOffsets: + description: Configuration for listing offsets. + properties: + toConfigMap: + description: Reference to the ConfigMap where the list of offsets will be written to. + properties: + name: + type: string + type: object + required: + - toConfigMap + type: object + pause: + description: Whether the connector should be paused. Defaults to false. + type: boolean + state: + description: The state the connector should be in. Defaults to running. + enum: + - paused + - stopped + - running + type: string + tasksMax: + description: The maximum number of tasks for the Kafka Connector. + minimum: 1 + type: integer + type: object + groupsBlacklistPattern: + description: A regular expression matching the consumer groups to exclude from mirroring. Comma-separated lists are also supported. + type: string + groupsExcludePattern: + description: A regular expression matching the consumer groups to exclude from mirroring. Comma-separated lists are also supported. + type: string + groupsPattern: + description: A regular expression matching the consumer groups to be mirrored. Comma-separated lists are also supported. + type: string + heartbeatConnector: + description: The specification of the Kafka MirrorMaker 2 heartbeat connector. + properties: + alterOffsets: + description: Configuration for altering offsets. + properties: + fromConfigMap: + description: Reference to the ConfigMap where the new offsets are stored. + properties: + name: + type: string + type: object + required: + - fromConfigMap + type: object + autoRestart: + description: Automatic restart of connector and tasks configuration. + properties: + enabled: + description: Whether automatic restart for failed connectors and tasks should be enabled or disabled. + type: boolean + maxRestarts: + description: The maximum number of connector restarts that the operator will try. If the connector remains in a failed state after reaching this limit, it must be restarted manually by the user. Defaults to an unlimited number of restarts. + type: integer + type: object + config: + description: 'The Kafka Connector configuration. The following properties cannot be set: name, connector.class, tasks.max.' + type: object + x-kubernetes-preserve-unknown-fields: true + listOffsets: + description: Configuration for listing offsets. + properties: + toConfigMap: + description: Reference to the ConfigMap where the list of offsets will be written to. + properties: + name: + type: string + type: object + required: + - toConfigMap + type: object + pause: + description: Whether the connector should be paused. Defaults to false. + type: boolean + state: + description: The state the connector should be in. Defaults to running. + enum: + - paused + - stopped + - running + type: string + tasksMax: + description: The maximum number of tasks for the Kafka Connector. + minimum: 1 + type: integer + type: object + sourceCluster: + description: The alias of the source cluster used by the Kafka MirrorMaker 2 connectors. The alias must match a cluster in the list at `spec.clusters`. + type: string + sourceConnector: + description: The specification of the Kafka MirrorMaker 2 source connector. + properties: + alterOffsets: + description: Configuration for altering offsets. + properties: + fromConfigMap: + description: Reference to the ConfigMap where the new offsets are stored. + properties: + name: + type: string + type: object + required: + - fromConfigMap + type: object + autoRestart: + description: Automatic restart of connector and tasks configuration. + properties: + enabled: + description: Whether automatic restart for failed connectors and tasks should be enabled or disabled. + type: boolean + maxRestarts: + description: The maximum number of connector restarts that the operator will try. If the connector remains in a failed state after reaching this limit, it must be restarted manually by the user. Defaults to an unlimited number of restarts. + type: integer + type: object + config: + description: 'The Kafka Connector configuration. The following properties cannot be set: name, connector.class, tasks.max.' + type: object + x-kubernetes-preserve-unknown-fields: true + listOffsets: + description: Configuration for listing offsets. + properties: + toConfigMap: + description: Reference to the ConfigMap where the list of offsets will be written to. + properties: + name: + type: string + type: object + required: + - toConfigMap + type: object + pause: + description: Whether the connector should be paused. Defaults to false. + type: boolean + state: + description: The state the connector should be in. Defaults to running. + enum: + - paused + - stopped + - running + type: string + tasksMax: + description: The maximum number of tasks for the Kafka Connector. + minimum: 1 + type: integer + type: object + targetCluster: + description: The alias of the target cluster used by the Kafka MirrorMaker 2 connectors. The alias must match a cluster in the list at `spec.clusters`. + type: string + topicsBlacklistPattern: + description: A regular expression matching the topics to exclude from mirroring. Comma-separated lists are also supported. + type: string + topicsExcludePattern: + description: A regular expression matching the topics to exclude from mirroring. Comma-separated lists are also supported. + type: string + topicsPattern: + description: A regular expression matching the topics to be mirrored, for example, "topic1\|topic2\|topic3". Comma-separated lists are also supported. + type: string + required: + - sourceCluster + - targetCluster + type: object + type: array + rack: + description: Configuration of the node label which will be used as the `client.rack` consumer configuration. + properties: + topologyKey: + description: A key that matches labels assigned to the Kubernetes cluster nodes. The value of the label is used to set a broker's `broker.rack` config, and the `client.rack` config for Kafka Connect or MirrorMaker 2. + example: topology.kubernetes.io/zone + type: string + required: + - topologyKey + type: object + readinessProbe: + description: Pod readiness checking. + properties: + failureThreshold: + description: Minimum consecutive failures for the probe to be considered failed after having succeeded. Defaults to 3. Minimum value is 1. + minimum: 1 + type: integer + initialDelaySeconds: + description: The initial delay before first the health is first checked. Default to 15 seconds. Minimum value is 0. + minimum: 0 + type: integer + periodSeconds: + description: How often (in seconds) to perform the probe. Default to 10 seconds. Minimum value is 1. + minimum: 1 + type: integer + successThreshold: + description: Minimum consecutive successes for the probe to be considered successful after having failed. Defaults to 1. Must be 1 for liveness. Minimum value is 1. + minimum: 1 + type: integer + timeoutSeconds: + description: The timeout for each attempted health check. Default to 5 seconds. Minimum value is 1. + minimum: 1 + type: integer + type: object + replicas: + description: The number of pods in the Kafka Connect group. Defaults to `3`. + type: integer + resources: + description: The maximum limits for CPU and memory resources and the requested initial resources. + properties: + claims: + items: + properties: + name: + type: string + type: object + type: array + limits: + additionalProperties: + anyOf: + - type: integer + - type: string + pattern: ^(\+|-)?(([0-9]+(\.[0-9]*)?)|(\.[0-9]+))(([KMGTPE]i)|[numkMGTPE]|([eE](\+|-)?(([0-9]+(\.[0-9]*)?)|(\.[0-9]+))))?$ + x-kubernetes-int-or-string: true + type: object + requests: + additionalProperties: + anyOf: + - type: integer + - type: string + pattern: ^(\+|-)?(([0-9]+(\.[0-9]*)?)|(\.[0-9]+))(([KMGTPE]i)|[numkMGTPE]|([eE](\+|-)?(([0-9]+(\.[0-9]*)?)|(\.[0-9]+))))?$ + x-kubernetes-int-or-string: true + type: object + type: object + template: + description: Template for Kafka Connect and Kafka MirrorMaker 2 resources. The template allows users to specify how the `Pods`, `Service`, and other services are generated. + properties: + apiService: + description: Template for Kafka Connect API `Service`. + properties: + ipFamilies: + description: Specifies the IP Families used by the service. Available options are `IPv4` and `IPv6`. If unspecified, Kubernetes will choose the default value based on the `ipFamilyPolicy` setting. + items: + enum: + - IPv4 + - IPv6 + type: string + type: array + ipFamilyPolicy: + description: Specifies the IP Family Policy used by the service. Available options are `SingleStack`, `PreferDualStack` and `RequireDualStack`. `SingleStack` is for a single IP family. `PreferDualStack` is for two IP families on dual-stack configured clusters or a single IP family on single-stack clusters. `RequireDualStack` fails unless there are two IP families on dual-stack configured clusters. If unspecified, Kubernetes will choose the default value based on the service type. + enum: + - SingleStack + - PreferDualStack + - RequireDualStack + type: string + metadata: + description: Metadata applied to the resource. + properties: + annotations: + additionalProperties: + type: string + description: Annotations added to the Kubernetes resource. + type: object + labels: + additionalProperties: + type: string + description: Labels added to the Kubernetes resource. + type: object + type: object + type: object + buildConfig: + description: Template for the Kafka Connect BuildConfig used to build new container images. The BuildConfig is used only on OpenShift. + properties: + metadata: + description: Metadata to apply to the `PodDisruptionBudgetTemplate` resource. + properties: + annotations: + additionalProperties: + type: string + description: Annotations added to the Kubernetes resource. + type: object + labels: + additionalProperties: + type: string + description: Labels added to the Kubernetes resource. + type: object + type: object + pullSecret: + description: Container Registry Secret with the credentials for pulling the base image. + type: string + type: object + buildContainer: + description: Template for the Kafka Connect Build container. The build container is used only on Kubernetes. + properties: + env: + description: Environment variables which should be applied to the container. + items: + oneOf: + - properties: + value: {} + required: + - value + - properties: + valueFrom: {} + required: + - valueFrom + properties: + name: + description: The environment variable key. + type: string + value: + description: The environment variable value. + type: string + valueFrom: + description: Reference to the secret or config map property to which the environment variable is set. + oneOf: + - properties: + secretKeyRef: {} + required: + - secretKeyRef + - properties: + configMapKeyRef: {} + required: + - configMapKeyRef + properties: + configMapKeyRef: + description: Reference to a key in a config map. + properties: + key: + type: string + name: + type: string + optional: + type: boolean + type: object + secretKeyRef: + description: Reference to a key in a secret. + properties: + key: + type: string + name: + type: string + optional: + type: boolean + type: object + type: object + type: object + type: array + securityContext: + description: Security context for the container. + properties: + allowPrivilegeEscalation: + type: boolean + appArmorProfile: + properties: + localhostProfile: + type: string + type: + type: string + type: object + capabilities: + properties: + add: + items: + type: string + type: array + drop: + items: + type: string + type: array + type: object + privileged: + type: boolean + procMount: + type: string + readOnlyRootFilesystem: + type: boolean + runAsGroup: + type: integer + runAsNonRoot: + type: boolean + runAsUser: + type: integer + seLinuxOptions: + properties: + level: + type: string + role: + type: string + type: + type: string + user: + type: string + type: object + seccompProfile: + properties: + localhostProfile: + type: string + type: + type: string + type: object + windowsOptions: + properties: + gmsaCredentialSpec: + type: string + gmsaCredentialSpecName: + type: string + hostProcess: + type: boolean + runAsUserName: + type: string + type: object + type: object + volumeMounts: + description: Additional volume mounts which should be applied to the container. + items: + properties: + mountPath: + type: string + mountPropagation: + type: string + name: + type: string + readOnly: + type: boolean + recursiveReadOnly: + type: string + subPath: + type: string + subPathExpr: + type: string + type: object + type: array + type: object + buildPod: + description: Template for Kafka Connect Build `Pods`. The build pod is used only on Kubernetes. + properties: + affinity: + description: The pod's affinity rules. + properties: + nodeAffinity: + properties: + preferredDuringSchedulingIgnoredDuringExecution: + items: + properties: + preference: + properties: + matchExpressions: + items: + properties: + key: + type: string + operator: + type: string + values: + items: + type: string + type: array + type: object + type: array + matchFields: + items: + properties: + key: + type: string + operator: + type: string + values: + items: + type: string + type: array + type: object + type: array + type: object + weight: + type: integer + type: object + type: array + requiredDuringSchedulingIgnoredDuringExecution: + properties: + nodeSelectorTerms: + items: + properties: + matchExpressions: + items: + properties: + key: + type: string + operator: + type: string + values: + items: + type: string + type: array + type: object + type: array + matchFields: + items: + properties: + key: + type: string + operator: + type: string + values: + items: + type: string + type: array + type: object + type: array + type: object + type: array + type: object + type: object + podAffinity: + properties: + preferredDuringSchedulingIgnoredDuringExecution: + items: + properties: + podAffinityTerm: + properties: + labelSelector: + properties: + matchExpressions: + items: + properties: + key: + type: string + operator: + type: string + values: + items: + type: string + type: array + type: object + type: array + matchLabels: + additionalProperties: + type: string + type: object + type: object + matchLabelKeys: + items: + type: string + type: array + mismatchLabelKeys: + items: + type: string + type: array + namespaceSelector: + properties: + matchExpressions: + items: + properties: + key: + type: string + operator: + type: string + values: + items: + type: string + type: array + type: object + type: array + matchLabels: + additionalProperties: + type: string + type: object + type: object + namespaces: + items: + type: string + type: array + topologyKey: + type: string + type: object + weight: + type: integer + type: object + type: array + requiredDuringSchedulingIgnoredDuringExecution: + items: + properties: + labelSelector: + properties: + matchExpressions: + items: + properties: + key: + type: string + operator: + type: string + values: + items: + type: string + type: array + type: object + type: array + matchLabels: + additionalProperties: + type: string + type: object + type: object + matchLabelKeys: + items: + type: string + type: array + mismatchLabelKeys: + items: + type: string + type: array + namespaceSelector: + properties: + matchExpressions: + items: + properties: + key: + type: string + operator: + type: string + values: + items: + type: string + type: array + type: object + type: array + matchLabels: + additionalProperties: + type: string + type: object + type: object + namespaces: + items: + type: string + type: array + topologyKey: + type: string + type: object + type: array + type: object + podAntiAffinity: + properties: + preferredDuringSchedulingIgnoredDuringExecution: + items: + properties: + podAffinityTerm: + properties: + labelSelector: + properties: + matchExpressions: + items: + properties: + key: + type: string + operator: + type: string + values: + items: + type: string + type: array + type: object + type: array + matchLabels: + additionalProperties: + type: string + type: object + type: object + matchLabelKeys: + items: + type: string + type: array + mismatchLabelKeys: + items: + type: string + type: array + namespaceSelector: + properties: + matchExpressions: + items: + properties: + key: + type: string + operator: + type: string + values: + items: + type: string + type: array + type: object + type: array + matchLabels: + additionalProperties: + type: string + type: object + type: object + namespaces: + items: + type: string + type: array + topologyKey: + type: string + type: object + weight: + type: integer + type: object + type: array + requiredDuringSchedulingIgnoredDuringExecution: + items: + properties: + labelSelector: + properties: + matchExpressions: + items: + properties: + key: + type: string + operator: + type: string + values: + items: + type: string + type: array + type: object + type: array + matchLabels: + additionalProperties: + type: string + type: object + type: object + matchLabelKeys: + items: + type: string + type: array + mismatchLabelKeys: + items: + type: string + type: array + namespaceSelector: + properties: + matchExpressions: + items: + properties: + key: + type: string + operator: + type: string + values: + items: + type: string + type: array + type: object + type: array + matchLabels: + additionalProperties: + type: string + type: object + type: object + namespaces: + items: + type: string + type: array + topologyKey: + type: string + type: object + type: array + type: object + type: object + enableServiceLinks: + description: Indicates whether information about services should be injected into Pod's environment variables. + type: boolean + hostAliases: + description: The pod's HostAliases. HostAliases is an optional list of hosts and IPs that will be injected into the Pod's hosts file if specified. + items: + properties: + hostnames: + items: + type: string + type: array + ip: + type: string + type: object + type: array + imagePullSecrets: + description: List of references to secrets in the same namespace to use for pulling any of the images used by this Pod. When the `STRIMZI_IMAGE_PULL_SECRETS` environment variable in Cluster Operator and the `imagePullSecrets` option are specified, only the `imagePullSecrets` variable is used and the `STRIMZI_IMAGE_PULL_SECRETS` variable is ignored. + items: + properties: + name: + type: string + type: object + type: array + metadata: + description: Metadata applied to the resource. + properties: + annotations: + additionalProperties: + type: string + description: Annotations added to the Kubernetes resource. + type: object + labels: + additionalProperties: + type: string + description: Labels added to the Kubernetes resource. + type: object + type: object + priorityClassName: + description: 'The name of the priority class used to assign priority to the pods. ' + type: string + schedulerName: + description: The name of the scheduler used to dispatch this `Pod`. If not specified, the default scheduler will be used. + type: string + securityContext: + description: Configures pod-level security attributes and common container settings. + properties: + appArmorProfile: + properties: + localhostProfile: + type: string + type: + type: string + type: object + fsGroup: + type: integer + fsGroupChangePolicy: + type: string + runAsGroup: + type: integer + runAsNonRoot: + type: boolean + runAsUser: + type: integer + seLinuxOptions: + properties: + level: + type: string + role: + type: string + type: + type: string + user: + type: string + type: object + seccompProfile: + properties: + localhostProfile: + type: string + type: + type: string + type: object + supplementalGroups: + items: + type: integer + type: array + sysctls: + items: + properties: + name: + type: string + value: + type: string + type: object + type: array + windowsOptions: + properties: + gmsaCredentialSpec: + type: string + gmsaCredentialSpecName: + type: string + hostProcess: + type: boolean + runAsUserName: + type: string + type: object + type: object + terminationGracePeriodSeconds: + description: The grace period is the duration in seconds after the processes running in the pod are sent a termination signal, and the time when the processes are forcibly halted with a kill signal. Set this value to longer than the expected cleanup time for your process. Value must be a non-negative integer. A zero value indicates delete immediately. You might need to increase the grace period for very large Kafka clusters, so that the Kafka brokers have enough time to transfer their work to another broker before they are terminated. Defaults to 30 seconds. + minimum: 0 + type: integer + tmpDirSizeLimit: + description: Defines the total amount of pod memory allocated for the temporary `EmptyDir` volume `/tmp`. Specify the allocation in memory units, for example, `100Mi` for 100 mebibytes. Default value is `5Mi`. The `/tmp` volume is backed by pod memory, not disk storage, so avoid setting a high value as it consumes pod memory resources. + pattern: ^([0-9.]+)([eEinumkKMGTP]*[-+]?[0-9]*)$ + type: string + tolerations: + description: The pod's tolerations. + items: + properties: + effect: + type: string + key: + type: string + operator: + type: string + tolerationSeconds: + type: integer + value: + type: string + type: object + type: array + topologySpreadConstraints: + description: The pod's topology spread constraints. + items: + properties: + labelSelector: + properties: + matchExpressions: + items: + properties: + key: + type: string + operator: + type: string + values: + items: + type: string + type: array + type: object + type: array + matchLabels: + additionalProperties: + type: string + type: object + type: object + matchLabelKeys: + items: + type: string + type: array + maxSkew: + type: integer + minDomains: + type: integer + nodeAffinityPolicy: + type: string + nodeTaintsPolicy: + type: string + topologyKey: + type: string + whenUnsatisfiable: + type: string + type: object + type: array + volumes: + description: Additional volumes that can be mounted to the pod. + items: + oneOf: + - properties: + configMap: {} + csi: {} + emptyDir: {} + persistentVolumeClaim: {} + secret: {} + properties: + configMap: + description: ConfigMap to use to populate the volume. + properties: + defaultMode: + type: integer + items: + items: + properties: + key: + type: string + mode: + type: integer + path: + type: string + type: object + type: array + name: + type: string + optional: + type: boolean + type: object + csi: + description: CSIVolumeSource object to use to populate the volume. + properties: + driver: + type: string + fsType: + type: string + nodePublishSecretRef: + properties: + name: + type: string + type: object + readOnly: + type: boolean + volumeAttributes: + additionalProperties: + type: string + type: object + type: object + emptyDir: + description: EmptyDir to use to populate the volume. + properties: + medium: + type: string + sizeLimit: + properties: + amount: + type: string + format: + type: string + type: object + type: object + name: + description: Name to use for the volume. Required. + type: string + persistentVolumeClaim: + description: PersistentVolumeClaim object to use to populate the volume. + properties: + claimName: + type: string + readOnly: + type: boolean + type: object + secret: + description: Secret to use populate the volume. + properties: + defaultMode: + type: integer + items: + items: + properties: + key: + type: string + mode: + type: integer + path: + type: string + type: object + type: array + optional: + type: boolean + secretName: + type: string + type: object + type: object + type: array + type: object + buildServiceAccount: + description: Template for the Kafka Connect Build service account. + properties: + metadata: + description: Metadata applied to the resource. + properties: + annotations: + additionalProperties: + type: string + description: Annotations added to the Kubernetes resource. + type: object + labels: + additionalProperties: + type: string + description: Labels added to the Kubernetes resource. + type: object + type: object + type: object + clusterRoleBinding: + description: Template for the Kafka Connect ClusterRoleBinding. + properties: + metadata: + description: Metadata applied to the resource. + properties: + annotations: + additionalProperties: + type: string + description: Annotations added to the Kubernetes resource. + type: object + labels: + additionalProperties: + type: string + description: Labels added to the Kubernetes resource. + type: object + type: object + type: object + connectContainer: + description: Template for the Kafka Connect container. + properties: + env: + description: Environment variables which should be applied to the container. + items: + oneOf: + - properties: + value: {} + required: + - value + - properties: + valueFrom: {} + required: + - valueFrom + properties: + name: + description: The environment variable key. + type: string + value: + description: The environment variable value. + type: string + valueFrom: + description: Reference to the secret or config map property to which the environment variable is set. + oneOf: + - properties: + secretKeyRef: {} + required: + - secretKeyRef + - properties: + configMapKeyRef: {} + required: + - configMapKeyRef + properties: + configMapKeyRef: + description: Reference to a key in a config map. + properties: + key: + type: string + name: + type: string + optional: + type: boolean + type: object + secretKeyRef: + description: Reference to a key in a secret. + properties: + key: + type: string + name: + type: string + optional: + type: boolean + type: object + type: object + type: object + type: array + securityContext: + description: Security context for the container. + properties: + allowPrivilegeEscalation: + type: boolean + appArmorProfile: + properties: + localhostProfile: + type: string + type: + type: string + type: object + capabilities: + properties: + add: + items: + type: string + type: array + drop: + items: + type: string + type: array + type: object + privileged: + type: boolean + procMount: + type: string + readOnlyRootFilesystem: + type: boolean + runAsGroup: + type: integer + runAsNonRoot: + type: boolean + runAsUser: + type: integer + seLinuxOptions: + properties: + level: + type: string + role: + type: string + type: + type: string + user: + type: string + type: object + seccompProfile: + properties: + localhostProfile: + type: string + type: + type: string + type: object + windowsOptions: + properties: + gmsaCredentialSpec: + type: string + gmsaCredentialSpecName: + type: string + hostProcess: + type: boolean + runAsUserName: + type: string + type: object + type: object + volumeMounts: + description: Additional volume mounts which should be applied to the container. + items: + properties: + mountPath: + type: string + mountPropagation: + type: string + name: + type: string + readOnly: + type: boolean + recursiveReadOnly: + type: string + subPath: + type: string + subPathExpr: + type: string + type: object + type: array + type: object + deployment: + description: Template for Kafka Connect `Deployment`. + properties: + deploymentStrategy: + description: Pod replacement strategy for deployment configuration changes. Valid values are `RollingUpdate` and `Recreate`. Defaults to `RollingUpdate`. + enum: + - RollingUpdate + - Recreate + type: string + metadata: + description: Metadata applied to the resource. + properties: + annotations: + additionalProperties: + type: string + description: Annotations added to the Kubernetes resource. + type: object + labels: + additionalProperties: + type: string + description: Labels added to the Kubernetes resource. + type: object + type: object + type: object + headlessService: + description: Template for Kafka Connect headless `Service`. + properties: + ipFamilies: + description: Specifies the IP Families used by the service. Available options are `IPv4` and `IPv6`. If unspecified, Kubernetes will choose the default value based on the `ipFamilyPolicy` setting. + items: + enum: + - IPv4 + - IPv6 + type: string + type: array + ipFamilyPolicy: + description: Specifies the IP Family Policy used by the service. Available options are `SingleStack`, `PreferDualStack` and `RequireDualStack`. `SingleStack` is for a single IP family. `PreferDualStack` is for two IP families on dual-stack configured clusters or a single IP family on single-stack clusters. `RequireDualStack` fails unless there are two IP families on dual-stack configured clusters. If unspecified, Kubernetes will choose the default value based on the service type. + enum: + - SingleStack + - PreferDualStack + - RequireDualStack + type: string + metadata: + description: Metadata applied to the resource. + properties: + annotations: + additionalProperties: + type: string + description: Annotations added to the Kubernetes resource. + type: object + labels: + additionalProperties: + type: string + description: Labels added to the Kubernetes resource. + type: object + type: object + type: object + initContainer: + description: Template for the Kafka init container. + properties: + env: + description: Environment variables which should be applied to the container. + items: + oneOf: + - properties: + value: {} + required: + - value + - properties: + valueFrom: {} + required: + - valueFrom + properties: + name: + description: The environment variable key. + type: string + value: + description: The environment variable value. + type: string + valueFrom: + description: Reference to the secret or config map property to which the environment variable is set. + oneOf: + - properties: + secretKeyRef: {} + required: + - secretKeyRef + - properties: + configMapKeyRef: {} + required: + - configMapKeyRef + properties: + configMapKeyRef: + description: Reference to a key in a config map. + properties: + key: + type: string + name: + type: string + optional: + type: boolean + type: object + secretKeyRef: + description: Reference to a key in a secret. + properties: + key: + type: string + name: + type: string + optional: + type: boolean + type: object + type: object + type: object + type: array + securityContext: + description: Security context for the container. + properties: + allowPrivilegeEscalation: + type: boolean + appArmorProfile: + properties: + localhostProfile: + type: string + type: + type: string + type: object + capabilities: + properties: + add: + items: + type: string + type: array + drop: + items: + type: string + type: array + type: object + privileged: + type: boolean + procMount: + type: string + readOnlyRootFilesystem: + type: boolean + runAsGroup: + type: integer + runAsNonRoot: + type: boolean + runAsUser: + type: integer + seLinuxOptions: + properties: + level: + type: string + role: + type: string + type: + type: string + user: + type: string + type: object + seccompProfile: + properties: + localhostProfile: + type: string + type: + type: string + type: object + windowsOptions: + properties: + gmsaCredentialSpec: + type: string + gmsaCredentialSpecName: + type: string + hostProcess: + type: boolean + runAsUserName: + type: string + type: object + type: object + volumeMounts: + description: Additional volume mounts which should be applied to the container. + items: + properties: + mountPath: + type: string + mountPropagation: + type: string + name: + type: string + readOnly: + type: boolean + recursiveReadOnly: + type: string + subPath: + type: string + subPathExpr: + type: string + type: object + type: array + type: object + jmxSecret: + description: Template for Secret of the Kafka Connect Cluster JMX authentication. + properties: + metadata: + description: Metadata applied to the resource. + properties: + annotations: + additionalProperties: + type: string + description: Annotations added to the Kubernetes resource. + type: object + labels: + additionalProperties: + type: string + description: Labels added to the Kubernetes resource. + type: object + type: object + type: object + pod: + description: Template for Kafka Connect `Pods`. + properties: + affinity: + description: The pod's affinity rules. + properties: + nodeAffinity: + properties: + preferredDuringSchedulingIgnoredDuringExecution: + items: + properties: + preference: + properties: + matchExpressions: + items: + properties: + key: + type: string + operator: + type: string + values: + items: + type: string + type: array + type: object + type: array + matchFields: + items: + properties: + key: + type: string + operator: + type: string + values: + items: + type: string + type: array + type: object + type: array + type: object + weight: + type: integer + type: object + type: array + requiredDuringSchedulingIgnoredDuringExecution: + properties: + nodeSelectorTerms: + items: + properties: + matchExpressions: + items: + properties: + key: + type: string + operator: + type: string + values: + items: + type: string + type: array + type: object + type: array + matchFields: + items: + properties: + key: + type: string + operator: + type: string + values: + items: + type: string + type: array + type: object + type: array + type: object + type: array + type: object + type: object + podAffinity: + properties: + preferredDuringSchedulingIgnoredDuringExecution: + items: + properties: + podAffinityTerm: + properties: + labelSelector: + properties: + matchExpressions: + items: + properties: + key: + type: string + operator: + type: string + values: + items: + type: string + type: array + type: object + type: array + matchLabels: + additionalProperties: + type: string + type: object + type: object + matchLabelKeys: + items: + type: string + type: array + mismatchLabelKeys: + items: + type: string + type: array + namespaceSelector: + properties: + matchExpressions: + items: + properties: + key: + type: string + operator: + type: string + values: + items: + type: string + type: array + type: object + type: array + matchLabels: + additionalProperties: + type: string + type: object + type: object + namespaces: + items: + type: string + type: array + topologyKey: + type: string + type: object + weight: + type: integer + type: object + type: array + requiredDuringSchedulingIgnoredDuringExecution: + items: + properties: + labelSelector: + properties: + matchExpressions: + items: + properties: + key: + type: string + operator: + type: string + values: + items: + type: string + type: array + type: object + type: array + matchLabels: + additionalProperties: + type: string + type: object + type: object + matchLabelKeys: + items: + type: string + type: array + mismatchLabelKeys: + items: + type: string + type: array + namespaceSelector: + properties: + matchExpressions: + items: + properties: + key: + type: string + operator: + type: string + values: + items: + type: string + type: array + type: object + type: array + matchLabels: + additionalProperties: + type: string + type: object + type: object + namespaces: + items: + type: string + type: array + topologyKey: + type: string + type: object + type: array + type: object + podAntiAffinity: + properties: + preferredDuringSchedulingIgnoredDuringExecution: + items: + properties: + podAffinityTerm: + properties: + labelSelector: + properties: + matchExpressions: + items: + properties: + key: + type: string + operator: + type: string + values: + items: + type: string + type: array + type: object + type: array + matchLabels: + additionalProperties: + type: string + type: object + type: object + matchLabelKeys: + items: + type: string + type: array + mismatchLabelKeys: + items: + type: string + type: array + namespaceSelector: + properties: + matchExpressions: + items: + properties: + key: + type: string + operator: + type: string + values: + items: + type: string + type: array + type: object + type: array + matchLabels: + additionalProperties: + type: string + type: object + type: object + namespaces: + items: + type: string + type: array + topologyKey: + type: string + type: object + weight: + type: integer + type: object + type: array + requiredDuringSchedulingIgnoredDuringExecution: + items: + properties: + labelSelector: + properties: + matchExpressions: + items: + properties: + key: + type: string + operator: + type: string + values: + items: + type: string + type: array + type: object + type: array + matchLabels: + additionalProperties: + type: string + type: object + type: object + matchLabelKeys: + items: + type: string + type: array + mismatchLabelKeys: + items: + type: string + type: array + namespaceSelector: + properties: + matchExpressions: + items: + properties: + key: + type: string + operator: + type: string + values: + items: + type: string + type: array + type: object + type: array + matchLabels: + additionalProperties: + type: string + type: object + type: object + namespaces: + items: + type: string + type: array + topologyKey: + type: string + type: object + type: array + type: object + type: object + enableServiceLinks: + description: Indicates whether information about services should be injected into Pod's environment variables. + type: boolean + hostAliases: + description: The pod's HostAliases. HostAliases is an optional list of hosts and IPs that will be injected into the Pod's hosts file if specified. + items: + properties: + hostnames: + items: + type: string + type: array + ip: + type: string + type: object + type: array + imagePullSecrets: + description: List of references to secrets in the same namespace to use for pulling any of the images used by this Pod. When the `STRIMZI_IMAGE_PULL_SECRETS` environment variable in Cluster Operator and the `imagePullSecrets` option are specified, only the `imagePullSecrets` variable is used and the `STRIMZI_IMAGE_PULL_SECRETS` variable is ignored. + items: + properties: + name: + type: string + type: object + type: array + metadata: + description: Metadata applied to the resource. + properties: + annotations: + additionalProperties: + type: string + description: Annotations added to the Kubernetes resource. + type: object + labels: + additionalProperties: + type: string + description: Labels added to the Kubernetes resource. + type: object + type: object + priorityClassName: + description: 'The name of the priority class used to assign priority to the pods. ' + type: string + schedulerName: + description: The name of the scheduler used to dispatch this `Pod`. If not specified, the default scheduler will be used. + type: string + securityContext: + description: Configures pod-level security attributes and common container settings. + properties: + appArmorProfile: + properties: + localhostProfile: + type: string + type: + type: string + type: object + fsGroup: + type: integer + fsGroupChangePolicy: + type: string + runAsGroup: + type: integer + runAsNonRoot: + type: boolean + runAsUser: + type: integer + seLinuxOptions: + properties: + level: + type: string + role: + type: string + type: + type: string + user: + type: string + type: object + seccompProfile: + properties: + localhostProfile: + type: string + type: + type: string + type: object + supplementalGroups: + items: + type: integer + type: array + sysctls: + items: + properties: + name: + type: string + value: + type: string + type: object + type: array + windowsOptions: + properties: + gmsaCredentialSpec: + type: string + gmsaCredentialSpecName: + type: string + hostProcess: + type: boolean + runAsUserName: + type: string + type: object + type: object + terminationGracePeriodSeconds: + description: The grace period is the duration in seconds after the processes running in the pod are sent a termination signal, and the time when the processes are forcibly halted with a kill signal. Set this value to longer than the expected cleanup time for your process. Value must be a non-negative integer. A zero value indicates delete immediately. You might need to increase the grace period for very large Kafka clusters, so that the Kafka brokers have enough time to transfer their work to another broker before they are terminated. Defaults to 30 seconds. + minimum: 0 + type: integer + tmpDirSizeLimit: + description: Defines the total amount of pod memory allocated for the temporary `EmptyDir` volume `/tmp`. Specify the allocation in memory units, for example, `100Mi` for 100 mebibytes. Default value is `5Mi`. The `/tmp` volume is backed by pod memory, not disk storage, so avoid setting a high value as it consumes pod memory resources. + pattern: ^([0-9.]+)([eEinumkKMGTP]*[-+]?[0-9]*)$ + type: string + tolerations: + description: The pod's tolerations. + items: + properties: + effect: + type: string + key: + type: string + operator: + type: string + tolerationSeconds: + type: integer + value: + type: string + type: object + type: array + topologySpreadConstraints: + description: The pod's topology spread constraints. + items: + properties: + labelSelector: + properties: + matchExpressions: + items: + properties: + key: + type: string + operator: + type: string + values: + items: + type: string + type: array + type: object + type: array + matchLabels: + additionalProperties: + type: string + type: object + type: object + matchLabelKeys: + items: + type: string + type: array + maxSkew: + type: integer + minDomains: + type: integer + nodeAffinityPolicy: + type: string + nodeTaintsPolicy: + type: string + topologyKey: + type: string + whenUnsatisfiable: + type: string + type: object + type: array + volumes: + description: Additional volumes that can be mounted to the pod. + items: + oneOf: + - properties: + configMap: {} + csi: {} + emptyDir: {} + persistentVolumeClaim: {} + secret: {} + properties: + configMap: + description: ConfigMap to use to populate the volume. + properties: + defaultMode: + type: integer + items: + items: + properties: + key: + type: string + mode: + type: integer + path: + type: string + type: object + type: array + name: + type: string + optional: + type: boolean + type: object + csi: + description: CSIVolumeSource object to use to populate the volume. + properties: + driver: + type: string + fsType: + type: string + nodePublishSecretRef: + properties: + name: + type: string + type: object + readOnly: + type: boolean + volumeAttributes: + additionalProperties: + type: string + type: object + type: object + emptyDir: + description: EmptyDir to use to populate the volume. + properties: + medium: + type: string + sizeLimit: + properties: + amount: + type: string + format: + type: string + type: object + type: object + name: + description: Name to use for the volume. Required. + type: string + persistentVolumeClaim: + description: PersistentVolumeClaim object to use to populate the volume. + properties: + claimName: + type: string + readOnly: + type: boolean + type: object + secret: + description: Secret to use populate the volume. + properties: + defaultMode: + type: integer + items: + items: + properties: + key: + type: string + mode: + type: integer + path: + type: string + type: object + type: array + optional: + type: boolean + secretName: + type: string + type: object + type: object + type: array + type: object + podDisruptionBudget: + description: Template for Kafka Connect `PodDisruptionBudget`. + properties: + maxUnavailable: + description: Maximum number of unavailable pods to allow automatic Pod eviction. A Pod eviction is allowed when the `maxUnavailable` number of pods or fewer are unavailable after the eviction. Setting this value to 0 prevents all voluntary evictions, so the pods must be evicted manually. Defaults to 1. + minimum: 0 + type: integer + metadata: + description: Metadata to apply to the `PodDisruptionBudgetTemplate` resource. + properties: + annotations: + additionalProperties: + type: string + description: Annotations added to the Kubernetes resource. + type: object + labels: + additionalProperties: + type: string + description: Labels added to the Kubernetes resource. + type: object + type: object + type: object + podSet: + description: Template for Kafka Connect `StrimziPodSet` resource. + properties: + metadata: + description: Metadata applied to the resource. + properties: + annotations: + additionalProperties: + type: string + description: Annotations added to the Kubernetes resource. + type: object + labels: + additionalProperties: + type: string + description: Labels added to the Kubernetes resource. + type: object + type: object + type: object + serviceAccount: + description: Template for the Kafka Connect service account. + properties: + metadata: + description: Metadata applied to the resource. + properties: + annotations: + additionalProperties: + type: string + description: Annotations added to the Kubernetes resource. + type: object + labels: + additionalProperties: + type: string + description: Labels added to the Kubernetes resource. + type: object + type: object + type: object + type: object + tracing: + description: The configuration of tracing in Kafka Connect. + properties: + type: + description: Type of the tracing used. Currently the only supported type is `opentelemetry` for OpenTelemetry tracing. As of Strimzi 0.37.0, `jaeger` type is not supported anymore and this option is ignored. + enum: + - jaeger + - opentelemetry + type: string + required: + - type + type: object + version: + description: The Kafka Connect version. Defaults to the latest version. Consult the user documentation to understand the process required to upgrade or downgrade the version. + type: string + required: + - connectCluster + type: object + status: + description: The status of the Kafka MirrorMaker 2 cluster. + properties: + autoRestartStatuses: + description: List of MirrorMaker 2 connector auto restart statuses. + items: + properties: + connectorName: + description: The name of the connector being restarted. + type: string + count: + description: The number of times the connector or task is restarted. + type: integer + lastRestartTimestamp: + description: The last time the automatic restart was attempted. The required format is 'yyyy-MM-ddTHH:mm:ssZ' in the UTC time zone. + type: string + type: object + type: array + conditions: + description: List of status conditions. + items: + properties: + lastTransitionTime: + description: Last time the condition of a type changed from one status to another. The required format is 'yyyy-MM-ddTHH:mm:ssZ', in the UTC time zone. + type: string + message: + description: Human-readable message indicating details about the condition's last transition. + type: string + reason: + description: The reason for the condition's last transition (a single word in CamelCase). + type: string + status: + description: The status of the condition, either True, False or Unknown. + type: string + type: + description: The unique identifier of a condition, used to distinguish between other conditions in the resource. + type: string + type: object + type: array + connectorPlugins: + description: The list of connector plugins available in this Kafka Connect deployment. + items: + properties: + class: + description: The class of the connector plugin. + type: string + type: + description: The type of the connector plugin. The available types are `sink` and `source`. + type: string + version: + description: The version of the connector plugin. + type: string + type: object + type: array + connectors: + description: List of MirrorMaker 2 connector statuses, as reported by the Kafka Connect REST API. + items: + type: object + x-kubernetes-preserve-unknown-fields: true + type: array + labelSelector: + description: Label selector for pods providing this resource. + type: string + observedGeneration: + description: The generation of the CRD that was last reconciled by the operator. + type: integer + replicas: + description: The current number of pods being used to provide this resource. + type: integer + url: + description: The URL of the REST API endpoint for managing and monitoring Kafka Connect connectors. + type: string + type: object + type: object + served: true + storage: true + subresources: + scale: + labelSelectorPath: .status.labelSelector + specReplicasPath: .spec.replicas + statusReplicasPath: .status.replicas + status: {} +--- +apiVersion: apiextensions.k8s.io/v1 +kind: CustomResourceDefinition +metadata: + labels: + app: strimzi + strimzi.io/crd-install: "true" + name: strimzipodsets.core.strimzi.io +spec: + conversion: + strategy: None + group: core.strimzi.io + names: + categories: + - strimzi + kind: StrimziPodSet + listKind: StrimziPodSetList + plural: strimzipodsets + shortNames: + - sps + singular: strimzipodset + scope: Namespaced + versions: + - additionalPrinterColumns: + - description: Number of pods managed by the StrimziPodSet + jsonPath: .status.pods + name: Pods + type: integer + - description: Number of ready pods managed by the StrimziPodSet + jsonPath: .status.readyPods + name: Ready Pods + type: integer + - description: Number of up-to-date pods managed by the StrimziPodSet + jsonPath: .status.currentPods + name: Current Pods + type: integer + - description: Age of the StrimziPodSet + jsonPath: .metadata.creationTimestamp + name: Age + type: date + name: v1beta2 + schema: + openAPIV3Schema: + properties: + apiVersion: + description: 'APIVersion defines the versioned schema of this representation of an object. Servers should convert recognized schemas to the latest internal value, and may reject unrecognized values. More info: https://git.k8s.io/community/contributors/devel/sig-architecture/api-conventions.md#resources' + type: string + kind: + description: 'Kind is a string value representing the REST resource this object represents. Servers may infer this from the endpoint the client submits requests to. Cannot be updated. In CamelCase. More info: https://git.k8s.io/community/contributors/devel/sig-architecture/api-conventions.md#types-kinds' + type: string + metadata: + type: object + spec: + description: The specification of the StrimziPodSet. + properties: + pods: + description: The Pods managed by this StrimziPodSet. + items: + type: object + x-kubernetes-preserve-unknown-fields: true + type: array + selector: + description: Selector is a label query which matches all the pods managed by this `StrimziPodSet`. Only `matchLabels` is supported. If `matchExpressions` is set, it will be ignored. + properties: + matchExpressions: + items: + properties: + key: + type: string + operator: + type: string + values: + items: + type: string + type: array + type: object + type: array + matchLabels: + additionalProperties: + type: string + type: object + type: object + required: + - selector + - pods + type: object + status: + description: The status of the StrimziPodSet. + properties: + conditions: + description: List of status conditions. + items: + properties: + lastTransitionTime: + description: Last time the condition of a type changed from one status to another. The required format is 'yyyy-MM-ddTHH:mm:ssZ', in the UTC time zone. + type: string + message: + description: Human-readable message indicating details about the condition's last transition. + type: string + reason: + description: The reason for the condition's last transition (a single word in CamelCase). + type: string + status: + description: The status of the condition, either True, False or Unknown. + type: string + type: + description: The unique identifier of a condition, used to distinguish between other conditions in the resource. + type: string + type: object + type: array + currentPods: + description: Number of pods managed by this `StrimziPodSet` resource that have the current revision. + type: integer + observedGeneration: + description: The generation of the CRD that was last reconciled by the operator. + type: integer + pods: + description: Number of pods managed by this `StrimziPodSet` resource. + type: integer + readyPods: + description: Number of pods managed by this `StrimziPodSet` resource that are ready. + type: integer + type: object + type: object + served: true + storage: true + subresources: + status: {} +--- +apiVersion: apiextensions.k8s.io/v1 +kind: CustomResourceDefinition +metadata: + labels: + app: strimzi + strimzi.io/crd-install: "true" + name: kafkanodepools.kafka.strimzi.io +spec: + conversion: + strategy: None + group: kafka.strimzi.io + names: + categories: + - strimzi + kind: KafkaNodePool + listKind: KafkaNodePoolList + plural: kafkanodepools + shortNames: + - knp + singular: kafkanodepool + scope: Namespaced + versions: + - additionalPrinterColumns: + - description: The desired number of replicas + jsonPath: .spec.replicas + name: Desired replicas + type: integer + - description: Roles of the nodes in the pool + jsonPath: .status.roles + name: Roles + type: string + - description: Node IDs used by Kafka nodes in this pool + jsonPath: .status.nodeIds + name: NodeIds + type: string + name: v1beta2 + schema: + openAPIV3Schema: + properties: + apiVersion: + description: 'APIVersion defines the versioned schema of this representation of an object. Servers should convert recognized schemas to the latest internal value, and may reject unrecognized values. More info: https://git.k8s.io/community/contributors/devel/sig-architecture/api-conventions.md#resources' + type: string + kind: + description: 'Kind is a string value representing the REST resource this object represents. Servers may infer this from the endpoint the client submits requests to. Cannot be updated. In CamelCase. More info: https://git.k8s.io/community/contributors/devel/sig-architecture/api-conventions.md#types-kinds' + type: string + metadata: + type: object + spec: + description: The specification of the KafkaNodePool. + properties: + jvmOptions: + description: JVM Options for pods. + properties: + -XX: + additionalProperties: + type: string + description: A map of -XX options to the JVM. + type: object + -Xms: + description: -Xms option to to the JVM. + pattern: ^[0-9]+[mMgG]?$ + type: string + -Xmx: + description: -Xmx option to to the JVM. + pattern: ^[0-9]+[mMgG]?$ + type: string + gcLoggingEnabled: + description: Specifies whether the Garbage Collection logging is enabled. The default is false. + type: boolean + javaSystemProperties: + description: A map of additional system properties which will be passed using the `-D` option to the JVM. + items: + properties: + name: + description: The system property name. + type: string + value: + description: The system property value. + type: string + type: object + type: array + type: object + replicas: + description: The number of pods in the pool. + minimum: 0 + type: integer + resources: + description: CPU and memory resources to reserve. + properties: + claims: + items: + properties: + name: + type: string + type: object + type: array + limits: + additionalProperties: + anyOf: + - type: integer + - type: string + pattern: ^(\+|-)?(([0-9]+(\.[0-9]*)?)|(\.[0-9]+))(([KMGTPE]i)|[numkMGTPE]|([eE](\+|-)?(([0-9]+(\.[0-9]*)?)|(\.[0-9]+))))?$ + x-kubernetes-int-or-string: true + type: object + requests: + additionalProperties: + anyOf: + - type: integer + - type: string + pattern: ^(\+|-)?(([0-9]+(\.[0-9]*)?)|(\.[0-9]+))(([KMGTPE]i)|[numkMGTPE]|([eE](\+|-)?(([0-9]+(\.[0-9]*)?)|(\.[0-9]+))))?$ + x-kubernetes-int-or-string: true + type: object + type: object + roles: + description: The roles that the nodes in this pool will have when KRaft mode is enabled. Supported values are 'broker' and 'controller'. This field is required. When KRaft mode is disabled, the only allowed value if `broker`. + items: + enum: + - controller + - broker + type: string + type: array + storage: + description: Storage configuration (disk). Cannot be updated. + properties: + class: + description: The storage class to use for dynamic volume allocation. + type: string + deleteClaim: + description: Specifies if the persistent volume claim has to be deleted when the cluster is un-deployed. + type: boolean + id: + description: Storage identification number. It is mandatory only for storage volumes defined in a storage of type 'jbod'. + minimum: 0 + type: integer + kraftMetadata: + description: Specifies whether this volume should be used for storing KRaft metadata. This property is optional. When set, the only currently supported value is `shared`. At most one volume can have this property set. + enum: + - shared + type: string + overrides: + description: Overrides for individual brokers. The `overrides` field allows you to specify a different configuration for different brokers. + items: + properties: + broker: + description: Id of the kafka broker (broker identifier). + type: integer + class: + description: The storage class to use for dynamic volume allocation for this broker. + type: string + type: object + type: array + selector: + additionalProperties: + type: string + description: Specifies a specific persistent volume to use. It contains key:value pairs representing labels for selecting such a volume. + type: object + size: + description: When `type=persistent-claim`, defines the size of the persistent volume claim, such as 100Gi. Mandatory when `type=persistent-claim`. + type: string + sizeLimit: + description: When type=ephemeral, defines the total amount of local storage required for this EmptyDir volume (for example 1Gi). + pattern: ^([0-9.]+)([eEinumkKMGTP]*[-+]?[0-9]*)$ + type: string + type: + description: Storage type, must be either 'ephemeral', 'persistent-claim', or 'jbod'. + enum: + - ephemeral + - persistent-claim + - jbod + type: string + volumes: + description: List of volumes as Storage objects representing the JBOD disks array. + items: + properties: + class: + description: The storage class to use for dynamic volume allocation. + type: string + deleteClaim: + description: Specifies if the persistent volume claim has to be deleted when the cluster is un-deployed. + type: boolean + id: + description: Storage identification number. Mandatory for storage volumes defined with a `jbod` storage type configuration. + minimum: 0 + type: integer + kraftMetadata: + description: Specifies whether this volume should be used for storing KRaft metadata. This property is optional. When set, the only currently supported value is `shared`. At most one volume can have this property set. + enum: + - shared + type: string + overrides: + description: Overrides for individual brokers. The `overrides` field allows you to specify a different configuration for different brokers. + items: + properties: + broker: + description: Id of the kafka broker (broker identifier). + type: integer + class: + description: The storage class to use for dynamic volume allocation for this broker. + type: string + type: object + type: array + selector: + additionalProperties: + type: string + description: Specifies a specific persistent volume to use. It contains key:value pairs representing labels for selecting such a volume. + type: object + size: + description: When `type=persistent-claim`, defines the size of the persistent volume claim, such as 100Gi. Mandatory when `type=persistent-claim`. + type: string + sizeLimit: + description: When type=ephemeral, defines the total amount of local storage required for this EmptyDir volume (for example 1Gi). + pattern: ^([0-9.]+)([eEinumkKMGTP]*[-+]?[0-9]*)$ + type: string + type: + description: Storage type, must be either 'ephemeral' or 'persistent-claim'. + enum: + - ephemeral + - persistent-claim + type: string + required: + - type + type: object + type: array + required: + - type + type: object + template: + description: Template for pool resources. The template allows users to specify how the resources belonging to this pool are generated. + properties: + initContainer: + description: Template for the Kafka init container. + properties: + env: + description: Environment variables which should be applied to the container. + items: + oneOf: + - properties: + value: {} + required: + - value + - properties: + valueFrom: {} + required: + - valueFrom + properties: + name: + description: The environment variable key. + type: string + value: + description: The environment variable value. + type: string + valueFrom: + description: Reference to the secret or config map property to which the environment variable is set. + oneOf: + - properties: + secretKeyRef: {} + required: + - secretKeyRef + - properties: + configMapKeyRef: {} + required: + - configMapKeyRef + properties: + configMapKeyRef: + description: Reference to a key in a config map. + properties: + key: + type: string + name: + type: string + optional: + type: boolean + type: object + secretKeyRef: + description: Reference to a key in a secret. + properties: + key: + type: string + name: + type: string + optional: + type: boolean + type: object + type: object + type: object + type: array + securityContext: + description: Security context for the container. + properties: + allowPrivilegeEscalation: + type: boolean + appArmorProfile: + properties: + localhostProfile: + type: string + type: + type: string + type: object + capabilities: + properties: + add: + items: + type: string + type: array + drop: + items: + type: string + type: array + type: object + privileged: + type: boolean + procMount: + type: string + readOnlyRootFilesystem: + type: boolean + runAsGroup: + type: integer + runAsNonRoot: + type: boolean + runAsUser: + type: integer + seLinuxOptions: + properties: + level: + type: string + role: + type: string + type: + type: string + user: + type: string + type: object + seccompProfile: + properties: + localhostProfile: + type: string + type: + type: string + type: object + windowsOptions: + properties: + gmsaCredentialSpec: + type: string + gmsaCredentialSpecName: + type: string + hostProcess: + type: boolean + runAsUserName: + type: string + type: object + type: object + volumeMounts: + description: Additional volume mounts which should be applied to the container. + items: + properties: + mountPath: + type: string + mountPropagation: + type: string + name: + type: string + readOnly: + type: boolean + recursiveReadOnly: + type: string + subPath: + type: string + subPathExpr: + type: string + type: object + type: array + type: object + kafkaContainer: + description: Template for the Kafka broker container. + properties: + env: + description: Environment variables which should be applied to the container. + items: + oneOf: + - properties: + value: {} + required: + - value + - properties: + valueFrom: {} + required: + - valueFrom + properties: + name: + description: The environment variable key. + type: string + value: + description: The environment variable value. + type: string + valueFrom: + description: Reference to the secret or config map property to which the environment variable is set. + oneOf: + - properties: + secretKeyRef: {} + required: + - secretKeyRef + - properties: + configMapKeyRef: {} + required: + - configMapKeyRef + properties: + configMapKeyRef: + description: Reference to a key in a config map. + properties: + key: + type: string + name: + type: string + optional: + type: boolean + type: object + secretKeyRef: + description: Reference to a key in a secret. + properties: + key: + type: string + name: + type: string + optional: + type: boolean + type: object + type: object + type: object + type: array + securityContext: + description: Security context for the container. + properties: + allowPrivilegeEscalation: + type: boolean + appArmorProfile: + properties: + localhostProfile: + type: string + type: + type: string + type: object + capabilities: + properties: + add: + items: + type: string + type: array + drop: + items: + type: string + type: array + type: object + privileged: + type: boolean + procMount: + type: string + readOnlyRootFilesystem: + type: boolean + runAsGroup: + type: integer + runAsNonRoot: + type: boolean + runAsUser: + type: integer + seLinuxOptions: + properties: + level: + type: string + role: + type: string + type: + type: string + user: + type: string + type: object + seccompProfile: + properties: + localhostProfile: + type: string + type: + type: string + type: object + windowsOptions: + properties: + gmsaCredentialSpec: + type: string + gmsaCredentialSpecName: + type: string + hostProcess: + type: boolean + runAsUserName: + type: string + type: object + type: object + volumeMounts: + description: Additional volume mounts which should be applied to the container. + items: + properties: + mountPath: + type: string + mountPropagation: + type: string + name: + type: string + readOnly: + type: boolean + recursiveReadOnly: + type: string + subPath: + type: string + subPathExpr: + type: string + type: object + type: array + type: object + perPodIngress: + description: Template for Kafka per-pod `Ingress` used for access from outside of Kubernetes. + properties: + metadata: + description: Metadata applied to the resource. + properties: + annotations: + additionalProperties: + type: string + description: Annotations added to the Kubernetes resource. + type: object + labels: + additionalProperties: + type: string + description: Labels added to the Kubernetes resource. + type: object + type: object + type: object + perPodRoute: + description: Template for Kafka per-pod `Routes` used for access from outside of OpenShift. + properties: + metadata: + description: Metadata applied to the resource. + properties: + annotations: + additionalProperties: + type: string + description: Annotations added to the Kubernetes resource. + type: object + labels: + additionalProperties: + type: string + description: Labels added to the Kubernetes resource. + type: object + type: object + type: object + perPodService: + description: Template for Kafka per-pod `Services` used for access from outside of Kubernetes. + properties: + metadata: + description: Metadata applied to the resource. + properties: + annotations: + additionalProperties: + type: string + description: Annotations added to the Kubernetes resource. + type: object + labels: + additionalProperties: + type: string + description: Labels added to the Kubernetes resource. + type: object + type: object + type: object + persistentVolumeClaim: + description: Template for all Kafka `PersistentVolumeClaims`. + properties: + metadata: + description: Metadata applied to the resource. + properties: + annotations: + additionalProperties: + type: string + description: Annotations added to the Kubernetes resource. + type: object + labels: + additionalProperties: + type: string + description: Labels added to the Kubernetes resource. + type: object + type: object + type: object + pod: + description: Template for Kafka `Pods`. + properties: + affinity: + description: The pod's affinity rules. + properties: + nodeAffinity: + properties: + preferredDuringSchedulingIgnoredDuringExecution: + items: + properties: + preference: + properties: + matchExpressions: + items: + properties: + key: + type: string + operator: + type: string + values: + items: + type: string + type: array + type: object + type: array + matchFields: + items: + properties: + key: + type: string + operator: + type: string + values: + items: + type: string + type: array + type: object + type: array + type: object + weight: + type: integer + type: object + type: array + requiredDuringSchedulingIgnoredDuringExecution: + properties: + nodeSelectorTerms: + items: + properties: + matchExpressions: + items: + properties: + key: + type: string + operator: + type: string + values: + items: + type: string + type: array + type: object + type: array + matchFields: + items: + properties: + key: + type: string + operator: + type: string + values: + items: + type: string + type: array + type: object + type: array + type: object + type: array + type: object + type: object + podAffinity: + properties: + preferredDuringSchedulingIgnoredDuringExecution: + items: + properties: + podAffinityTerm: + properties: + labelSelector: + properties: + matchExpressions: + items: + properties: + key: + type: string + operator: + type: string + values: + items: + type: string + type: array + type: object + type: array + matchLabels: + additionalProperties: + type: string + type: object + type: object + matchLabelKeys: + items: + type: string + type: array + mismatchLabelKeys: + items: + type: string + type: array + namespaceSelector: + properties: + matchExpressions: + items: + properties: + key: + type: string + operator: + type: string + values: + items: + type: string + type: array + type: object + type: array + matchLabels: + additionalProperties: + type: string + type: object + type: object + namespaces: + items: + type: string + type: array + topologyKey: + type: string + type: object + weight: + type: integer + type: object + type: array + requiredDuringSchedulingIgnoredDuringExecution: + items: + properties: + labelSelector: + properties: + matchExpressions: + items: + properties: + key: + type: string + operator: + type: string + values: + items: + type: string + type: array + type: object + type: array + matchLabels: + additionalProperties: + type: string + type: object + type: object + matchLabelKeys: + items: + type: string + type: array + mismatchLabelKeys: + items: + type: string + type: array + namespaceSelector: + properties: + matchExpressions: + items: + properties: + key: + type: string + operator: + type: string + values: + items: + type: string + type: array + type: object + type: array + matchLabels: + additionalProperties: + type: string + type: object + type: object + namespaces: + items: + type: string + type: array + topologyKey: + type: string + type: object + type: array + type: object + podAntiAffinity: + properties: + preferredDuringSchedulingIgnoredDuringExecution: + items: + properties: + podAffinityTerm: + properties: + labelSelector: + properties: + matchExpressions: + items: + properties: + key: + type: string + operator: + type: string + values: + items: + type: string + type: array + type: object + type: array + matchLabels: + additionalProperties: + type: string + type: object + type: object + matchLabelKeys: + items: + type: string + type: array + mismatchLabelKeys: + items: + type: string + type: array + namespaceSelector: + properties: + matchExpressions: + items: + properties: + key: + type: string + operator: + type: string + values: + items: + type: string + type: array + type: object + type: array + matchLabels: + additionalProperties: + type: string + type: object + type: object + namespaces: + items: + type: string + type: array + topologyKey: + type: string + type: object + weight: + type: integer + type: object + type: array + requiredDuringSchedulingIgnoredDuringExecution: + items: + properties: + labelSelector: + properties: + matchExpressions: + items: + properties: + key: + type: string + operator: + type: string + values: + items: + type: string + type: array + type: object + type: array + matchLabels: + additionalProperties: + type: string + type: object + type: object + matchLabelKeys: + items: + type: string + type: array + mismatchLabelKeys: + items: + type: string + type: array + namespaceSelector: + properties: + matchExpressions: + items: + properties: + key: + type: string + operator: + type: string + values: + items: + type: string + type: array + type: object + type: array + matchLabels: + additionalProperties: + type: string + type: object + type: object + namespaces: + items: + type: string + type: array + topologyKey: + type: string + type: object + type: array + type: object + type: object + enableServiceLinks: + description: Indicates whether information about services should be injected into Pod's environment variables. + type: boolean + hostAliases: + description: The pod's HostAliases. HostAliases is an optional list of hosts and IPs that will be injected into the Pod's hosts file if specified. + items: + properties: + hostnames: + items: + type: string + type: array + ip: + type: string + type: object + type: array + imagePullSecrets: + description: List of references to secrets in the same namespace to use for pulling any of the images used by this Pod. When the `STRIMZI_IMAGE_PULL_SECRETS` environment variable in Cluster Operator and the `imagePullSecrets` option are specified, only the `imagePullSecrets` variable is used and the `STRIMZI_IMAGE_PULL_SECRETS` variable is ignored. + items: + properties: + name: + type: string + type: object + type: array + metadata: + description: Metadata applied to the resource. + properties: + annotations: + additionalProperties: + type: string + description: Annotations added to the Kubernetes resource. + type: object + labels: + additionalProperties: + type: string + description: Labels added to the Kubernetes resource. + type: object + type: object + priorityClassName: + description: 'The name of the priority class used to assign priority to the pods. ' + type: string + schedulerName: + description: The name of the scheduler used to dispatch this `Pod`. If not specified, the default scheduler will be used. + type: string + securityContext: + description: Configures pod-level security attributes and common container settings. + properties: + appArmorProfile: + properties: + localhostProfile: + type: string + type: + type: string + type: object + fsGroup: + type: integer + fsGroupChangePolicy: + type: string + runAsGroup: + type: integer + runAsNonRoot: + type: boolean + runAsUser: + type: integer + seLinuxOptions: + properties: + level: + type: string + role: + type: string + type: + type: string + user: + type: string + type: object + seccompProfile: + properties: + localhostProfile: + type: string + type: + type: string + type: object + supplementalGroups: + items: + type: integer + type: array + sysctls: + items: + properties: + name: + type: string + value: + type: string + type: object + type: array + windowsOptions: + properties: + gmsaCredentialSpec: + type: string + gmsaCredentialSpecName: + type: string + hostProcess: + type: boolean + runAsUserName: + type: string + type: object + type: object + terminationGracePeriodSeconds: + description: The grace period is the duration in seconds after the processes running in the pod are sent a termination signal, and the time when the processes are forcibly halted with a kill signal. Set this value to longer than the expected cleanup time for your process. Value must be a non-negative integer. A zero value indicates delete immediately. You might need to increase the grace period for very large Kafka clusters, so that the Kafka brokers have enough time to transfer their work to another broker before they are terminated. Defaults to 30 seconds. + minimum: 0 + type: integer + tmpDirSizeLimit: + description: Defines the total amount of pod memory allocated for the temporary `EmptyDir` volume `/tmp`. Specify the allocation in memory units, for example, `100Mi` for 100 mebibytes. Default value is `5Mi`. The `/tmp` volume is backed by pod memory, not disk storage, so avoid setting a high value as it consumes pod memory resources. + pattern: ^([0-9.]+)([eEinumkKMGTP]*[-+]?[0-9]*)$ + type: string + tolerations: + description: The pod's tolerations. + items: + properties: + effect: + type: string + key: + type: string + operator: + type: string + tolerationSeconds: + type: integer + value: + type: string + type: object + type: array + topologySpreadConstraints: + description: The pod's topology spread constraints. + items: + properties: + labelSelector: + properties: + matchExpressions: + items: + properties: + key: + type: string + operator: + type: string + values: + items: + type: string + type: array + type: object + type: array + matchLabels: + additionalProperties: + type: string + type: object + type: object + matchLabelKeys: + items: + type: string + type: array + maxSkew: + type: integer + minDomains: + type: integer + nodeAffinityPolicy: + type: string + nodeTaintsPolicy: + type: string + topologyKey: + type: string + whenUnsatisfiable: + type: string + type: object + type: array + volumes: + description: Additional volumes that can be mounted to the pod. + items: + oneOf: + - properties: + configMap: {} + csi: {} + emptyDir: {} + persistentVolumeClaim: {} + secret: {} + properties: + configMap: + description: ConfigMap to use to populate the volume. + properties: + defaultMode: + type: integer + items: + items: + properties: + key: + type: string + mode: + type: integer + path: + type: string + type: object + type: array + name: + type: string + optional: + type: boolean + type: object + csi: + description: CSIVolumeSource object to use to populate the volume. + properties: + driver: + type: string + fsType: + type: string + nodePublishSecretRef: + properties: + name: + type: string + type: object + readOnly: + type: boolean + volumeAttributes: + additionalProperties: + type: string + type: object + type: object + emptyDir: + description: EmptyDir to use to populate the volume. + properties: + medium: + type: string + sizeLimit: + properties: + amount: + type: string + format: + type: string + type: object + type: object + name: + description: Name to use for the volume. Required. + type: string + persistentVolumeClaim: + description: PersistentVolumeClaim object to use to populate the volume. + properties: + claimName: + type: string + readOnly: + type: boolean + type: object + secret: + description: Secret to use populate the volume. + properties: + defaultMode: + type: integer + items: + items: + properties: + key: + type: string + mode: + type: integer + path: + type: string + type: object + type: array + optional: + type: boolean + secretName: + type: string + type: object + type: object + type: array + type: object + podSet: + description: Template for Kafka `StrimziPodSet` resource. + properties: + metadata: + description: Metadata applied to the resource. + properties: + annotations: + additionalProperties: + type: string + description: Annotations added to the Kubernetes resource. + type: object + labels: + additionalProperties: + type: string + description: Labels added to the Kubernetes resource. + type: object + type: object + type: object + type: object + required: + - replicas + - storage + - roles + type: object + status: + description: The status of the KafkaNodePool. + properties: + clusterId: + description: Kafka cluster ID. + type: string + conditions: + description: List of status conditions. + items: + properties: + lastTransitionTime: + description: Last time the condition of a type changed from one status to another. The required format is 'yyyy-MM-ddTHH:mm:ssZ', in the UTC time zone. + type: string + message: + description: Human-readable message indicating details about the condition's last transition. + type: string + reason: + description: The reason for the condition's last transition (a single word in CamelCase). + type: string + status: + description: The status of the condition, either True, False or Unknown. + type: string + type: + description: The unique identifier of a condition, used to distinguish between other conditions in the resource. + type: string + type: object + type: array + labelSelector: + description: Label selector for pods providing this resource. + type: string + nodeIds: + description: Node IDs used by Kafka nodes in this pool. + items: + type: integer + type: array + observedGeneration: + description: The generation of the CRD that was last reconciled by the operator. + type: integer + replicas: + description: The current number of pods being used to provide this resource. + type: integer + roles: + description: The roles currently assigned to this pool. + items: + enum: + - controller + - broker + type: string + type: array + type: object + type: object + served: true + storage: true + subresources: + scale: + labelSelectorPath: .status.labelSelector + specReplicasPath: .spec.replicas + statusReplicasPath: .status.replicas + status: {} +--- +apiVersion: apiextensions.k8s.io/v1 +kind: CustomResourceDefinition +metadata: + labels: + app: strimzi + strimzi.io/crd-install: "true" + name: kafkamirrormakers.kafka.strimzi.io +spec: + conversion: + strategy: None + group: kafka.strimzi.io + names: + categories: + - strimzi + kind: KafkaMirrorMaker + listKind: KafkaMirrorMakerList + plural: kafkamirrormakers + shortNames: + - kmm + singular: kafkamirrormaker + scope: Namespaced + versions: + - additionalPrinterColumns: + - description: The desired number of Kafka MirrorMaker replicas + jsonPath: .spec.replicas + name: Desired replicas + type: integer + - description: The boostrap servers for the consumer + jsonPath: .spec.consumer.bootstrapServers + name: Consumer Bootstrap Servers + priority: 1 + type: string + - description: The boostrap servers for the producer + jsonPath: .spec.producer.bootstrapServers + name: Producer Bootstrap Servers + priority: 1 + type: string + - description: The state of the custom resource + jsonPath: .status.conditions[?(@.type=="Ready")].status + name: Ready + type: string + name: v1beta2 + schema: + openAPIV3Schema: + properties: + apiVersion: + description: 'APIVersion defines the versioned schema of this representation of an object. Servers should convert recognized schemas to the latest internal value, and may reject unrecognized values. More info: https://git.k8s.io/community/contributors/devel/sig-architecture/api-conventions.md#resources' + type: string + kind: + description: 'Kind is a string value representing the REST resource this object represents. Servers may infer this from the endpoint the client submits requests to. Cannot be updated. In CamelCase. More info: https://git.k8s.io/community/contributors/devel/sig-architecture/api-conventions.md#types-kinds' + type: string + metadata: + type: object + spec: + description: The specification of Kafka MirrorMaker. + oneOf: + - properties: + include: {} + required: + - include + - properties: + whitelist: {} + required: + - whitelist + properties: + consumer: + description: Configuration of source cluster. + properties: + authentication: + description: Authentication configuration for connecting to the cluster. + properties: + accessToken: + description: Link to Kubernetes Secret containing the access token which was obtained from the authorization server. + properties: + key: + description: The key under which the secret value is stored in the Kubernetes Secret. + type: string + secretName: + description: The name of the Kubernetes Secret containing the secret value. + type: string + required: + - key + - secretName + type: object + accessTokenIsJwt: + description: Configure whether access token should be treated as JWT. This should be set to `false` if the authorization server returns opaque tokens. Defaults to `true`. + type: boolean + accessTokenLocation: + description: Path to the token file containing an access token to be used for authentication. + type: string + audience: + description: OAuth audience to use when authenticating against the authorization server. Some authorization servers require the audience to be explicitly set. The possible values depend on how the authorization server is configured. By default, `audience` is not specified when performing the token endpoint request. + type: string + certificateAndKey: + description: Reference to the `Secret` which holds the certificate and private key pair. + properties: + certificate: + description: The name of the file certificate in the Secret. + type: string + key: + description: The name of the private key in the Secret. + type: string + secretName: + description: The name of the Secret containing the certificate. + type: string + required: + - secretName + - certificate + - key + type: object + clientAssertion: + description: Link to Kubernetes secret containing the client assertion which was manually configured for the client. + properties: + key: + description: The key under which the secret value is stored in the Kubernetes Secret. + type: string + secretName: + description: The name of the Kubernetes Secret containing the secret value. + type: string + required: + - key + - secretName + type: object + clientAssertionLocation: + description: Path to the file containing the client assertion to be used for authentication. + type: string + clientAssertionType: + description: The client assertion type. If not set, and either `clientAssertion` or `clientAssertionLocation` is configured, this value defaults to `urn:ietf:params:oauth:client-assertion-type:jwt-bearer`. + type: string + clientId: + description: OAuth Client ID which the Kafka client can use to authenticate against the OAuth server and use the token endpoint URI. + type: string + clientSecret: + description: Link to Kubernetes Secret containing the OAuth client secret which the Kafka client can use to authenticate against the OAuth server and use the token endpoint URI. + properties: + key: + description: The key under which the secret value is stored in the Kubernetes Secret. + type: string + secretName: + description: The name of the Kubernetes Secret containing the secret value. + type: string + required: + - key + - secretName + type: object + connectTimeoutSeconds: + description: The connect timeout in seconds when connecting to authorization server. If not set, the effective connect timeout is 60 seconds. + type: integer + disableTlsHostnameVerification: + description: Enable or disable TLS hostname verification. Default value is `false`. + type: boolean + enableMetrics: + description: Enable or disable OAuth metrics. Default value is `false`. + type: boolean + httpRetries: + description: The maximum number of retries to attempt if an initial HTTP request fails. If not set, the default is to not attempt any retries. + type: integer + httpRetryPauseMs: + description: The pause to take before retrying a failed HTTP request. If not set, the default is to not pause at all but to immediately repeat a request. + type: integer + includeAcceptHeader: + description: Whether the Accept header should be set in requests to the authorization servers. The default value is `true`. + type: boolean + maxTokenExpirySeconds: + description: Set or limit time-to-live of the access tokens to the specified number of seconds. This should be set if the authorization server returns opaque tokens. + type: integer + passwordSecret: + description: Reference to the `Secret` which holds the password. + properties: + password: + description: The name of the key in the Secret under which the password is stored. + type: string + secretName: + description: The name of the Secret containing the password. + type: string + required: + - secretName + - password + type: object + readTimeoutSeconds: + description: The read timeout in seconds when connecting to authorization server. If not set, the effective read timeout is 60 seconds. + type: integer + refreshToken: + description: Link to Kubernetes Secret containing the refresh token which can be used to obtain access token from the authorization server. + properties: + key: + description: The key under which the secret value is stored in the Kubernetes Secret. + type: string + secretName: + description: The name of the Kubernetes Secret containing the secret value. + type: string + required: + - key + - secretName + type: object + saslExtensions: + additionalProperties: + type: string + description: SASL extensions parameters. + type: object + scope: + description: OAuth scope to use when authenticating against the authorization server. Some authorization servers require this to be set. The possible values depend on how authorization server is configured. By default `scope` is not specified when doing the token endpoint request. + type: string + tlsTrustedCertificates: + description: Trusted certificates for TLS connection to the OAuth server. + items: + oneOf: + - properties: + certificate: {} + required: + - certificate + - properties: + pattern: {} + required: + - pattern + properties: + certificate: + description: The name of the file certificate in the secret. + type: string + pattern: + description: Pattern for the certificate files in the secret. Use the link:https://en.wikipedia.org/wiki/Glob_(programming)[_glob syntax_] for the pattern. All files in the secret that match the pattern are used. + type: string + secretName: + description: The name of the Secret containing the certificate. + type: string + required: + - secretName + type: object + type: array + tokenEndpointUri: + description: Authorization server token endpoint URI. + type: string + type: + description: Authentication type. Currently the supported types are `tls`, `scram-sha-256`, `scram-sha-512`, `plain`, and 'oauth'. `scram-sha-256` and `scram-sha-512` types use SASL SCRAM-SHA-256 and SASL SCRAM-SHA-512 Authentication, respectively. `plain` type uses SASL PLAIN Authentication. `oauth` type uses SASL OAUTHBEARER Authentication. The `tls` type uses TLS Client Authentication. The `tls` type is supported only over TLS connections. + enum: + - tls + - scram-sha-256 + - scram-sha-512 + - plain + - oauth + type: string + username: + description: Username used for the authentication. + type: string + required: + - type + type: object + bootstrapServers: + description: A list of host:port pairs for establishing the initial connection to the Kafka cluster. + type: string + config: + description: 'The MirrorMaker consumer config. Properties with the following prefixes cannot be set: ssl., bootstrap.servers, group.id, sasl., security., interceptor.classes (with the exception of: ssl.endpoint.identification.algorithm, ssl.cipher.suites, ssl.protocol, ssl.enabled.protocols).' + type: object + x-kubernetes-preserve-unknown-fields: true + groupId: + description: A unique string that identifies the consumer group this consumer belongs to. + type: string + numStreams: + description: Specifies the number of consumer stream threads to create. + minimum: 1 + type: integer + offsetCommitInterval: + description: Specifies the offset auto-commit interval in ms. Default value is 60000. + type: integer + tls: + description: TLS configuration for connecting MirrorMaker to the cluster. + properties: + trustedCertificates: + description: Trusted certificates for TLS connection. + items: + oneOf: + - properties: + certificate: {} + required: + - certificate + - properties: + pattern: {} + required: + - pattern + properties: + certificate: + description: The name of the file certificate in the secret. + type: string + pattern: + description: Pattern for the certificate files in the secret. Use the link:https://en.wikipedia.org/wiki/Glob_(programming)[_glob syntax_] for the pattern. All files in the secret that match the pattern are used. + type: string + secretName: + description: The name of the Secret containing the certificate. + type: string + required: + - secretName + type: object + type: array + type: object + required: + - bootstrapServers + - groupId + type: object + image: + description: The container image used for Kafka MirrorMaker pods. If no image name is explicitly specified, it is determined based on the `spec.version` configuration. The image names are specifically mapped to corresponding versions in the Cluster Operator configuration. + type: string + include: + description: List of topics which are included for mirroring. This option allows any regular expression using Java-style regular expressions. Mirroring two topics named A and B is achieved by using the expression `A\|B`. Or, as a special case, you can mirror all topics using the regular expression `*`. You can also specify multiple regular expressions separated by commas. + type: string + jvmOptions: + description: JVM Options for pods. + properties: + -XX: + additionalProperties: + type: string + description: A map of -XX options to the JVM. + type: object + -Xms: + description: -Xms option to to the JVM. + pattern: ^[0-9]+[mMgG]?$ + type: string + -Xmx: + description: -Xmx option to to the JVM. + pattern: ^[0-9]+[mMgG]?$ + type: string + gcLoggingEnabled: + description: Specifies whether the Garbage Collection logging is enabled. The default is false. + type: boolean + javaSystemProperties: + description: A map of additional system properties which will be passed using the `-D` option to the JVM. + items: + properties: + name: + description: The system property name. + type: string + value: + description: The system property value. + type: string + type: object + type: array + type: object + livenessProbe: + description: Pod liveness checking. + properties: + failureThreshold: + description: Minimum consecutive failures for the probe to be considered failed after having succeeded. Defaults to 3. Minimum value is 1. + minimum: 1 + type: integer + initialDelaySeconds: + description: The initial delay before first the health is first checked. Default to 15 seconds. Minimum value is 0. + minimum: 0 + type: integer + periodSeconds: + description: How often (in seconds) to perform the probe. Default to 10 seconds. Minimum value is 1. + minimum: 1 + type: integer + successThreshold: + description: Minimum consecutive successes for the probe to be considered successful after having failed. Defaults to 1. Must be 1 for liveness. Minimum value is 1. + minimum: 1 + type: integer + timeoutSeconds: + description: The timeout for each attempted health check. Default to 5 seconds. Minimum value is 1. + minimum: 1 + type: integer + type: object + logging: + description: Logging configuration for MirrorMaker. + properties: + loggers: + additionalProperties: + type: string + description: A Map from logger name to logger level. + type: object + type: + description: Logging type, must be either 'inline' or 'external'. + enum: + - inline + - external + type: string + valueFrom: + description: '`ConfigMap` entry where the logging configuration is stored. ' + properties: + configMapKeyRef: + description: Reference to the key in the ConfigMap containing the configuration. + properties: + key: + type: string + name: + type: string + optional: + type: boolean + type: object + type: object + required: + - type + type: object + metricsConfig: + description: Metrics configuration. + properties: + type: + description: Metrics type. Only 'jmxPrometheusExporter' supported currently. + enum: + - jmxPrometheusExporter + type: string + valueFrom: + description: 'ConfigMap entry where the Prometheus JMX Exporter configuration is stored. ' + properties: + configMapKeyRef: + description: Reference to the key in the ConfigMap containing the configuration. + properties: + key: + type: string + name: + type: string + optional: + type: boolean + type: object + type: object + required: + - type + - valueFrom + type: object + producer: + description: Configuration of target cluster. + properties: + abortOnSendFailure: + description: Flag to set the MirrorMaker to exit on a failed send. Default value is `true`. + type: boolean + authentication: + description: Authentication configuration for connecting to the cluster. + properties: + accessToken: + description: Link to Kubernetes Secret containing the access token which was obtained from the authorization server. + properties: + key: + description: The key under which the secret value is stored in the Kubernetes Secret. + type: string + secretName: + description: The name of the Kubernetes Secret containing the secret value. + type: string + required: + - key + - secretName + type: object + accessTokenIsJwt: + description: Configure whether access token should be treated as JWT. This should be set to `false` if the authorization server returns opaque tokens. Defaults to `true`. + type: boolean + accessTokenLocation: + description: Path to the token file containing an access token to be used for authentication. + type: string + audience: + description: OAuth audience to use when authenticating against the authorization server. Some authorization servers require the audience to be explicitly set. The possible values depend on how the authorization server is configured. By default, `audience` is not specified when performing the token endpoint request. + type: string + certificateAndKey: + description: Reference to the `Secret` which holds the certificate and private key pair. + properties: + certificate: + description: The name of the file certificate in the Secret. + type: string + key: + description: The name of the private key in the Secret. + type: string + secretName: + description: The name of the Secret containing the certificate. + type: string + required: + - secretName + - certificate + - key + type: object + clientAssertion: + description: Link to Kubernetes secret containing the client assertion which was manually configured for the client. + properties: + key: + description: The key under which the secret value is stored in the Kubernetes Secret. + type: string + secretName: + description: The name of the Kubernetes Secret containing the secret value. + type: string + required: + - key + - secretName + type: object + clientAssertionLocation: + description: Path to the file containing the client assertion to be used for authentication. + type: string + clientAssertionType: + description: The client assertion type. If not set, and either `clientAssertion` or `clientAssertionLocation` is configured, this value defaults to `urn:ietf:params:oauth:client-assertion-type:jwt-bearer`. + type: string + clientId: + description: OAuth Client ID which the Kafka client can use to authenticate against the OAuth server and use the token endpoint URI. + type: string + clientSecret: + description: Link to Kubernetes Secret containing the OAuth client secret which the Kafka client can use to authenticate against the OAuth server and use the token endpoint URI. + properties: + key: + description: The key under which the secret value is stored in the Kubernetes Secret. + type: string + secretName: + description: The name of the Kubernetes Secret containing the secret value. + type: string + required: + - key + - secretName + type: object + connectTimeoutSeconds: + description: The connect timeout in seconds when connecting to authorization server. If not set, the effective connect timeout is 60 seconds. + type: integer + disableTlsHostnameVerification: + description: Enable or disable TLS hostname verification. Default value is `false`. + type: boolean + enableMetrics: + description: Enable or disable OAuth metrics. Default value is `false`. + type: boolean + httpRetries: + description: The maximum number of retries to attempt if an initial HTTP request fails. If not set, the default is to not attempt any retries. + type: integer + httpRetryPauseMs: + description: The pause to take before retrying a failed HTTP request. If not set, the default is to not pause at all but to immediately repeat a request. + type: integer + includeAcceptHeader: + description: Whether the Accept header should be set in requests to the authorization servers. The default value is `true`. + type: boolean + maxTokenExpirySeconds: + description: Set or limit time-to-live of the access tokens to the specified number of seconds. This should be set if the authorization server returns opaque tokens. + type: integer + passwordSecret: + description: Reference to the `Secret` which holds the password. + properties: + password: + description: The name of the key in the Secret under which the password is stored. + type: string + secretName: + description: The name of the Secret containing the password. + type: string + required: + - secretName + - password + type: object + readTimeoutSeconds: + description: The read timeout in seconds when connecting to authorization server. If not set, the effective read timeout is 60 seconds. + type: integer + refreshToken: + description: Link to Kubernetes Secret containing the refresh token which can be used to obtain access token from the authorization server. + properties: + key: + description: The key under which the secret value is stored in the Kubernetes Secret. + type: string + secretName: + description: The name of the Kubernetes Secret containing the secret value. + type: string + required: + - key + - secretName + type: object + saslExtensions: + additionalProperties: + type: string + description: SASL extensions parameters. + type: object + scope: + description: OAuth scope to use when authenticating against the authorization server. Some authorization servers require this to be set. The possible values depend on how authorization server is configured. By default `scope` is not specified when doing the token endpoint request. + type: string + tlsTrustedCertificates: + description: Trusted certificates for TLS connection to the OAuth server. + items: + oneOf: + - properties: + certificate: {} + required: + - certificate + - properties: + pattern: {} + required: + - pattern + properties: + certificate: + description: The name of the file certificate in the secret. + type: string + pattern: + description: Pattern for the certificate files in the secret. Use the link:https://en.wikipedia.org/wiki/Glob_(programming)[_glob syntax_] for the pattern. All files in the secret that match the pattern are used. + type: string + secretName: + description: The name of the Secret containing the certificate. + type: string + required: + - secretName + type: object + type: array + tokenEndpointUri: + description: Authorization server token endpoint URI. + type: string + type: + description: Authentication type. Currently the supported types are `tls`, `scram-sha-256`, `scram-sha-512`, `plain`, and 'oauth'. `scram-sha-256` and `scram-sha-512` types use SASL SCRAM-SHA-256 and SASL SCRAM-SHA-512 Authentication, respectively. `plain` type uses SASL PLAIN Authentication. `oauth` type uses SASL OAUTHBEARER Authentication. The `tls` type uses TLS Client Authentication. The `tls` type is supported only over TLS connections. + enum: + - tls + - scram-sha-256 + - scram-sha-512 + - plain + - oauth + type: string + username: + description: Username used for the authentication. + type: string + required: + - type + type: object + bootstrapServers: + description: A list of host:port pairs for establishing the initial connection to the Kafka cluster. + type: string + config: + description: 'The MirrorMaker producer config. Properties with the following prefixes cannot be set: ssl., bootstrap.servers, sasl., security., interceptor.classes (with the exception of: ssl.endpoint.identification.algorithm, ssl.cipher.suites, ssl.protocol, ssl.enabled.protocols).' + type: object + x-kubernetes-preserve-unknown-fields: true + tls: + description: TLS configuration for connecting MirrorMaker to the cluster. + properties: + trustedCertificates: + description: Trusted certificates for TLS connection. + items: + oneOf: + - properties: + certificate: {} + required: + - certificate + - properties: + pattern: {} + required: + - pattern + properties: + certificate: + description: The name of the file certificate in the secret. + type: string + pattern: + description: Pattern for the certificate files in the secret. Use the link:https://en.wikipedia.org/wiki/Glob_(programming)[_glob syntax_] for the pattern. All files in the secret that match the pattern are used. + type: string + secretName: + description: The name of the Secret containing the certificate. + type: string + required: + - secretName + type: object + type: array + type: object + required: + - bootstrapServers + type: object + readinessProbe: + description: Pod readiness checking. + properties: + failureThreshold: + description: Minimum consecutive failures for the probe to be considered failed after having succeeded. Defaults to 3. Minimum value is 1. + minimum: 1 + type: integer + initialDelaySeconds: + description: The initial delay before first the health is first checked. Default to 15 seconds. Minimum value is 0. + minimum: 0 + type: integer + periodSeconds: + description: How often (in seconds) to perform the probe. Default to 10 seconds. Minimum value is 1. + minimum: 1 + type: integer + successThreshold: + description: Minimum consecutive successes for the probe to be considered successful after having failed. Defaults to 1. Must be 1 for liveness. Minimum value is 1. + minimum: 1 + type: integer + timeoutSeconds: + description: The timeout for each attempted health check. Default to 5 seconds. Minimum value is 1. + minimum: 1 + type: integer + type: object + replicas: + description: The number of pods in the `Deployment`. + minimum: 0 + type: integer + resources: + description: CPU and memory resources to reserve. + properties: + claims: + items: + properties: + name: + type: string + type: object + type: array + limits: + additionalProperties: + anyOf: + - type: integer + - type: string + pattern: ^(\+|-)?(([0-9]+(\.[0-9]*)?)|(\.[0-9]+))(([KMGTPE]i)|[numkMGTPE]|([eE](\+|-)?(([0-9]+(\.[0-9]*)?)|(\.[0-9]+))))?$ + x-kubernetes-int-or-string: true + type: object + requests: + additionalProperties: + anyOf: + - type: integer + - type: string + pattern: ^(\+|-)?(([0-9]+(\.[0-9]*)?)|(\.[0-9]+))(([KMGTPE]i)|[numkMGTPE]|([eE](\+|-)?(([0-9]+(\.[0-9]*)?)|(\.[0-9]+))))?$ + x-kubernetes-int-or-string: true + type: object + type: object + template: + description: Template to specify how Kafka MirrorMaker resources, `Deployments` and `Pods`, are generated. + properties: + deployment: + description: Template for Kafka MirrorMaker `Deployment`. + properties: + deploymentStrategy: + description: Pod replacement strategy for deployment configuration changes. Valid values are `RollingUpdate` and `Recreate`. Defaults to `RollingUpdate`. + enum: + - RollingUpdate + - Recreate + type: string + metadata: + description: Metadata applied to the resource. + properties: + annotations: + additionalProperties: + type: string + description: Annotations added to the Kubernetes resource. + type: object + labels: + additionalProperties: + type: string + description: Labels added to the Kubernetes resource. + type: object + type: object + type: object + mirrorMakerContainer: + description: Template for Kafka MirrorMaker container. + properties: + env: + description: Environment variables which should be applied to the container. + items: + oneOf: + - properties: + value: {} + required: + - value + - properties: + valueFrom: {} + required: + - valueFrom + properties: + name: + description: The environment variable key. + type: string + value: + description: The environment variable value. + type: string + valueFrom: + description: Reference to the secret or config map property to which the environment variable is set. + oneOf: + - properties: + secretKeyRef: {} + required: + - secretKeyRef + - properties: + configMapKeyRef: {} + required: + - configMapKeyRef + properties: + configMapKeyRef: + description: Reference to a key in a config map. + properties: + key: + type: string + name: + type: string + optional: + type: boolean + type: object + secretKeyRef: + description: Reference to a key in a secret. + properties: + key: + type: string + name: + type: string + optional: + type: boolean + type: object + type: object + type: object + type: array + securityContext: + description: Security context for the container. + properties: + allowPrivilegeEscalation: + type: boolean + appArmorProfile: + properties: + localhostProfile: + type: string + type: + type: string + type: object + capabilities: + properties: + add: + items: + type: string + type: array + drop: + items: + type: string + type: array + type: object + privileged: + type: boolean + procMount: + type: string + readOnlyRootFilesystem: + type: boolean + runAsGroup: + type: integer + runAsNonRoot: + type: boolean + runAsUser: + type: integer + seLinuxOptions: + properties: + level: + type: string + role: + type: string + type: + type: string + user: + type: string + type: object + seccompProfile: + properties: + localhostProfile: + type: string + type: + type: string + type: object + windowsOptions: + properties: + gmsaCredentialSpec: + type: string + gmsaCredentialSpecName: + type: string + hostProcess: + type: boolean + runAsUserName: + type: string + type: object + type: object + volumeMounts: + description: Additional volume mounts which should be applied to the container. + items: + properties: + mountPath: + type: string + mountPropagation: + type: string + name: + type: string + readOnly: + type: boolean + recursiveReadOnly: + type: string + subPath: + type: string + subPathExpr: + type: string + type: object + type: array + type: object + pod: + description: Template for Kafka MirrorMaker `Pods`. + properties: + affinity: + description: The pod's affinity rules. + properties: + nodeAffinity: + properties: + preferredDuringSchedulingIgnoredDuringExecution: + items: + properties: + preference: + properties: + matchExpressions: + items: + properties: + key: + type: string + operator: + type: string + values: + items: + type: string + type: array + type: object + type: array + matchFields: + items: + properties: + key: + type: string + operator: + type: string + values: + items: + type: string + type: array + type: object + type: array + type: object + weight: + type: integer + type: object + type: array + requiredDuringSchedulingIgnoredDuringExecution: + properties: + nodeSelectorTerms: + items: + properties: + matchExpressions: + items: + properties: + key: + type: string + operator: + type: string + values: + items: + type: string + type: array + type: object + type: array + matchFields: + items: + properties: + key: + type: string + operator: + type: string + values: + items: + type: string + type: array + type: object + type: array + type: object + type: array + type: object + type: object + podAffinity: + properties: + preferredDuringSchedulingIgnoredDuringExecution: + items: + properties: + podAffinityTerm: + properties: + labelSelector: + properties: + matchExpressions: + items: + properties: + key: + type: string + operator: + type: string + values: + items: + type: string + type: array + type: object + type: array + matchLabels: + additionalProperties: + type: string + type: object + type: object + matchLabelKeys: + items: + type: string + type: array + mismatchLabelKeys: + items: + type: string + type: array + namespaceSelector: + properties: + matchExpressions: + items: + properties: + key: + type: string + operator: + type: string + values: + items: + type: string + type: array + type: object + type: array + matchLabels: + additionalProperties: + type: string + type: object + type: object + namespaces: + items: + type: string + type: array + topologyKey: + type: string + type: object + weight: + type: integer + type: object + type: array + requiredDuringSchedulingIgnoredDuringExecution: + items: + properties: + labelSelector: + properties: + matchExpressions: + items: + properties: + key: + type: string + operator: + type: string + values: + items: + type: string + type: array + type: object + type: array + matchLabels: + additionalProperties: + type: string + type: object + type: object + matchLabelKeys: + items: + type: string + type: array + mismatchLabelKeys: + items: + type: string + type: array + namespaceSelector: + properties: + matchExpressions: + items: + properties: + key: + type: string + operator: + type: string + values: + items: + type: string + type: array + type: object + type: array + matchLabels: + additionalProperties: + type: string + type: object + type: object + namespaces: + items: + type: string + type: array + topologyKey: + type: string + type: object + type: array + type: object + podAntiAffinity: + properties: + preferredDuringSchedulingIgnoredDuringExecution: + items: + properties: + podAffinityTerm: + properties: + labelSelector: + properties: + matchExpressions: + items: + properties: + key: + type: string + operator: + type: string + values: + items: + type: string + type: array + type: object + type: array + matchLabels: + additionalProperties: + type: string + type: object + type: object + matchLabelKeys: + items: + type: string + type: array + mismatchLabelKeys: + items: + type: string + type: array + namespaceSelector: + properties: + matchExpressions: + items: + properties: + key: + type: string + operator: + type: string + values: + items: + type: string + type: array + type: object + type: array + matchLabels: + additionalProperties: + type: string + type: object + type: object + namespaces: + items: + type: string + type: array + topologyKey: + type: string + type: object + weight: + type: integer + type: object + type: array + requiredDuringSchedulingIgnoredDuringExecution: + items: + properties: + labelSelector: + properties: + matchExpressions: + items: + properties: + key: + type: string + operator: + type: string + values: + items: + type: string + type: array + type: object + type: array + matchLabels: + additionalProperties: + type: string + type: object + type: object + matchLabelKeys: + items: + type: string + type: array + mismatchLabelKeys: + items: + type: string + type: array + namespaceSelector: + properties: + matchExpressions: + items: + properties: + key: + type: string + operator: + type: string + values: + items: + type: string + type: array + type: object + type: array + matchLabels: + additionalProperties: + type: string + type: object + type: object + namespaces: + items: + type: string + type: array + topologyKey: + type: string + type: object + type: array + type: object + type: object + enableServiceLinks: + description: Indicates whether information about services should be injected into Pod's environment variables. + type: boolean + hostAliases: + description: The pod's HostAliases. HostAliases is an optional list of hosts and IPs that will be injected into the Pod's hosts file if specified. + items: + properties: + hostnames: + items: + type: string + type: array + ip: + type: string + type: object + type: array + imagePullSecrets: + description: List of references to secrets in the same namespace to use for pulling any of the images used by this Pod. When the `STRIMZI_IMAGE_PULL_SECRETS` environment variable in Cluster Operator and the `imagePullSecrets` option are specified, only the `imagePullSecrets` variable is used and the `STRIMZI_IMAGE_PULL_SECRETS` variable is ignored. + items: + properties: + name: + type: string + type: object + type: array + metadata: + description: Metadata applied to the resource. + properties: + annotations: + additionalProperties: + type: string + description: Annotations added to the Kubernetes resource. + type: object + labels: + additionalProperties: + type: string + description: Labels added to the Kubernetes resource. + type: object + type: object + priorityClassName: + description: 'The name of the priority class used to assign priority to the pods. ' + type: string + schedulerName: + description: The name of the scheduler used to dispatch this `Pod`. If not specified, the default scheduler will be used. + type: string + securityContext: + description: Configures pod-level security attributes and common container settings. + properties: + appArmorProfile: + properties: + localhostProfile: + type: string + type: + type: string + type: object + fsGroup: + type: integer + fsGroupChangePolicy: + type: string + runAsGroup: + type: integer + runAsNonRoot: + type: boolean + runAsUser: + type: integer + seLinuxOptions: + properties: + level: + type: string + role: + type: string + type: + type: string + user: + type: string + type: object + seccompProfile: + properties: + localhostProfile: + type: string + type: + type: string + type: object + supplementalGroups: + items: + type: integer + type: array + sysctls: + items: + properties: + name: + type: string + value: + type: string + type: object + type: array + windowsOptions: + properties: + gmsaCredentialSpec: + type: string + gmsaCredentialSpecName: + type: string + hostProcess: + type: boolean + runAsUserName: + type: string + type: object + type: object + terminationGracePeriodSeconds: + description: The grace period is the duration in seconds after the processes running in the pod are sent a termination signal, and the time when the processes are forcibly halted with a kill signal. Set this value to longer than the expected cleanup time for your process. Value must be a non-negative integer. A zero value indicates delete immediately. You might need to increase the grace period for very large Kafka clusters, so that the Kafka brokers have enough time to transfer their work to another broker before they are terminated. Defaults to 30 seconds. + minimum: 0 + type: integer + tmpDirSizeLimit: + description: Defines the total amount of pod memory allocated for the temporary `EmptyDir` volume `/tmp`. Specify the allocation in memory units, for example, `100Mi` for 100 mebibytes. Default value is `5Mi`. The `/tmp` volume is backed by pod memory, not disk storage, so avoid setting a high value as it consumes pod memory resources. + pattern: ^([0-9.]+)([eEinumkKMGTP]*[-+]?[0-9]*)$ + type: string + tolerations: + description: The pod's tolerations. + items: + properties: + effect: + type: string + key: + type: string + operator: + type: string + tolerationSeconds: + type: integer + value: + type: string + type: object + type: array + topologySpreadConstraints: + description: The pod's topology spread constraints. + items: + properties: + labelSelector: + properties: + matchExpressions: + items: + properties: + key: + type: string + operator: + type: string + values: + items: + type: string + type: array + type: object + type: array + matchLabels: + additionalProperties: + type: string + type: object + type: object + matchLabelKeys: + items: + type: string + type: array + maxSkew: + type: integer + minDomains: + type: integer + nodeAffinityPolicy: + type: string + nodeTaintsPolicy: + type: string + topologyKey: + type: string + whenUnsatisfiable: + type: string + type: object + type: array + volumes: + description: Additional volumes that can be mounted to the pod. + items: + oneOf: + - properties: + configMap: {} + csi: {} + emptyDir: {} + persistentVolumeClaim: {} + secret: {} + properties: + configMap: + description: ConfigMap to use to populate the volume. + properties: + defaultMode: + type: integer + items: + items: + properties: + key: + type: string + mode: + type: integer + path: + type: string + type: object + type: array + name: + type: string + optional: + type: boolean + type: object + csi: + description: CSIVolumeSource object to use to populate the volume. + properties: + driver: + type: string + fsType: + type: string + nodePublishSecretRef: + properties: + name: + type: string + type: object + readOnly: + type: boolean + volumeAttributes: + additionalProperties: + type: string + type: object + type: object + emptyDir: + description: EmptyDir to use to populate the volume. + properties: + medium: + type: string + sizeLimit: + properties: + amount: + type: string + format: + type: string + type: object + type: object + name: + description: Name to use for the volume. Required. + type: string + persistentVolumeClaim: + description: PersistentVolumeClaim object to use to populate the volume. + properties: + claimName: + type: string + readOnly: + type: boolean + type: object + secret: + description: Secret to use populate the volume. + properties: + defaultMode: + type: integer + items: + items: + properties: + key: + type: string + mode: + type: integer + path: + type: string + type: object + type: array + optional: + type: boolean + secretName: + type: string + type: object + type: object + type: array + type: object + podDisruptionBudget: + description: Template for Kafka MirrorMaker `PodDisruptionBudget`. + properties: + maxUnavailable: + description: Maximum number of unavailable pods to allow automatic Pod eviction. A Pod eviction is allowed when the `maxUnavailable` number of pods or fewer are unavailable after the eviction. Setting this value to 0 prevents all voluntary evictions, so the pods must be evicted manually. Defaults to 1. + minimum: 0 + type: integer + metadata: + description: Metadata to apply to the `PodDisruptionBudgetTemplate` resource. + properties: + annotations: + additionalProperties: + type: string + description: Annotations added to the Kubernetes resource. + type: object + labels: + additionalProperties: + type: string + description: Labels added to the Kubernetes resource. + type: object + type: object + type: object + serviceAccount: + description: Template for the Kafka MirrorMaker service account. + properties: + metadata: + description: Metadata applied to the resource. + properties: + annotations: + additionalProperties: + type: string + description: Annotations added to the Kubernetes resource. + type: object + labels: + additionalProperties: + type: string + description: Labels added to the Kubernetes resource. + type: object + type: object + type: object + type: object + tracing: + description: The configuration of tracing in Kafka MirrorMaker. + properties: + type: + description: Type of the tracing used. Currently the only supported type is `opentelemetry` for OpenTelemetry tracing. As of Strimzi 0.37.0, `jaeger` type is not supported anymore and this option is ignored. + enum: + - jaeger + - opentelemetry + type: string + required: + - type + type: object + version: + description: The Kafka MirrorMaker version. Defaults to the latest version. Consult the documentation to understand the process required to upgrade or downgrade the version. + type: string + whitelist: + description: List of topics which are included for mirroring. This option allows any regular expression using Java-style regular expressions. Mirroring two topics named A and B is achieved by using the expression `A\|B`. Or, as a special case, you can mirror all topics using the regular expression `*`. You can also specify multiple regular expressions separated by commas. + type: string + required: + - replicas + - consumer + - producer + type: object + status: + description: The status of Kafka MirrorMaker. + properties: + conditions: + description: List of status conditions. + items: + properties: + lastTransitionTime: + description: Last time the condition of a type changed from one status to another. The required format is 'yyyy-MM-ddTHH:mm:ssZ', in the UTC time zone. + type: string + message: + description: Human-readable message indicating details about the condition's last transition. + type: string + reason: + description: The reason for the condition's last transition (a single word in CamelCase). + type: string + status: + description: The status of the condition, either True, False or Unknown. + type: string + type: + description: The unique identifier of a condition, used to distinguish between other conditions in the resource. + type: string + type: object + type: array + labelSelector: + description: Label selector for pods providing this resource. + type: string + observedGeneration: + description: The generation of the CRD that was last reconciled by the operator. + type: integer + replicas: + description: The current number of pods being used to provide this resource. + type: integer + type: object + type: object + served: true + storage: true + subresources: + scale: + labelSelectorPath: .status.labelSelector + specReplicasPath: .spec.replicas + statusReplicasPath: .status.replicas + status: {} +--- +apiVersion: apiextensions.k8s.io/v1 +kind: CustomResourceDefinition +metadata: + labels: + app: strimzi + strimzi.io/crd-install: "true" + name: kafkas.kafka.strimzi.io +spec: + conversion: + strategy: None + group: kafka.strimzi.io + names: + categories: + - strimzi + kind: Kafka + listKind: KafkaList + plural: kafkas + shortNames: + - k + singular: kafka + scope: Namespaced + versions: + - additionalPrinterColumns: + - description: The desired number of Kafka replicas in the cluster + jsonPath: .spec.kafka.replicas + name: Desired Kafka replicas + type: integer + - description: The desired number of ZooKeeper replicas in the cluster + jsonPath: .spec.zookeeper.replicas + name: Desired ZK replicas + type: integer + - description: The state of the custom resource + jsonPath: .status.conditions[?(@.type=="Ready")].status + name: Ready + type: string + - description: The state of the cluster metadata + jsonPath: .status.kafkaMetadataState + name: Metadata State + type: string + - description: Warnings related to the custom resource + jsonPath: .status.conditions[?(@.type=="Warning")].status + name: Warnings + type: string + name: v1beta2 + schema: + openAPIV3Schema: + properties: + apiVersion: + description: 'APIVersion defines the versioned schema of this representation of an object. Servers should convert recognized schemas to the latest internal value, and may reject unrecognized values. More info: https://git.k8s.io/community/contributors/devel/sig-architecture/api-conventions.md#resources' + type: string + kind: + description: 'Kind is a string value representing the REST resource this object represents. Servers may infer this from the endpoint the client submits requests to. Cannot be updated. In CamelCase. More info: https://git.k8s.io/community/contributors/devel/sig-architecture/api-conventions.md#types-kinds' + type: string + metadata: + type: object + spec: + description: The specification of the Kafka and ZooKeeper clusters, and Topic Operator. + properties: + clientsCa: + description: Configuration of the clients certificate authority. + properties: + certificateExpirationPolicy: + description: How should CA certificate expiration be handled when `generateCertificateAuthority=true`. The default is for a new CA certificate to be generated reusing the existing private key. + enum: + - renew-certificate + - replace-key + type: string + generateCertificateAuthority: + description: If true then Certificate Authority certificates will be generated automatically. Otherwise the user will need to provide a Secret with the CA certificate. Default is true. + type: boolean + generateSecretOwnerReference: + description: If `true`, the Cluster and Client CA Secrets are configured with the `ownerReference` set to the `Kafka` resource. If the `Kafka` resource is deleted when `true`, the CA Secrets are also deleted. If `false`, the `ownerReference` is disabled. If the `Kafka` resource is deleted when `false`, the CA Secrets are retained and available for reuse. Default is `true`. + type: boolean + renewalDays: + description: The number of days in the certificate renewal period. This is the number of days before the a certificate expires during which renewal actions may be performed. When `generateCertificateAuthority` is true, this will cause the generation of a new certificate. When `generateCertificateAuthority` is true, this will cause extra logging at WARN level about the pending certificate expiry. Default is 30. + minimum: 1 + type: integer + validityDays: + description: The number of days generated certificates should be valid for. The default is 365. + minimum: 1 + type: integer + type: object + clusterCa: + description: Configuration of the cluster certificate authority. + properties: + certificateExpirationPolicy: + description: How should CA certificate expiration be handled when `generateCertificateAuthority=true`. The default is for a new CA certificate to be generated reusing the existing private key. + enum: + - renew-certificate + - replace-key + type: string + generateCertificateAuthority: + description: If true then Certificate Authority certificates will be generated automatically. Otherwise the user will need to provide a Secret with the CA certificate. Default is true. + type: boolean + generateSecretOwnerReference: + description: If `true`, the Cluster and Client CA Secrets are configured with the `ownerReference` set to the `Kafka` resource. If the `Kafka` resource is deleted when `true`, the CA Secrets are also deleted. If `false`, the `ownerReference` is disabled. If the `Kafka` resource is deleted when `false`, the CA Secrets are retained and available for reuse. Default is `true`. + type: boolean + renewalDays: + description: The number of days in the certificate renewal period. This is the number of days before the a certificate expires during which renewal actions may be performed. When `generateCertificateAuthority` is true, this will cause the generation of a new certificate. When `generateCertificateAuthority` is true, this will cause extra logging at WARN level about the pending certificate expiry. Default is 30. + minimum: 1 + type: integer + validityDays: + description: The number of days generated certificates should be valid for. The default is 365. + minimum: 1 + type: integer + type: object + cruiseControl: + description: Configuration for Cruise Control deployment. Deploys a Cruise Control instance when specified. + properties: + apiUsers: + description: Configuration of the Cruise Control REST API users. + properties: + type: + description: 'Type of the Cruise Control API users configuration. Supported format is: `hashLoginService`.' + enum: + - hashLoginService + type: string + valueFrom: + description: Secret from which the custom Cruise Control API authentication credentials are read. + properties: + secretKeyRef: + description: Selects a key of a Secret in the resource's namespace. + properties: + key: + type: string + name: + type: string + optional: + type: boolean + type: object + type: object + required: + - type + - valueFrom + type: object + autoRebalance: + description: Auto-rebalancing on scaling related configuration listing the modes, when brokers are added or removed, with the corresponding rebalance template configurations.If this field is set, at least one mode has to be defined. + items: + properties: + mode: + description: "Specifies the mode for automatically rebalancing when brokers are added or removed. Supported modes are `add-brokers` and `remove-brokers`. \n" + enum: + - add-brokers + - remove-brokers + type: string + template: + description: Reference to the KafkaRebalance custom resource to be used as the configuration template for the auto-rebalancing on scaling when running for the corresponding mode. + properties: + name: + type: string + type: object + required: + - mode + type: object + minItems: 1 + type: array + brokerCapacity: + description: The Cruise Control `brokerCapacity` configuration. + properties: + cpu: + description: Broker capacity for CPU resource in cores or millicores. For example, 1, 1.500, 1500m. For more information on valid CPU resource units see https://kubernetes.io/docs/concepts/configuration/manage-resources-containers/#meaning-of-cpu. + pattern: ^[0-9]+([.][0-9]{0,3}|[m]?)$ + type: string + cpuUtilization: + description: Broker capacity for CPU resource utilization as a percentage (0 - 100). + maximum: 100 + minimum: 0 + type: integer + disk: + description: Broker capacity for disk in bytes. Use a number value with either standard Kubernetes byte units (K, M, G, or T), their bibyte (power of two) equivalents (Ki, Mi, Gi, or Ti), or a byte value with or without E notation. For example, 100000M, 100000Mi, 104857600000, or 1e+11. + pattern: ^[0-9]+([.][0-9]*)?([KMGTPE]i?|e[0-9]+)?$ + type: string + inboundNetwork: + description: Broker capacity for inbound network throughput in bytes per second. Use an integer value with standard Kubernetes byte units (K, M, G) or their bibyte (power of two) equivalents (Ki, Mi, Gi) per second. For example, 10000KiB/s. + pattern: ^[0-9]+([KMG]i?)?B/s$ + type: string + outboundNetwork: + description: Broker capacity for outbound network throughput in bytes per second. Use an integer value with standard Kubernetes byte units (K, M, G) or their bibyte (power of two) equivalents (Ki, Mi, Gi) per second. For example, 10000KiB/s. + pattern: ^[0-9]+([KMG]i?)?B/s$ + type: string + overrides: + description: Overrides for individual brokers. The `overrides` property lets you specify a different capacity configuration for different brokers. + items: + properties: + brokers: + description: List of Kafka brokers (broker identifiers). + items: + type: integer + type: array + cpu: + description: Broker capacity for CPU resource in cores or millicores. For example, 1, 1.500, 1500m. For more information on valid CPU resource units see https://kubernetes.io/docs/concepts/configuration/manage-resources-containers/#meaning-of-cpu. + pattern: ^[0-9]+([.][0-9]{0,3}|[m]?)$ + type: string + inboundNetwork: + description: Broker capacity for inbound network throughput in bytes per second. Use an integer value with standard Kubernetes byte units (K, M, G) or their bibyte (power of two) equivalents (Ki, Mi, Gi) per second. For example, 10000KiB/s. + pattern: ^[0-9]+([KMG]i?)?B/s$ + type: string + outboundNetwork: + description: Broker capacity for outbound network throughput in bytes per second. Use an integer value with standard Kubernetes byte units (K, M, G) or their bibyte (power of two) equivalents (Ki, Mi, Gi) per second. For example, 10000KiB/s. + pattern: ^[0-9]+([KMG]i?)?B/s$ + type: string + required: + - brokers + type: object + type: array + type: object + config: + description: 'The Cruise Control configuration. For a full list of configuration options refer to https://github.com/linkedin/cruise-control/wiki/Configurations. Note that properties with the following prefixes cannot be set: bootstrap.servers, client.id, zookeeper., network., security., failed.brokers.zk.path,webserver.http., webserver.api.urlprefix, webserver.session.path, webserver.accesslog., two.step., request.reason.required,metric.reporter.sampler.bootstrap.servers, capacity.config.file, self.healing., ssl., kafka.broker.failure.detection.enable, topic.config.provider.class (with the exception of: ssl.cipher.suites, ssl.protocol, ssl.enabled.protocols, webserver.http.cors.enabled, webserver.http.cors.origin, webserver.http.cors.exposeheaders, webserver.security.enable, webserver.ssl.enable).' + type: object + x-kubernetes-preserve-unknown-fields: true + image: + description: The container image used for Cruise Control pods. If no image name is explicitly specified, the image name corresponds to the name specified in the Cluster Operator configuration. If an image name is not defined in the Cluster Operator configuration, a default value is used. + type: string + jvmOptions: + description: JVM Options for the Cruise Control container. + properties: + -XX: + additionalProperties: + type: string + description: A map of -XX options to the JVM. + type: object + -Xms: + description: -Xms option to to the JVM. + pattern: ^[0-9]+[mMgG]?$ + type: string + -Xmx: + description: -Xmx option to to the JVM. + pattern: ^[0-9]+[mMgG]?$ + type: string + gcLoggingEnabled: + description: Specifies whether the Garbage Collection logging is enabled. The default is false. + type: boolean + javaSystemProperties: + description: A map of additional system properties which will be passed using the `-D` option to the JVM. + items: + properties: + name: + description: The system property name. + type: string + value: + description: The system property value. + type: string + type: object + type: array + type: object + livenessProbe: + description: Pod liveness checking for the Cruise Control container. + properties: + failureThreshold: + description: Minimum consecutive failures for the probe to be considered failed after having succeeded. Defaults to 3. Minimum value is 1. + minimum: 1 + type: integer + initialDelaySeconds: + description: The initial delay before first the health is first checked. Default to 15 seconds. Minimum value is 0. + minimum: 0 + type: integer + periodSeconds: + description: How often (in seconds) to perform the probe. Default to 10 seconds. Minimum value is 1. + minimum: 1 + type: integer + successThreshold: + description: Minimum consecutive successes for the probe to be considered successful after having failed. Defaults to 1. Must be 1 for liveness. Minimum value is 1. + minimum: 1 + type: integer + timeoutSeconds: + description: The timeout for each attempted health check. Default to 5 seconds. Minimum value is 1. + minimum: 1 + type: integer + type: object + logging: + description: Logging configuration (Log4j 2) for Cruise Control. + properties: + loggers: + additionalProperties: + type: string + description: A Map from logger name to logger level. + type: object + type: + description: Logging type, must be either 'inline' or 'external'. + enum: + - inline + - external + type: string + valueFrom: + description: '`ConfigMap` entry where the logging configuration is stored. ' + properties: + configMapKeyRef: + description: Reference to the key in the ConfigMap containing the configuration. + properties: + key: + type: string + name: + type: string + optional: + type: boolean + type: object + type: object + required: + - type + type: object + metricsConfig: + description: Metrics configuration. + properties: + type: + description: Metrics type. Only 'jmxPrometheusExporter' supported currently. + enum: + - jmxPrometheusExporter + type: string + valueFrom: + description: 'ConfigMap entry where the Prometheus JMX Exporter configuration is stored. ' + properties: + configMapKeyRef: + description: Reference to the key in the ConfigMap containing the configuration. + properties: + key: + type: string + name: + type: string + optional: + type: boolean + type: object + type: object + required: + - type + - valueFrom + type: object + readinessProbe: + description: Pod readiness checking for the Cruise Control container. + properties: + failureThreshold: + description: Minimum consecutive failures for the probe to be considered failed after having succeeded. Defaults to 3. Minimum value is 1. + minimum: 1 + type: integer + initialDelaySeconds: + description: The initial delay before first the health is first checked. Default to 15 seconds. Minimum value is 0. + minimum: 0 + type: integer + periodSeconds: + description: How often (in seconds) to perform the probe. Default to 10 seconds. Minimum value is 1. + minimum: 1 + type: integer + successThreshold: + description: Minimum consecutive successes for the probe to be considered successful after having failed. Defaults to 1. Must be 1 for liveness. Minimum value is 1. + minimum: 1 + type: integer + timeoutSeconds: + description: The timeout for each attempted health check. Default to 5 seconds. Minimum value is 1. + minimum: 1 + type: integer + type: object + resources: + description: CPU and memory resources to reserve for the Cruise Control container. + properties: + claims: + items: + properties: + name: + type: string + type: object + type: array + limits: + additionalProperties: + anyOf: + - type: integer + - type: string + pattern: ^(\+|-)?(([0-9]+(\.[0-9]*)?)|(\.[0-9]+))(([KMGTPE]i)|[numkMGTPE]|([eE](\+|-)?(([0-9]+(\.[0-9]*)?)|(\.[0-9]+))))?$ + x-kubernetes-int-or-string: true + type: object + requests: + additionalProperties: + anyOf: + - type: integer + - type: string + pattern: ^(\+|-)?(([0-9]+(\.[0-9]*)?)|(\.[0-9]+))(([KMGTPE]i)|[numkMGTPE]|([eE](\+|-)?(([0-9]+(\.[0-9]*)?)|(\.[0-9]+))))?$ + x-kubernetes-int-or-string: true + type: object + type: object + template: + description: Template to specify how Cruise Control resources, `Deployments` and `Pods`, are generated. + properties: + apiService: + description: Template for Cruise Control API `Service`. + properties: + ipFamilies: + description: Specifies the IP Families used by the service. Available options are `IPv4` and `IPv6`. If unspecified, Kubernetes will choose the default value based on the `ipFamilyPolicy` setting. + items: + enum: + - IPv4 + - IPv6 + type: string + type: array + ipFamilyPolicy: + description: Specifies the IP Family Policy used by the service. Available options are `SingleStack`, `PreferDualStack` and `RequireDualStack`. `SingleStack` is for a single IP family. `PreferDualStack` is for two IP families on dual-stack configured clusters or a single IP family on single-stack clusters. `RequireDualStack` fails unless there are two IP families on dual-stack configured clusters. If unspecified, Kubernetes will choose the default value based on the service type. + enum: + - SingleStack + - PreferDualStack + - RequireDualStack + type: string + metadata: + description: Metadata applied to the resource. + properties: + annotations: + additionalProperties: + type: string + description: Annotations added to the Kubernetes resource. + type: object + labels: + additionalProperties: + type: string + description: Labels added to the Kubernetes resource. + type: object + type: object + type: object + cruiseControlContainer: + description: Template for the Cruise Control container. + properties: + env: + description: Environment variables which should be applied to the container. + items: + oneOf: + - properties: + value: {} + required: + - value + - properties: + valueFrom: {} + required: + - valueFrom + properties: + name: + description: The environment variable key. + type: string + value: + description: The environment variable value. + type: string + valueFrom: + description: Reference to the secret or config map property to which the environment variable is set. + oneOf: + - properties: + secretKeyRef: {} + required: + - secretKeyRef + - properties: + configMapKeyRef: {} + required: + - configMapKeyRef + properties: + configMapKeyRef: + description: Reference to a key in a config map. + properties: + key: + type: string + name: + type: string + optional: + type: boolean + type: object + secretKeyRef: + description: Reference to a key in a secret. + properties: + key: + type: string + name: + type: string + optional: + type: boolean + type: object + type: object + type: object + type: array + securityContext: + description: Security context for the container. + properties: + allowPrivilegeEscalation: + type: boolean + appArmorProfile: + properties: + localhostProfile: + type: string + type: + type: string + type: object + capabilities: + properties: + add: + items: + type: string + type: array + drop: + items: + type: string + type: array + type: object + privileged: + type: boolean + procMount: + type: string + readOnlyRootFilesystem: + type: boolean + runAsGroup: + type: integer + runAsNonRoot: + type: boolean + runAsUser: + type: integer + seLinuxOptions: + properties: + level: + type: string + role: + type: string + type: + type: string + user: + type: string + type: object + seccompProfile: + properties: + localhostProfile: + type: string + type: + type: string + type: object + windowsOptions: + properties: + gmsaCredentialSpec: + type: string + gmsaCredentialSpecName: + type: string + hostProcess: + type: boolean + runAsUserName: + type: string + type: object + type: object + volumeMounts: + description: Additional volume mounts which should be applied to the container. + items: + properties: + mountPath: + type: string + mountPropagation: + type: string + name: + type: string + readOnly: + type: boolean + recursiveReadOnly: + type: string + subPath: + type: string + subPathExpr: + type: string + type: object + type: array + type: object + deployment: + description: Template for Cruise Control `Deployment`. + properties: + deploymentStrategy: + description: Pod replacement strategy for deployment configuration changes. Valid values are `RollingUpdate` and `Recreate`. Defaults to `RollingUpdate`. + enum: + - RollingUpdate + - Recreate + type: string + metadata: + description: Metadata applied to the resource. + properties: + annotations: + additionalProperties: + type: string + description: Annotations added to the Kubernetes resource. + type: object + labels: + additionalProperties: + type: string + description: Labels added to the Kubernetes resource. + type: object + type: object + type: object + pod: + description: Template for Cruise Control `Pods`. + properties: + affinity: + description: The pod's affinity rules. + properties: + nodeAffinity: + properties: + preferredDuringSchedulingIgnoredDuringExecution: + items: + properties: + preference: + properties: + matchExpressions: + items: + properties: + key: + type: string + operator: + type: string + values: + items: + type: string + type: array + type: object + type: array + matchFields: + items: + properties: + key: + type: string + operator: + type: string + values: + items: + type: string + type: array + type: object + type: array + type: object + weight: + type: integer + type: object + type: array + requiredDuringSchedulingIgnoredDuringExecution: + properties: + nodeSelectorTerms: + items: + properties: + matchExpressions: + items: + properties: + key: + type: string + operator: + type: string + values: + items: + type: string + type: array + type: object + type: array + matchFields: + items: + properties: + key: + type: string + operator: + type: string + values: + items: + type: string + type: array + type: object + type: array + type: object + type: array + type: object + type: object + podAffinity: + properties: + preferredDuringSchedulingIgnoredDuringExecution: + items: + properties: + podAffinityTerm: + properties: + labelSelector: + properties: + matchExpressions: + items: + properties: + key: + type: string + operator: + type: string + values: + items: + type: string + type: array + type: object + type: array + matchLabels: + additionalProperties: + type: string + type: object + type: object + matchLabelKeys: + items: + type: string + type: array + mismatchLabelKeys: + items: + type: string + type: array + namespaceSelector: + properties: + matchExpressions: + items: + properties: + key: + type: string + operator: + type: string + values: + items: + type: string + type: array + type: object + type: array + matchLabels: + additionalProperties: + type: string + type: object + type: object + namespaces: + items: + type: string + type: array + topologyKey: + type: string + type: object + weight: + type: integer + type: object + type: array + requiredDuringSchedulingIgnoredDuringExecution: + items: + properties: + labelSelector: + properties: + matchExpressions: + items: + properties: + key: + type: string + operator: + type: string + values: + items: + type: string + type: array + type: object + type: array + matchLabels: + additionalProperties: + type: string + type: object + type: object + matchLabelKeys: + items: + type: string + type: array + mismatchLabelKeys: + items: + type: string + type: array + namespaceSelector: + properties: + matchExpressions: + items: + properties: + key: + type: string + operator: + type: string + values: + items: + type: string + type: array + type: object + type: array + matchLabels: + additionalProperties: + type: string + type: object + type: object + namespaces: + items: + type: string + type: array + topologyKey: + type: string + type: object + type: array + type: object + podAntiAffinity: + properties: + preferredDuringSchedulingIgnoredDuringExecution: + items: + properties: + podAffinityTerm: + properties: + labelSelector: + properties: + matchExpressions: + items: + properties: + key: + type: string + operator: + type: string + values: + items: + type: string + type: array + type: object + type: array + matchLabels: + additionalProperties: + type: string + type: object + type: object + matchLabelKeys: + items: + type: string + type: array + mismatchLabelKeys: + items: + type: string + type: array + namespaceSelector: + properties: + matchExpressions: + items: + properties: + key: + type: string + operator: + type: string + values: + items: + type: string + type: array + type: object + type: array + matchLabels: + additionalProperties: + type: string + type: object + type: object + namespaces: + items: + type: string + type: array + topologyKey: + type: string + type: object + weight: + type: integer + type: object + type: array + requiredDuringSchedulingIgnoredDuringExecution: + items: + properties: + labelSelector: + properties: + matchExpressions: + items: + properties: + key: + type: string + operator: + type: string + values: + items: + type: string + type: array + type: object + type: array + matchLabels: + additionalProperties: + type: string + type: object + type: object + matchLabelKeys: + items: + type: string + type: array + mismatchLabelKeys: + items: + type: string + type: array + namespaceSelector: + properties: + matchExpressions: + items: + properties: + key: + type: string + operator: + type: string + values: + items: + type: string + type: array + type: object + type: array + matchLabels: + additionalProperties: + type: string + type: object + type: object + namespaces: + items: + type: string + type: array + topologyKey: + type: string + type: object + type: array + type: object + type: object + enableServiceLinks: + description: Indicates whether information about services should be injected into Pod's environment variables. + type: boolean + hostAliases: + description: The pod's HostAliases. HostAliases is an optional list of hosts and IPs that will be injected into the Pod's hosts file if specified. + items: + properties: + hostnames: + items: + type: string + type: array + ip: + type: string + type: object + type: array + imagePullSecrets: + description: List of references to secrets in the same namespace to use for pulling any of the images used by this Pod. When the `STRIMZI_IMAGE_PULL_SECRETS` environment variable in Cluster Operator and the `imagePullSecrets` option are specified, only the `imagePullSecrets` variable is used and the `STRIMZI_IMAGE_PULL_SECRETS` variable is ignored. + items: + properties: + name: + type: string + type: object + type: array + metadata: + description: Metadata applied to the resource. + properties: + annotations: + additionalProperties: + type: string + description: Annotations added to the Kubernetes resource. + type: object + labels: + additionalProperties: + type: string + description: Labels added to the Kubernetes resource. + type: object + type: object + priorityClassName: + description: 'The name of the priority class used to assign priority to the pods. ' + type: string + schedulerName: + description: The name of the scheduler used to dispatch this `Pod`. If not specified, the default scheduler will be used. + type: string + securityContext: + description: Configures pod-level security attributes and common container settings. + properties: + appArmorProfile: + properties: + localhostProfile: + type: string + type: + type: string + type: object + fsGroup: + type: integer + fsGroupChangePolicy: + type: string + runAsGroup: + type: integer + runAsNonRoot: + type: boolean + runAsUser: + type: integer + seLinuxOptions: + properties: + level: + type: string + role: + type: string + type: + type: string + user: + type: string + type: object + seccompProfile: + properties: + localhostProfile: + type: string + type: + type: string + type: object + supplementalGroups: + items: + type: integer + type: array + sysctls: + items: + properties: + name: + type: string + value: + type: string + type: object + type: array + windowsOptions: + properties: + gmsaCredentialSpec: + type: string + gmsaCredentialSpecName: + type: string + hostProcess: + type: boolean + runAsUserName: + type: string + type: object + type: object + terminationGracePeriodSeconds: + description: The grace period is the duration in seconds after the processes running in the pod are sent a termination signal, and the time when the processes are forcibly halted with a kill signal. Set this value to longer than the expected cleanup time for your process. Value must be a non-negative integer. A zero value indicates delete immediately. You might need to increase the grace period for very large Kafka clusters, so that the Kafka brokers have enough time to transfer their work to another broker before they are terminated. Defaults to 30 seconds. + minimum: 0 + type: integer + tmpDirSizeLimit: + description: Defines the total amount of pod memory allocated for the temporary `EmptyDir` volume `/tmp`. Specify the allocation in memory units, for example, `100Mi` for 100 mebibytes. Default value is `5Mi`. The `/tmp` volume is backed by pod memory, not disk storage, so avoid setting a high value as it consumes pod memory resources. + pattern: ^([0-9.]+)([eEinumkKMGTP]*[-+]?[0-9]*)$ + type: string + tolerations: + description: The pod's tolerations. + items: + properties: + effect: + type: string + key: + type: string + operator: + type: string + tolerationSeconds: + type: integer + value: + type: string + type: object + type: array + topologySpreadConstraints: + description: The pod's topology spread constraints. + items: + properties: + labelSelector: + properties: + matchExpressions: + items: + properties: + key: + type: string + operator: + type: string + values: + items: + type: string + type: array + type: object + type: array + matchLabels: + additionalProperties: + type: string + type: object + type: object + matchLabelKeys: + items: + type: string + type: array + maxSkew: + type: integer + minDomains: + type: integer + nodeAffinityPolicy: + type: string + nodeTaintsPolicy: + type: string + topologyKey: + type: string + whenUnsatisfiable: + type: string + type: object + type: array + volumes: + description: Additional volumes that can be mounted to the pod. + items: + oneOf: + - properties: + configMap: {} + csi: {} + emptyDir: {} + persistentVolumeClaim: {} + secret: {} + properties: + configMap: + description: ConfigMap to use to populate the volume. + properties: + defaultMode: + type: integer + items: + items: + properties: + key: + type: string + mode: + type: integer + path: + type: string + type: object + type: array + name: + type: string + optional: + type: boolean + type: object + csi: + description: CSIVolumeSource object to use to populate the volume. + properties: + driver: + type: string + fsType: + type: string + nodePublishSecretRef: + properties: + name: + type: string + type: object + readOnly: + type: boolean + volumeAttributes: + additionalProperties: + type: string + type: object + type: object + emptyDir: + description: EmptyDir to use to populate the volume. + properties: + medium: + type: string + sizeLimit: + properties: + amount: + type: string + format: + type: string + type: object + type: object + name: + description: Name to use for the volume. Required. + type: string + persistentVolumeClaim: + description: PersistentVolumeClaim object to use to populate the volume. + properties: + claimName: + type: string + readOnly: + type: boolean + type: object + secret: + description: Secret to use populate the volume. + properties: + defaultMode: + type: integer + items: + items: + properties: + key: + type: string + mode: + type: integer + path: + type: string + type: object + type: array + optional: + type: boolean + secretName: + type: string + type: object + type: object + type: array + type: object + podDisruptionBudget: + description: Template for Cruise Control `PodDisruptionBudget`. + properties: + maxUnavailable: + description: Maximum number of unavailable pods to allow automatic Pod eviction. A Pod eviction is allowed when the `maxUnavailable` number of pods or fewer are unavailable after the eviction. Setting this value to 0 prevents all voluntary evictions, so the pods must be evicted manually. Defaults to 1. + minimum: 0 + type: integer + metadata: + description: Metadata to apply to the `PodDisruptionBudgetTemplate` resource. + properties: + annotations: + additionalProperties: + type: string + description: Annotations added to the Kubernetes resource. + type: object + labels: + additionalProperties: + type: string + description: Labels added to the Kubernetes resource. + type: object + type: object + type: object + serviceAccount: + description: Template for the Cruise Control service account. + properties: + metadata: + description: Metadata applied to the resource. + properties: + annotations: + additionalProperties: + type: string + description: Annotations added to the Kubernetes resource. + type: object + labels: + additionalProperties: + type: string + description: Labels added to the Kubernetes resource. + type: object + type: object + type: object + tlsSidecarContainer: + description: Template for the Cruise Control TLS sidecar container. + properties: + env: + description: Environment variables which should be applied to the container. + items: + oneOf: + - properties: + value: {} + required: + - value + - properties: + valueFrom: {} + required: + - valueFrom + properties: + name: + description: The environment variable key. + type: string + value: + description: The environment variable value. + type: string + valueFrom: + description: Reference to the secret or config map property to which the environment variable is set. + oneOf: + - properties: + secretKeyRef: {} + required: + - secretKeyRef + - properties: + configMapKeyRef: {} + required: + - configMapKeyRef + properties: + configMapKeyRef: + description: Reference to a key in a config map. + properties: + key: + type: string + name: + type: string + optional: + type: boolean + type: object + secretKeyRef: + description: Reference to a key in a secret. + properties: + key: + type: string + name: + type: string + optional: + type: boolean + type: object + type: object + type: object + type: array + securityContext: + description: Security context for the container. + properties: + allowPrivilegeEscalation: + type: boolean + appArmorProfile: + properties: + localhostProfile: + type: string + type: + type: string + type: object + capabilities: + properties: + add: + items: + type: string + type: array + drop: + items: + type: string + type: array + type: object + privileged: + type: boolean + procMount: + type: string + readOnlyRootFilesystem: + type: boolean + runAsGroup: + type: integer + runAsNonRoot: + type: boolean + runAsUser: + type: integer + seLinuxOptions: + properties: + level: + type: string + role: + type: string + type: + type: string + user: + type: string + type: object + seccompProfile: + properties: + localhostProfile: + type: string + type: + type: string + type: object + windowsOptions: + properties: + gmsaCredentialSpec: + type: string + gmsaCredentialSpecName: + type: string + hostProcess: + type: boolean + runAsUserName: + type: string + type: object + type: object + volumeMounts: + description: Additional volume mounts which should be applied to the container. + items: + properties: + mountPath: + type: string + mountPropagation: + type: string + name: + type: string + readOnly: + type: boolean + recursiveReadOnly: + type: string + subPath: + type: string + subPathExpr: + type: string + type: object + type: array + type: object + type: object + tlsSidecar: + description: TLS sidecar configuration. + properties: + image: + description: The docker image for the container. + type: string + livenessProbe: + description: Pod liveness checking. + properties: + failureThreshold: + description: Minimum consecutive failures for the probe to be considered failed after having succeeded. Defaults to 3. Minimum value is 1. + minimum: 1 + type: integer + initialDelaySeconds: + description: The initial delay before first the health is first checked. Default to 15 seconds. Minimum value is 0. + minimum: 0 + type: integer + periodSeconds: + description: How often (in seconds) to perform the probe. Default to 10 seconds. Minimum value is 1. + minimum: 1 + type: integer + successThreshold: + description: Minimum consecutive successes for the probe to be considered successful after having failed. Defaults to 1. Must be 1 for liveness. Minimum value is 1. + minimum: 1 + type: integer + timeoutSeconds: + description: The timeout for each attempted health check. Default to 5 seconds. Minimum value is 1. + minimum: 1 + type: integer + type: object + logLevel: + description: The log level for the TLS sidecar. Default value is `notice`. + enum: + - emerg + - alert + - crit + - err + - warning + - notice + - info + - debug + type: string + readinessProbe: + description: Pod readiness checking. + properties: + failureThreshold: + description: Minimum consecutive failures for the probe to be considered failed after having succeeded. Defaults to 3. Minimum value is 1. + minimum: 1 + type: integer + initialDelaySeconds: + description: The initial delay before first the health is first checked. Default to 15 seconds. Minimum value is 0. + minimum: 0 + type: integer + periodSeconds: + description: How often (in seconds) to perform the probe. Default to 10 seconds. Minimum value is 1. + minimum: 1 + type: integer + successThreshold: + description: Minimum consecutive successes for the probe to be considered successful after having failed. Defaults to 1. Must be 1 for liveness. Minimum value is 1. + minimum: 1 + type: integer + timeoutSeconds: + description: The timeout for each attempted health check. Default to 5 seconds. Minimum value is 1. + minimum: 1 + type: integer + type: object + resources: + description: CPU and memory resources to reserve. + properties: + claims: + items: + properties: + name: + type: string + type: object + type: array + limits: + additionalProperties: + anyOf: + - type: integer + - type: string + pattern: ^(\+|-)?(([0-9]+(\.[0-9]*)?)|(\.[0-9]+))(([KMGTPE]i)|[numkMGTPE]|([eE](\+|-)?(([0-9]+(\.[0-9]*)?)|(\.[0-9]+))))?$ + x-kubernetes-int-or-string: true + type: object + requests: + additionalProperties: + anyOf: + - type: integer + - type: string + pattern: ^(\+|-)?(([0-9]+(\.[0-9]*)?)|(\.[0-9]+))(([KMGTPE]i)|[numkMGTPE]|([eE](\+|-)?(([0-9]+(\.[0-9]*)?)|(\.[0-9]+))))?$ + x-kubernetes-int-or-string: true + type: object + type: object + type: object + type: object + entityOperator: + description: Configuration of the Entity Operator. + properties: + template: + description: Template for Entity Operator resources. The template allows users to specify how a `Deployment` and `Pod` is generated. + properties: + deployment: + description: Template for Entity Operator `Deployment`. + properties: + deploymentStrategy: + description: Pod replacement strategy for deployment configuration changes. Valid values are `RollingUpdate` and `Recreate`. Defaults to `RollingUpdate`. + enum: + - RollingUpdate + - Recreate + type: string + metadata: + description: Metadata applied to the resource. + properties: + annotations: + additionalProperties: + type: string + description: Annotations added to the Kubernetes resource. + type: object + labels: + additionalProperties: + type: string + description: Labels added to the Kubernetes resource. + type: object + type: object + type: object + entityOperatorRole: + description: Template for the Entity Operator Role. + properties: + metadata: + description: Metadata applied to the resource. + properties: + annotations: + additionalProperties: + type: string + description: Annotations added to the Kubernetes resource. + type: object + labels: + additionalProperties: + type: string + description: Labels added to the Kubernetes resource. + type: object + type: object + type: object + pod: + description: Template for Entity Operator `Pods`. + properties: + affinity: + description: The pod's affinity rules. + properties: + nodeAffinity: + properties: + preferredDuringSchedulingIgnoredDuringExecution: + items: + properties: + preference: + properties: + matchExpressions: + items: + properties: + key: + type: string + operator: + type: string + values: + items: + type: string + type: array + type: object + type: array + matchFields: + items: + properties: + key: + type: string + operator: + type: string + values: + items: + type: string + type: array + type: object + type: array + type: object + weight: + type: integer + type: object + type: array + requiredDuringSchedulingIgnoredDuringExecution: + properties: + nodeSelectorTerms: + items: + properties: + matchExpressions: + items: + properties: + key: + type: string + operator: + type: string + values: + items: + type: string + type: array + type: object + type: array + matchFields: + items: + properties: + key: + type: string + operator: + type: string + values: + items: + type: string + type: array + type: object + type: array + type: object + type: array + type: object + type: object + podAffinity: + properties: + preferredDuringSchedulingIgnoredDuringExecution: + items: + properties: + podAffinityTerm: + properties: + labelSelector: + properties: + matchExpressions: + items: + properties: + key: + type: string + operator: + type: string + values: + items: + type: string + type: array + type: object + type: array + matchLabels: + additionalProperties: + type: string + type: object + type: object + matchLabelKeys: + items: + type: string + type: array + mismatchLabelKeys: + items: + type: string + type: array + namespaceSelector: + properties: + matchExpressions: + items: + properties: + key: + type: string + operator: + type: string + values: + items: + type: string + type: array + type: object + type: array + matchLabels: + additionalProperties: + type: string + type: object + type: object + namespaces: + items: + type: string + type: array + topologyKey: + type: string + type: object + weight: + type: integer + type: object + type: array + requiredDuringSchedulingIgnoredDuringExecution: + items: + properties: + labelSelector: + properties: + matchExpressions: + items: + properties: + key: + type: string + operator: + type: string + values: + items: + type: string + type: array + type: object + type: array + matchLabels: + additionalProperties: + type: string + type: object + type: object + matchLabelKeys: + items: + type: string + type: array + mismatchLabelKeys: + items: + type: string + type: array + namespaceSelector: + properties: + matchExpressions: + items: + properties: + key: + type: string + operator: + type: string + values: + items: + type: string + type: array + type: object + type: array + matchLabels: + additionalProperties: + type: string + type: object + type: object + namespaces: + items: + type: string + type: array + topologyKey: + type: string + type: object + type: array + type: object + podAntiAffinity: + properties: + preferredDuringSchedulingIgnoredDuringExecution: + items: + properties: + podAffinityTerm: + properties: + labelSelector: + properties: + matchExpressions: + items: + properties: + key: + type: string + operator: + type: string + values: + items: + type: string + type: array + type: object + type: array + matchLabels: + additionalProperties: + type: string + type: object + type: object + matchLabelKeys: + items: + type: string + type: array + mismatchLabelKeys: + items: + type: string + type: array + namespaceSelector: + properties: + matchExpressions: + items: + properties: + key: + type: string + operator: + type: string + values: + items: + type: string + type: array + type: object + type: array + matchLabels: + additionalProperties: + type: string + type: object + type: object + namespaces: + items: + type: string + type: array + topologyKey: + type: string + type: object + weight: + type: integer + type: object + type: array + requiredDuringSchedulingIgnoredDuringExecution: + items: + properties: + labelSelector: + properties: + matchExpressions: + items: + properties: + key: + type: string + operator: + type: string + values: + items: + type: string + type: array + type: object + type: array + matchLabels: + additionalProperties: + type: string + type: object + type: object + matchLabelKeys: + items: + type: string + type: array + mismatchLabelKeys: + items: + type: string + type: array + namespaceSelector: + properties: + matchExpressions: + items: + properties: + key: + type: string + operator: + type: string + values: + items: + type: string + type: array + type: object + type: array + matchLabels: + additionalProperties: + type: string + type: object + type: object + namespaces: + items: + type: string + type: array + topologyKey: + type: string + type: object + type: array + type: object + type: object + enableServiceLinks: + description: Indicates whether information about services should be injected into Pod's environment variables. + type: boolean + hostAliases: + description: The pod's HostAliases. HostAliases is an optional list of hosts and IPs that will be injected into the Pod's hosts file if specified. + items: + properties: + hostnames: + items: + type: string + type: array + ip: + type: string + type: object + type: array + imagePullSecrets: + description: List of references to secrets in the same namespace to use for pulling any of the images used by this Pod. When the `STRIMZI_IMAGE_PULL_SECRETS` environment variable in Cluster Operator and the `imagePullSecrets` option are specified, only the `imagePullSecrets` variable is used and the `STRIMZI_IMAGE_PULL_SECRETS` variable is ignored. + items: + properties: + name: + type: string + type: object + type: array + metadata: + description: Metadata applied to the resource. + properties: + annotations: + additionalProperties: + type: string + description: Annotations added to the Kubernetes resource. + type: object + labels: + additionalProperties: + type: string + description: Labels added to the Kubernetes resource. + type: object + type: object + priorityClassName: + description: 'The name of the priority class used to assign priority to the pods. ' + type: string + schedulerName: + description: The name of the scheduler used to dispatch this `Pod`. If not specified, the default scheduler will be used. + type: string + securityContext: + description: Configures pod-level security attributes and common container settings. + properties: + appArmorProfile: + properties: + localhostProfile: + type: string + type: + type: string + type: object + fsGroup: + type: integer + fsGroupChangePolicy: + type: string + runAsGroup: + type: integer + runAsNonRoot: + type: boolean + runAsUser: + type: integer + seLinuxOptions: + properties: + level: + type: string + role: + type: string + type: + type: string + user: + type: string + type: object + seccompProfile: + properties: + localhostProfile: + type: string + type: + type: string + type: object + supplementalGroups: + items: + type: integer + type: array + sysctls: + items: + properties: + name: + type: string + value: + type: string + type: object + type: array + windowsOptions: + properties: + gmsaCredentialSpec: + type: string + gmsaCredentialSpecName: + type: string + hostProcess: + type: boolean + runAsUserName: + type: string + type: object + type: object + terminationGracePeriodSeconds: + description: The grace period is the duration in seconds after the processes running in the pod are sent a termination signal, and the time when the processes are forcibly halted with a kill signal. Set this value to longer than the expected cleanup time for your process. Value must be a non-negative integer. A zero value indicates delete immediately. You might need to increase the grace period for very large Kafka clusters, so that the Kafka brokers have enough time to transfer their work to another broker before they are terminated. Defaults to 30 seconds. + minimum: 0 + type: integer + tmpDirSizeLimit: + description: Defines the total amount of pod memory allocated for the temporary `EmptyDir` volume `/tmp`. Specify the allocation in memory units, for example, `100Mi` for 100 mebibytes. Default value is `5Mi`. The `/tmp` volume is backed by pod memory, not disk storage, so avoid setting a high value as it consumes pod memory resources. + pattern: ^([0-9.]+)([eEinumkKMGTP]*[-+]?[0-9]*)$ + type: string + tolerations: + description: The pod's tolerations. + items: + properties: + effect: + type: string + key: + type: string + operator: + type: string + tolerationSeconds: + type: integer + value: + type: string + type: object + type: array + topologySpreadConstraints: + description: The pod's topology spread constraints. + items: + properties: + labelSelector: + properties: + matchExpressions: + items: + properties: + key: + type: string + operator: + type: string + values: + items: + type: string + type: array + type: object + type: array + matchLabels: + additionalProperties: + type: string + type: object + type: object + matchLabelKeys: + items: + type: string + type: array + maxSkew: + type: integer + minDomains: + type: integer + nodeAffinityPolicy: + type: string + nodeTaintsPolicy: + type: string + topologyKey: + type: string + whenUnsatisfiable: + type: string + type: object + type: array + volumes: + description: Additional volumes that can be mounted to the pod. + items: + oneOf: + - properties: + configMap: {} + csi: {} + emptyDir: {} + persistentVolumeClaim: {} + secret: {} + properties: + configMap: + description: ConfigMap to use to populate the volume. + properties: + defaultMode: + type: integer + items: + items: + properties: + key: + type: string + mode: + type: integer + path: + type: string + type: object + type: array + name: + type: string + optional: + type: boolean + type: object + csi: + description: CSIVolumeSource object to use to populate the volume. + properties: + driver: + type: string + fsType: + type: string + nodePublishSecretRef: + properties: + name: + type: string + type: object + readOnly: + type: boolean + volumeAttributes: + additionalProperties: + type: string + type: object + type: object + emptyDir: + description: EmptyDir to use to populate the volume. + properties: + medium: + type: string + sizeLimit: + properties: + amount: + type: string + format: + type: string + type: object + type: object + name: + description: Name to use for the volume. Required. + type: string + persistentVolumeClaim: + description: PersistentVolumeClaim object to use to populate the volume. + properties: + claimName: + type: string + readOnly: + type: boolean + type: object + secret: + description: Secret to use populate the volume. + properties: + defaultMode: + type: integer + items: + items: + properties: + key: + type: string + mode: + type: integer + path: + type: string + type: object + type: array + optional: + type: boolean + secretName: + type: string + type: object + type: object + type: array + type: object + serviceAccount: + description: Template for the Entity Operator service account. + properties: + metadata: + description: Metadata applied to the resource. + properties: + annotations: + additionalProperties: + type: string + description: Annotations added to the Kubernetes resource. + type: object + labels: + additionalProperties: + type: string + description: Labels added to the Kubernetes resource. + type: object + type: object + type: object + tlsSidecarContainer: + description: Template for the Entity Operator TLS sidecar container. + properties: + env: + description: Environment variables which should be applied to the container. + items: + oneOf: + - properties: + value: {} + required: + - value + - properties: + valueFrom: {} + required: + - valueFrom + properties: + name: + description: The environment variable key. + type: string + value: + description: The environment variable value. + type: string + valueFrom: + description: Reference to the secret or config map property to which the environment variable is set. + oneOf: + - properties: + secretKeyRef: {} + required: + - secretKeyRef + - properties: + configMapKeyRef: {} + required: + - configMapKeyRef + properties: + configMapKeyRef: + description: Reference to a key in a config map. + properties: + key: + type: string + name: + type: string + optional: + type: boolean + type: object + secretKeyRef: + description: Reference to a key in a secret. + properties: + key: + type: string + name: + type: string + optional: + type: boolean + type: object + type: object + type: object + type: array + securityContext: + description: Security context for the container. + properties: + allowPrivilegeEscalation: + type: boolean + appArmorProfile: + properties: + localhostProfile: + type: string + type: + type: string + type: object + capabilities: + properties: + add: + items: + type: string + type: array + drop: + items: + type: string + type: array + type: object + privileged: + type: boolean + procMount: + type: string + readOnlyRootFilesystem: + type: boolean + runAsGroup: + type: integer + runAsNonRoot: + type: boolean + runAsUser: + type: integer + seLinuxOptions: + properties: + level: + type: string + role: + type: string + type: + type: string + user: + type: string + type: object + seccompProfile: + properties: + localhostProfile: + type: string + type: + type: string + type: object + windowsOptions: + properties: + gmsaCredentialSpec: + type: string + gmsaCredentialSpecName: + type: string + hostProcess: + type: boolean + runAsUserName: + type: string + type: object + type: object + volumeMounts: + description: Additional volume mounts which should be applied to the container. + items: + properties: + mountPath: + type: string + mountPropagation: + type: string + name: + type: string + readOnly: + type: boolean + recursiveReadOnly: + type: string + subPath: + type: string + subPathExpr: + type: string + type: object + type: array + type: object + topicOperatorContainer: + description: Template for the Entity Topic Operator container. + properties: + env: + description: Environment variables which should be applied to the container. + items: + oneOf: + - properties: + value: {} + required: + - value + - properties: + valueFrom: {} + required: + - valueFrom + properties: + name: + description: The environment variable key. + type: string + value: + description: The environment variable value. + type: string + valueFrom: + description: Reference to the secret or config map property to which the environment variable is set. + oneOf: + - properties: + secretKeyRef: {} + required: + - secretKeyRef + - properties: + configMapKeyRef: {} + required: + - configMapKeyRef + properties: + configMapKeyRef: + description: Reference to a key in a config map. + properties: + key: + type: string + name: + type: string + optional: + type: boolean + type: object + secretKeyRef: + description: Reference to a key in a secret. + properties: + key: + type: string + name: + type: string + optional: + type: boolean + type: object + type: object + type: object + type: array + securityContext: + description: Security context for the container. + properties: + allowPrivilegeEscalation: + type: boolean + appArmorProfile: + properties: + localhostProfile: + type: string + type: + type: string + type: object + capabilities: + properties: + add: + items: + type: string + type: array + drop: + items: + type: string + type: array + type: object + privileged: + type: boolean + procMount: + type: string + readOnlyRootFilesystem: + type: boolean + runAsGroup: + type: integer + runAsNonRoot: + type: boolean + runAsUser: + type: integer + seLinuxOptions: + properties: + level: + type: string + role: + type: string + type: + type: string + user: + type: string + type: object + seccompProfile: + properties: + localhostProfile: + type: string + type: + type: string + type: object + windowsOptions: + properties: + gmsaCredentialSpec: + type: string + gmsaCredentialSpecName: + type: string + hostProcess: + type: boolean + runAsUserName: + type: string + type: object + type: object + volumeMounts: + description: Additional volume mounts which should be applied to the container. + items: + properties: + mountPath: + type: string + mountPropagation: + type: string + name: + type: string + readOnly: + type: boolean + recursiveReadOnly: + type: string + subPath: + type: string + subPathExpr: + type: string + type: object + type: array + type: object + topicOperatorRoleBinding: + description: Template for the Entity Topic Operator RoleBinding. + properties: + metadata: + description: Metadata applied to the resource. + properties: + annotations: + additionalProperties: + type: string + description: Annotations added to the Kubernetes resource. + type: object + labels: + additionalProperties: + type: string + description: Labels added to the Kubernetes resource. + type: object + type: object + type: object + userOperatorContainer: + description: Template for the Entity User Operator container. + properties: + env: + description: Environment variables which should be applied to the container. + items: + oneOf: + - properties: + value: {} + required: + - value + - properties: + valueFrom: {} + required: + - valueFrom + properties: + name: + description: The environment variable key. + type: string + value: + description: The environment variable value. + type: string + valueFrom: + description: Reference to the secret or config map property to which the environment variable is set. + oneOf: + - properties: + secretKeyRef: {} + required: + - secretKeyRef + - properties: + configMapKeyRef: {} + required: + - configMapKeyRef + properties: + configMapKeyRef: + description: Reference to a key in a config map. + properties: + key: + type: string + name: + type: string + optional: + type: boolean + type: object + secretKeyRef: + description: Reference to a key in a secret. + properties: + key: + type: string + name: + type: string + optional: + type: boolean + type: object + type: object + type: object + type: array + securityContext: + description: Security context for the container. + properties: + allowPrivilegeEscalation: + type: boolean + appArmorProfile: + properties: + localhostProfile: + type: string + type: + type: string + type: object + capabilities: + properties: + add: + items: + type: string + type: array + drop: + items: + type: string + type: array + type: object + privileged: + type: boolean + procMount: + type: string + readOnlyRootFilesystem: + type: boolean + runAsGroup: + type: integer + runAsNonRoot: + type: boolean + runAsUser: + type: integer + seLinuxOptions: + properties: + level: + type: string + role: + type: string + type: + type: string + user: + type: string + type: object + seccompProfile: + properties: + localhostProfile: + type: string + type: + type: string + type: object + windowsOptions: + properties: + gmsaCredentialSpec: + type: string + gmsaCredentialSpecName: + type: string + hostProcess: + type: boolean + runAsUserName: + type: string + type: object + type: object + volumeMounts: + description: Additional volume mounts which should be applied to the container. + items: + properties: + mountPath: + type: string + mountPropagation: + type: string + name: + type: string + readOnly: + type: boolean + recursiveReadOnly: + type: string + subPath: + type: string + subPathExpr: + type: string + type: object + type: array + type: object + userOperatorRoleBinding: + description: Template for the Entity Topic Operator RoleBinding. + properties: + metadata: + description: Metadata applied to the resource. + properties: + annotations: + additionalProperties: + type: string + description: Annotations added to the Kubernetes resource. + type: object + labels: + additionalProperties: + type: string + description: Labels added to the Kubernetes resource. + type: object + type: object + type: object + type: object + tlsSidecar: + description: TLS sidecar configuration. + properties: + image: + description: The docker image for the container. + type: string + livenessProbe: + description: Pod liveness checking. + properties: + failureThreshold: + description: Minimum consecutive failures for the probe to be considered failed after having succeeded. Defaults to 3. Minimum value is 1. + minimum: 1 + type: integer + initialDelaySeconds: + description: The initial delay before first the health is first checked. Default to 15 seconds. Minimum value is 0. + minimum: 0 + type: integer + periodSeconds: + description: How often (in seconds) to perform the probe. Default to 10 seconds. Minimum value is 1. + minimum: 1 + type: integer + successThreshold: + description: Minimum consecutive successes for the probe to be considered successful after having failed. Defaults to 1. Must be 1 for liveness. Minimum value is 1. + minimum: 1 + type: integer + timeoutSeconds: + description: The timeout for each attempted health check. Default to 5 seconds. Minimum value is 1. + minimum: 1 + type: integer + type: object + logLevel: + description: The log level for the TLS sidecar. Default value is `notice`. + enum: + - emerg + - alert + - crit + - err + - warning + - notice + - info + - debug + type: string + readinessProbe: + description: Pod readiness checking. + properties: + failureThreshold: + description: Minimum consecutive failures for the probe to be considered failed after having succeeded. Defaults to 3. Minimum value is 1. + minimum: 1 + type: integer + initialDelaySeconds: + description: The initial delay before first the health is first checked. Default to 15 seconds. Minimum value is 0. + minimum: 0 + type: integer + periodSeconds: + description: How often (in seconds) to perform the probe. Default to 10 seconds. Minimum value is 1. + minimum: 1 + type: integer + successThreshold: + description: Minimum consecutive successes for the probe to be considered successful after having failed. Defaults to 1. Must be 1 for liveness. Minimum value is 1. + minimum: 1 + type: integer + timeoutSeconds: + description: The timeout for each attempted health check. Default to 5 seconds. Minimum value is 1. + minimum: 1 + type: integer + type: object + resources: + description: CPU and memory resources to reserve. + properties: + claims: + items: + properties: + name: + type: string + type: object + type: array + limits: + additionalProperties: + anyOf: + - type: integer + - type: string + pattern: ^(\+|-)?(([0-9]+(\.[0-9]*)?)|(\.[0-9]+))(([KMGTPE]i)|[numkMGTPE]|([eE](\+|-)?(([0-9]+(\.[0-9]*)?)|(\.[0-9]+))))?$ + x-kubernetes-int-or-string: true + type: object + requests: + additionalProperties: + anyOf: + - type: integer + - type: string + pattern: ^(\+|-)?(([0-9]+(\.[0-9]*)?)|(\.[0-9]+))(([KMGTPE]i)|[numkMGTPE]|([eE](\+|-)?(([0-9]+(\.[0-9]*)?)|(\.[0-9]+))))?$ + x-kubernetes-int-or-string: true + type: object + type: object + type: object + topicOperator: + description: Configuration of the Topic Operator. + properties: + image: + description: The image to use for the Topic Operator. + type: string + jvmOptions: + description: JVM Options for pods. + properties: + -XX: + additionalProperties: + type: string + description: A map of -XX options to the JVM. + type: object + -Xms: + description: -Xms option to to the JVM. + pattern: ^[0-9]+[mMgG]?$ + type: string + -Xmx: + description: -Xmx option to to the JVM. + pattern: ^[0-9]+[mMgG]?$ + type: string + gcLoggingEnabled: + description: Specifies whether the Garbage Collection logging is enabled. The default is false. + type: boolean + javaSystemProperties: + description: A map of additional system properties which will be passed using the `-D` option to the JVM. + items: + properties: + name: + description: The system property name. + type: string + value: + description: The system property value. + type: string + type: object + type: array + type: object + livenessProbe: + description: Pod liveness checking. + properties: + failureThreshold: + description: Minimum consecutive failures for the probe to be considered failed after having succeeded. Defaults to 3. Minimum value is 1. + minimum: 1 + type: integer + initialDelaySeconds: + description: The initial delay before first the health is first checked. Default to 15 seconds. Minimum value is 0. + minimum: 0 + type: integer + periodSeconds: + description: How often (in seconds) to perform the probe. Default to 10 seconds. Minimum value is 1. + minimum: 1 + type: integer + successThreshold: + description: Minimum consecutive successes for the probe to be considered successful after having failed. Defaults to 1. Must be 1 for liveness. Minimum value is 1. + minimum: 1 + type: integer + timeoutSeconds: + description: The timeout for each attempted health check. Default to 5 seconds. Minimum value is 1. + minimum: 1 + type: integer + type: object + logging: + description: Logging configuration. + properties: + loggers: + additionalProperties: + type: string + description: A Map from logger name to logger level. + type: object + type: + description: Logging type, must be either 'inline' or 'external'. + enum: + - inline + - external + type: string + valueFrom: + description: '`ConfigMap` entry where the logging configuration is stored. ' + properties: + configMapKeyRef: + description: Reference to the key in the ConfigMap containing the configuration. + properties: + key: + type: string + name: + type: string + optional: + type: boolean + type: object + type: object + required: + - type + type: object + readinessProbe: + description: Pod readiness checking. + properties: + failureThreshold: + description: Minimum consecutive failures for the probe to be considered failed after having succeeded. Defaults to 3. Minimum value is 1. + minimum: 1 + type: integer + initialDelaySeconds: + description: The initial delay before first the health is first checked. Default to 15 seconds. Minimum value is 0. + minimum: 0 + type: integer + periodSeconds: + description: How often (in seconds) to perform the probe. Default to 10 seconds. Minimum value is 1. + minimum: 1 + type: integer + successThreshold: + description: Minimum consecutive successes for the probe to be considered successful after having failed. Defaults to 1. Must be 1 for liveness. Minimum value is 1. + minimum: 1 + type: integer + timeoutSeconds: + description: The timeout for each attempted health check. Default to 5 seconds. Minimum value is 1. + minimum: 1 + type: integer + type: object + reconciliationIntervalMs: + description: Interval between periodic reconciliations in milliseconds. + minimum: 0 + type: integer + reconciliationIntervalSeconds: + description: Interval between periodic reconciliations in seconds. Ignored if reconciliationIntervalMs is set. + minimum: 0 + type: integer + resources: + description: CPU and memory resources to reserve. + properties: + claims: + items: + properties: + name: + type: string + type: object + type: array + limits: + additionalProperties: + anyOf: + - type: integer + - type: string + pattern: ^(\+|-)?(([0-9]+(\.[0-9]*)?)|(\.[0-9]+))(([KMGTPE]i)|[numkMGTPE]|([eE](\+|-)?(([0-9]+(\.[0-9]*)?)|(\.[0-9]+))))?$ + x-kubernetes-int-or-string: true + type: object + requests: + additionalProperties: + anyOf: + - type: integer + - type: string + pattern: ^(\+|-)?(([0-9]+(\.[0-9]*)?)|(\.[0-9]+))(([KMGTPE]i)|[numkMGTPE]|([eE](\+|-)?(([0-9]+(\.[0-9]*)?)|(\.[0-9]+))))?$ + x-kubernetes-int-or-string: true + type: object + type: object + startupProbe: + description: Pod startup checking. + properties: + failureThreshold: + description: Minimum consecutive failures for the probe to be considered failed after having succeeded. Defaults to 3. Minimum value is 1. + minimum: 1 + type: integer + initialDelaySeconds: + description: The initial delay before first the health is first checked. Default to 15 seconds. Minimum value is 0. + minimum: 0 + type: integer + periodSeconds: + description: How often (in seconds) to perform the probe. Default to 10 seconds. Minimum value is 1. + minimum: 1 + type: integer + successThreshold: + description: Minimum consecutive successes for the probe to be considered successful after having failed. Defaults to 1. Must be 1 for liveness. Minimum value is 1. + minimum: 1 + type: integer + timeoutSeconds: + description: The timeout for each attempted health check. Default to 5 seconds. Minimum value is 1. + minimum: 1 + type: integer + type: object + topicMetadataMaxAttempts: + description: The number of attempts at getting topic metadata. + minimum: 0 + type: integer + watchedNamespace: + description: The namespace the Topic Operator should watch. + type: string + zookeeperSessionTimeoutSeconds: + description: Timeout for the ZooKeeper session. + minimum: 0 + type: integer + type: object + userOperator: + description: Configuration of the User Operator. + properties: + image: + description: The image to use for the User Operator. + type: string + jvmOptions: + description: JVM Options for pods. + properties: + -XX: + additionalProperties: + type: string + description: A map of -XX options to the JVM. + type: object + -Xms: + description: -Xms option to to the JVM. + pattern: ^[0-9]+[mMgG]?$ + type: string + -Xmx: + description: -Xmx option to to the JVM. + pattern: ^[0-9]+[mMgG]?$ + type: string + gcLoggingEnabled: + description: Specifies whether the Garbage Collection logging is enabled. The default is false. + type: boolean + javaSystemProperties: + description: A map of additional system properties which will be passed using the `-D` option to the JVM. + items: + properties: + name: + description: The system property name. + type: string + value: + description: The system property value. + type: string + type: object + type: array + type: object + livenessProbe: + description: Pod liveness checking. + properties: + failureThreshold: + description: Minimum consecutive failures for the probe to be considered failed after having succeeded. Defaults to 3. Minimum value is 1. + minimum: 1 + type: integer + initialDelaySeconds: + description: The initial delay before first the health is first checked. Default to 15 seconds. Minimum value is 0. + minimum: 0 + type: integer + periodSeconds: + description: How often (in seconds) to perform the probe. Default to 10 seconds. Minimum value is 1. + minimum: 1 + type: integer + successThreshold: + description: Minimum consecutive successes for the probe to be considered successful after having failed. Defaults to 1. Must be 1 for liveness. Minimum value is 1. + minimum: 1 + type: integer + timeoutSeconds: + description: The timeout for each attempted health check. Default to 5 seconds. Minimum value is 1. + minimum: 1 + type: integer + type: object + logging: + description: Logging configuration. + properties: + loggers: + additionalProperties: + type: string + description: A Map from logger name to logger level. + type: object + type: + description: Logging type, must be either 'inline' or 'external'. + enum: + - inline + - external + type: string + valueFrom: + description: '`ConfigMap` entry where the logging configuration is stored. ' + properties: + configMapKeyRef: + description: Reference to the key in the ConfigMap containing the configuration. + properties: + key: + type: string + name: + type: string + optional: + type: boolean + type: object + type: object + required: + - type + type: object + readinessProbe: + description: Pod readiness checking. + properties: + failureThreshold: + description: Minimum consecutive failures for the probe to be considered failed after having succeeded. Defaults to 3. Minimum value is 1. + minimum: 1 + type: integer + initialDelaySeconds: + description: The initial delay before first the health is first checked. Default to 15 seconds. Minimum value is 0. + minimum: 0 + type: integer + periodSeconds: + description: How often (in seconds) to perform the probe. Default to 10 seconds. Minimum value is 1. + minimum: 1 + type: integer + successThreshold: + description: Minimum consecutive successes for the probe to be considered successful after having failed. Defaults to 1. Must be 1 for liveness. Minimum value is 1. + minimum: 1 + type: integer + timeoutSeconds: + description: The timeout for each attempted health check. Default to 5 seconds. Minimum value is 1. + minimum: 1 + type: integer + type: object + reconciliationIntervalMs: + description: Interval between periodic reconciliations in milliseconds. + minimum: 0 + type: integer + reconciliationIntervalSeconds: + description: Interval between periodic reconciliations in seconds. Ignored if reconciliationIntervalMs is set. + minimum: 0 + type: integer + resources: + description: CPU and memory resources to reserve. + properties: + claims: + items: + properties: + name: + type: string + type: object + type: array + limits: + additionalProperties: + anyOf: + - type: integer + - type: string + pattern: ^(\+|-)?(([0-9]+(\.[0-9]*)?)|(\.[0-9]+))(([KMGTPE]i)|[numkMGTPE]|([eE](\+|-)?(([0-9]+(\.[0-9]*)?)|(\.[0-9]+))))?$ + x-kubernetes-int-or-string: true + type: object + requests: + additionalProperties: + anyOf: + - type: integer + - type: string + pattern: ^(\+|-)?(([0-9]+(\.[0-9]*)?)|(\.[0-9]+))(([KMGTPE]i)|[numkMGTPE]|([eE](\+|-)?(([0-9]+(\.[0-9]*)?)|(\.[0-9]+))))?$ + x-kubernetes-int-or-string: true + type: object + type: object + secretPrefix: + description: The prefix that will be added to the KafkaUser name to be used as the Secret name. + type: string + watchedNamespace: + description: The namespace the User Operator should watch. + type: string + zookeeperSessionTimeoutSeconds: + description: Timeout for the ZooKeeper session. + minimum: 0 + type: integer + type: object + type: object + jmxTrans: + description: As of Strimzi 0.35.0, JMXTrans is not supported anymore and this option is ignored. + properties: + image: + description: The image to use for the JmxTrans. + type: string + kafkaQueries: + description: Queries to send to the Kafka brokers to define what data should be read from each broker. For more information on these properties see, xref:type-JmxTransQueryTemplate-reference[`JmxTransQueryTemplate` schema reference]. + items: + properties: + attributes: + description: Determine which attributes of the targeted MBean should be included. + items: + type: string + type: array + outputs: + description: List of the names of output definitions specified in the spec.kafka.jmxTrans.outputDefinitions that have defined where JMX metrics are pushed to, and in which data format. + items: + type: string + type: array + targetMBean: + description: If using wildcards instead of a specific MBean then the data is gathered from multiple MBeans. Otherwise if specifying an MBean then data is gathered from that specified MBean. + type: string + required: + - targetMBean + - attributes + - outputs + type: object + type: array + logLevel: + description: Sets the logging level of the JmxTrans deployment.For more information see, https://github.com/jmxtrans/jmxtrans-agent/wiki/Troubleshooting[JmxTrans Logging Level]. + type: string + outputDefinitions: + description: Defines the output hosts that will be referenced later on. For more information on these properties see, xref:type-JmxTransOutputDefinitionTemplate-reference[`JmxTransOutputDefinitionTemplate` schema reference]. + items: + properties: + flushDelayInSeconds: + description: How many seconds the JmxTrans waits before pushing a new set of data out. + type: integer + host: + description: The DNS/hostname of the remote host that the data is pushed to. + type: string + name: + description: Template for setting the name of the output definition. This is used to identify where to send the results of queries should be sent. + type: string + outputType: + description: Template for setting the format of the data that will be pushed.For more information see https://github.com/jmxtrans/jmxtrans/wiki/OutputWriters[JmxTrans OutputWriters]. + type: string + port: + description: The port of the remote host that the data is pushed to. + type: integer + typeNames: + description: Template for filtering data to be included in response to a wildcard query. For more information see https://github.com/jmxtrans/jmxtrans/wiki/Queries[JmxTrans queries]. + items: + type: string + type: array + required: + - outputType + - name + type: object + type: array + resources: + description: CPU and memory resources to reserve. + properties: + claims: + items: + properties: + name: + type: string + type: object + type: array + limits: + additionalProperties: + anyOf: + - type: integer + - type: string + pattern: ^(\+|-)?(([0-9]+(\.[0-9]*)?)|(\.[0-9]+))(([KMGTPE]i)|[numkMGTPE]|([eE](\+|-)?(([0-9]+(\.[0-9]*)?)|(\.[0-9]+))))?$ + x-kubernetes-int-or-string: true + type: object + requests: + additionalProperties: + anyOf: + - type: integer + - type: string + pattern: ^(\+|-)?(([0-9]+(\.[0-9]*)?)|(\.[0-9]+))(([KMGTPE]i)|[numkMGTPE]|([eE](\+|-)?(([0-9]+(\.[0-9]*)?)|(\.[0-9]+))))?$ + x-kubernetes-int-or-string: true + type: object + type: object + template: + description: Template for JmxTrans resources. + properties: + container: + description: Template for JmxTrans container. + properties: + env: + description: Environment variables which should be applied to the container. + items: + oneOf: + - properties: + value: {} + required: + - value + - properties: + valueFrom: {} + required: + - valueFrom + properties: + name: + description: The environment variable key. + type: string + value: + description: The environment variable value. + type: string + valueFrom: + description: Reference to the secret or config map property to which the environment variable is set. + oneOf: + - properties: + secretKeyRef: {} + required: + - secretKeyRef + - properties: + configMapKeyRef: {} + required: + - configMapKeyRef + properties: + configMapKeyRef: + description: Reference to a key in a config map. + properties: + key: + type: string + name: + type: string + optional: + type: boolean + type: object + secretKeyRef: + description: Reference to a key in a secret. + properties: + key: + type: string + name: + type: string + optional: + type: boolean + type: object + type: object + type: object + type: array + securityContext: + description: Security context for the container. + properties: + allowPrivilegeEscalation: + type: boolean + appArmorProfile: + properties: + localhostProfile: + type: string + type: + type: string + type: object + capabilities: + properties: + add: + items: + type: string + type: array + drop: + items: + type: string + type: array + type: object + privileged: + type: boolean + procMount: + type: string + readOnlyRootFilesystem: + type: boolean + runAsGroup: + type: integer + runAsNonRoot: + type: boolean + runAsUser: + type: integer + seLinuxOptions: + properties: + level: + type: string + role: + type: string + type: + type: string + user: + type: string + type: object + seccompProfile: + properties: + localhostProfile: + type: string + type: + type: string + type: object + windowsOptions: + properties: + gmsaCredentialSpec: + type: string + gmsaCredentialSpecName: + type: string + hostProcess: + type: boolean + runAsUserName: + type: string + type: object + type: object + volumeMounts: + description: Additional volume mounts which should be applied to the container. + items: + properties: + mountPath: + type: string + mountPropagation: + type: string + name: + type: string + readOnly: + type: boolean + recursiveReadOnly: + type: string + subPath: + type: string + subPathExpr: + type: string + type: object + type: array + type: object + deployment: + description: Template for JmxTrans `Deployment`. + properties: + deploymentStrategy: + description: Pod replacement strategy for deployment configuration changes. Valid values are `RollingUpdate` and `Recreate`. Defaults to `RollingUpdate`. + enum: + - RollingUpdate + - Recreate + type: string + metadata: + description: Metadata applied to the resource. + properties: + annotations: + additionalProperties: + type: string + description: Annotations added to the Kubernetes resource. + type: object + labels: + additionalProperties: + type: string + description: Labels added to the Kubernetes resource. + type: object + type: object + type: object + pod: + description: Template for JmxTrans `Pods`. + properties: + affinity: + description: The pod's affinity rules. + properties: + nodeAffinity: + properties: + preferredDuringSchedulingIgnoredDuringExecution: + items: + properties: + preference: + properties: + matchExpressions: + items: + properties: + key: + type: string + operator: + type: string + values: + items: + type: string + type: array + type: object + type: array + matchFields: + items: + properties: + key: + type: string + operator: + type: string + values: + items: + type: string + type: array + type: object + type: array + type: object + weight: + type: integer + type: object + type: array + requiredDuringSchedulingIgnoredDuringExecution: + properties: + nodeSelectorTerms: + items: + properties: + matchExpressions: + items: + properties: + key: + type: string + operator: + type: string + values: + items: + type: string + type: array + type: object + type: array + matchFields: + items: + properties: + key: + type: string + operator: + type: string + values: + items: + type: string + type: array + type: object + type: array + type: object + type: array + type: object + type: object + podAffinity: + properties: + preferredDuringSchedulingIgnoredDuringExecution: + items: + properties: + podAffinityTerm: + properties: + labelSelector: + properties: + matchExpressions: + items: + properties: + key: + type: string + operator: + type: string + values: + items: + type: string + type: array + type: object + type: array + matchLabels: + additionalProperties: + type: string + type: object + type: object + matchLabelKeys: + items: + type: string + type: array + mismatchLabelKeys: + items: + type: string + type: array + namespaceSelector: + properties: + matchExpressions: + items: + properties: + key: + type: string + operator: + type: string + values: + items: + type: string + type: array + type: object + type: array + matchLabels: + additionalProperties: + type: string + type: object + type: object + namespaces: + items: + type: string + type: array + topologyKey: + type: string + type: object + weight: + type: integer + type: object + type: array + requiredDuringSchedulingIgnoredDuringExecution: + items: + properties: + labelSelector: + properties: + matchExpressions: + items: + properties: + key: + type: string + operator: + type: string + values: + items: + type: string + type: array + type: object + type: array + matchLabels: + additionalProperties: + type: string + type: object + type: object + matchLabelKeys: + items: + type: string + type: array + mismatchLabelKeys: + items: + type: string + type: array + namespaceSelector: + properties: + matchExpressions: + items: + properties: + key: + type: string + operator: + type: string + values: + items: + type: string + type: array + type: object + type: array + matchLabels: + additionalProperties: + type: string + type: object + type: object + namespaces: + items: + type: string + type: array + topologyKey: + type: string + type: object + type: array + type: object + podAntiAffinity: + properties: + preferredDuringSchedulingIgnoredDuringExecution: + items: + properties: + podAffinityTerm: + properties: + labelSelector: + properties: + matchExpressions: + items: + properties: + key: + type: string + operator: + type: string + values: + items: + type: string + type: array + type: object + type: array + matchLabels: + additionalProperties: + type: string + type: object + type: object + matchLabelKeys: + items: + type: string + type: array + mismatchLabelKeys: + items: + type: string + type: array + namespaceSelector: + properties: + matchExpressions: + items: + properties: + key: + type: string + operator: + type: string + values: + items: + type: string + type: array + type: object + type: array + matchLabels: + additionalProperties: + type: string + type: object + type: object + namespaces: + items: + type: string + type: array + topologyKey: + type: string + type: object + weight: + type: integer + type: object + type: array + requiredDuringSchedulingIgnoredDuringExecution: + items: + properties: + labelSelector: + properties: + matchExpressions: + items: + properties: + key: + type: string + operator: + type: string + values: + items: + type: string + type: array + type: object + type: array + matchLabels: + additionalProperties: + type: string + type: object + type: object + matchLabelKeys: + items: + type: string + type: array + mismatchLabelKeys: + items: + type: string + type: array + namespaceSelector: + properties: + matchExpressions: + items: + properties: + key: + type: string + operator: + type: string + values: + items: + type: string + type: array + type: object + type: array + matchLabels: + additionalProperties: + type: string + type: object + type: object + namespaces: + items: + type: string + type: array + topologyKey: + type: string + type: object + type: array + type: object + type: object + enableServiceLinks: + description: Indicates whether information about services should be injected into Pod's environment variables. + type: boolean + hostAliases: + description: The pod's HostAliases. HostAliases is an optional list of hosts and IPs that will be injected into the Pod's hosts file if specified. + items: + properties: + hostnames: + items: + type: string + type: array + ip: + type: string + type: object + type: array + imagePullSecrets: + description: List of references to secrets in the same namespace to use for pulling any of the images used by this Pod. When the `STRIMZI_IMAGE_PULL_SECRETS` environment variable in Cluster Operator and the `imagePullSecrets` option are specified, only the `imagePullSecrets` variable is used and the `STRIMZI_IMAGE_PULL_SECRETS` variable is ignored. + items: + properties: + name: + type: string + type: object + type: array + metadata: + description: Metadata applied to the resource. + properties: + annotations: + additionalProperties: + type: string + description: Annotations added to the Kubernetes resource. + type: object + labels: + additionalProperties: + type: string + description: Labels added to the Kubernetes resource. + type: object + type: object + priorityClassName: + description: 'The name of the priority class used to assign priority to the pods. ' + type: string + schedulerName: + description: The name of the scheduler used to dispatch this `Pod`. If not specified, the default scheduler will be used. + type: string + securityContext: + description: Configures pod-level security attributes and common container settings. + properties: + appArmorProfile: + properties: + localhostProfile: + type: string + type: + type: string + type: object + fsGroup: + type: integer + fsGroupChangePolicy: + type: string + runAsGroup: + type: integer + runAsNonRoot: + type: boolean + runAsUser: + type: integer + seLinuxOptions: + properties: + level: + type: string + role: + type: string + type: + type: string + user: + type: string + type: object + seccompProfile: + properties: + localhostProfile: + type: string + type: + type: string + type: object + supplementalGroups: + items: + type: integer + type: array + sysctls: + items: + properties: + name: + type: string + value: + type: string + type: object + type: array + windowsOptions: + properties: + gmsaCredentialSpec: + type: string + gmsaCredentialSpecName: + type: string + hostProcess: + type: boolean + runAsUserName: + type: string + type: object + type: object + terminationGracePeriodSeconds: + description: The grace period is the duration in seconds after the processes running in the pod are sent a termination signal, and the time when the processes are forcibly halted with a kill signal. Set this value to longer than the expected cleanup time for your process. Value must be a non-negative integer. A zero value indicates delete immediately. You might need to increase the grace period for very large Kafka clusters, so that the Kafka brokers have enough time to transfer their work to another broker before they are terminated. Defaults to 30 seconds. + minimum: 0 + type: integer + tmpDirSizeLimit: + description: Defines the total amount of pod memory allocated for the temporary `EmptyDir` volume `/tmp`. Specify the allocation in memory units, for example, `100Mi` for 100 mebibytes. Default value is `5Mi`. The `/tmp` volume is backed by pod memory, not disk storage, so avoid setting a high value as it consumes pod memory resources. + pattern: ^([0-9.]+)([eEinumkKMGTP]*[-+]?[0-9]*)$ + type: string + tolerations: + description: The pod's tolerations. + items: + properties: + effect: + type: string + key: + type: string + operator: + type: string + tolerationSeconds: + type: integer + value: + type: string + type: object + type: array + topologySpreadConstraints: + description: The pod's topology spread constraints. + items: + properties: + labelSelector: + properties: + matchExpressions: + items: + properties: + key: + type: string + operator: + type: string + values: + items: + type: string + type: array + type: object + type: array + matchLabels: + additionalProperties: + type: string + type: object + type: object + matchLabelKeys: + items: + type: string + type: array + maxSkew: + type: integer + minDomains: + type: integer + nodeAffinityPolicy: + type: string + nodeTaintsPolicy: + type: string + topologyKey: + type: string + whenUnsatisfiable: + type: string + type: object + type: array + volumes: + description: Additional volumes that can be mounted to the pod. + items: + oneOf: + - properties: + configMap: {} + csi: {} + emptyDir: {} + persistentVolumeClaim: {} + secret: {} + properties: + configMap: + description: ConfigMap to use to populate the volume. + properties: + defaultMode: + type: integer + items: + items: + properties: + key: + type: string + mode: + type: integer + path: + type: string + type: object + type: array + name: + type: string + optional: + type: boolean + type: object + csi: + description: CSIVolumeSource object to use to populate the volume. + properties: + driver: + type: string + fsType: + type: string + nodePublishSecretRef: + properties: + name: + type: string + type: object + readOnly: + type: boolean + volumeAttributes: + additionalProperties: + type: string + type: object + type: object + emptyDir: + description: EmptyDir to use to populate the volume. + properties: + medium: + type: string + sizeLimit: + properties: + amount: + type: string + format: + type: string + type: object + type: object + name: + description: Name to use for the volume. Required. + type: string + persistentVolumeClaim: + description: PersistentVolumeClaim object to use to populate the volume. + properties: + claimName: + type: string + readOnly: + type: boolean + type: object + secret: + description: Secret to use populate the volume. + properties: + defaultMode: + type: integer + items: + items: + properties: + key: + type: string + mode: + type: integer + path: + type: string + type: object + type: array + optional: + type: boolean + secretName: + type: string + type: object + type: object + type: array + type: object + serviceAccount: + description: Template for the JmxTrans service account. + properties: + metadata: + description: Metadata applied to the resource. + properties: + annotations: + additionalProperties: + type: string + description: Annotations added to the Kubernetes resource. + type: object + labels: + additionalProperties: + type: string + description: Labels added to the Kubernetes resource. + type: object + type: object + type: object + type: object + required: + - outputDefinitions + - kafkaQueries + type: object + kafka: + description: Configuration of the Kafka cluster. + properties: + authorization: + description: Authorization configuration for Kafka brokers. + properties: + allowOnError: + description: Defines whether a Kafka client should be allowed or denied by default when the authorizer fails to query the Open Policy Agent, for example, when it is temporarily unavailable). Defaults to `false` - all actions will be denied. + type: boolean + authorizerClass: + description: Authorization implementation class, which must be available in classpath. + type: string + clientId: + description: OAuth Client ID which the Kafka client can use to authenticate against the OAuth server and use the token endpoint URI. + type: string + connectTimeoutSeconds: + description: The connect timeout in seconds when connecting to authorization server. If not set, the effective connect timeout is 60 seconds. + minimum: 1 + type: integer + delegateToKafkaAcls: + description: Whether authorization decision should be delegated to the 'Simple' authorizer if DENIED by Keycloak Authorization Services policies. Default value is `false`. + type: boolean + disableTlsHostnameVerification: + description: Enable or disable TLS hostname verification. Default value is `false`. + type: boolean + enableMetrics: + description: Enable or disable OAuth metrics. The default value is `false`. + type: boolean + expireAfterMs: + description: The expiration of the records kept in the local cache to avoid querying the Open Policy Agent for every request. Defines how often the cached authorization decisions are reloaded from the Open Policy Agent server. In milliseconds. Defaults to `3600000`. + type: integer + grantsAlwaysLatest: + description: Controls whether the latest grants are fetched for a new session. When enabled, grants are retrieved from Keycloak and cached for the user. The default value is `false`. + type: boolean + grantsGcPeriodSeconds: + description: The time, in seconds, between consecutive runs of a job that cleans stale grants from the cache. The default value is 300. + minimum: 1 + type: integer + grantsMaxIdleTimeSeconds: + description: The time, in seconds, after which an idle grant can be evicted from the cache. The default value is 300. + minimum: 1 + type: integer + grantsRefreshPeriodSeconds: + description: The time between two consecutive grants refresh runs in seconds. The default value is 60. + minimum: 0 + type: integer + grantsRefreshPoolSize: + description: The number of threads to use to refresh grants for active sessions. The more threads, the more parallelism, so the sooner the job completes. However, using more threads places a heavier load on the authorization server. The default value is 5. + minimum: 1 + type: integer + httpRetries: + description: The maximum number of retries to attempt if an initial HTTP request fails. If not set, the default is to not attempt any retries. + minimum: 0 + type: integer + includeAcceptHeader: + description: Whether the Accept header should be set in requests to the authorization servers. The default value is `true`. + type: boolean + initialCacheCapacity: + description: Initial capacity of the local cache used by the authorizer to avoid querying the Open Policy Agent for every request Defaults to `5000`. + type: integer + maximumCacheSize: + description: Maximum capacity of the local cache used by the authorizer to avoid querying the Open Policy Agent for every request. Defaults to `50000`. + type: integer + readTimeoutSeconds: + description: The read timeout in seconds when connecting to authorization server. If not set, the effective read timeout is 60 seconds. + minimum: 1 + type: integer + superUsers: + description: List of super users, which are user principals with unlimited access rights. + items: + type: string + type: array + supportsAdminApi: + description: Indicates whether the custom authorizer supports the APIs for managing ACLs using the Kafka Admin API. Defaults to `false`. + type: boolean + tlsTrustedCertificates: + description: Trusted certificates for TLS connection to the OAuth server. + items: + oneOf: + - properties: + certificate: {} + required: + - certificate + - properties: + pattern: {} + required: + - pattern + properties: + certificate: + description: The name of the file certificate in the secret. + type: string + pattern: + description: Pattern for the certificate files in the secret. Use the link:https://en.wikipedia.org/wiki/Glob_(programming)[_glob syntax_] for the pattern. All files in the secret that match the pattern are used. + type: string + secretName: + description: The name of the Secret containing the certificate. + type: string + required: + - secretName + type: object + type: array + tokenEndpointUri: + description: Authorization server token endpoint URI. + type: string + type: + description: Authorization type. Currently, the supported types are `simple`, `keycloak`, `opa` and `custom`. `simple` authorization type uses Kafka's built-in authorizer for authorization. `keycloak` authorization type uses Keycloak Authorization Services for authorization. `opa` authorization type uses Open Policy Agent based authorization.`custom` authorization type uses user-provided implementation for authorization. + enum: + - simple + - opa + - keycloak + - custom + type: string + url: + description: The URL used to connect to the Open Policy Agent server. The URL has to include the policy which will be queried by the authorizer. This option is required. + example: http://opa:8181/v1/data/kafka/authz/allow + type: string + required: + - type + type: object + brokerRackInitImage: + description: The image of the init container used for initializing the `broker.rack`. + type: string + config: + description: 'Kafka broker config properties with the following prefixes cannot be set: listeners, advertised., broker., listener., host.name, port, inter.broker.listener.name, sasl., ssl., security., password., log.dir, zookeeper.connect, zookeeper.set.acl, zookeeper.ssl, zookeeper.clientCnxnSocket, authorizer., super.user, cruise.control.metrics.topic, cruise.control.metrics.reporter.bootstrap.servers, node.id, process.roles, controller., metadata.log.dir, zookeeper.metadata.migration.enable, client.quota.callback.static.kafka.admin., client.quota.callback.static.produce, client.quota.callback.static.fetch, client.quota.callback.static.storage.per.volume.limit.min.available., client.quota.callback.static.excluded.principal.name.list (with the exception of: zookeeper.connection.timeout.ms, sasl.server.max.receive.size, ssl.cipher.suites, ssl.protocol, ssl.enabled.protocols, ssl.secure.random.implementation, cruise.control.metrics.topic.num.partitions, cruise.control.metrics.topic.replication.factor, cruise.control.metrics.topic.retention.ms, cruise.control.metrics.topic.auto.create.retries, cruise.control.metrics.topic.auto.create.timeout.ms, cruise.control.metrics.topic.min.insync.replicas, controller.quorum.election.backoff.max.ms, controller.quorum.election.timeout.ms, controller.quorum.fetch.timeout.ms).' + type: object + x-kubernetes-preserve-unknown-fields: true + image: + description: 'The container image used for Kafka pods. If the property is not set, the default Kafka image version is determined based on the `version` configuration. The image names are specifically mapped to corresponding versions in the Cluster Operator configuration. Changing the Kafka image version does not automatically update the image versions for other components, such as Kafka Exporter. ' + type: string + jmxOptions: + description: JMX Options for Kafka brokers. + properties: + authentication: + description: Authentication configuration for connecting to the JMX port. + properties: + type: + description: Authentication type. Currently the only supported types are `password`.`password` type creates a username and protected port with no TLS. + enum: + - password + type: string + required: + - type + type: object + type: object + jvmOptions: + description: JVM Options for pods. + properties: + -XX: + additionalProperties: + type: string + description: A map of -XX options to the JVM. + type: object + -Xms: + description: -Xms option to to the JVM. + pattern: ^[0-9]+[mMgG]?$ + type: string + -Xmx: + description: -Xmx option to to the JVM. + pattern: ^[0-9]+[mMgG]?$ + type: string + gcLoggingEnabled: + description: Specifies whether the Garbage Collection logging is enabled. The default is false. + type: boolean + javaSystemProperties: + description: A map of additional system properties which will be passed using the `-D` option to the JVM. + items: + properties: + name: + description: The system property name. + type: string + value: + description: The system property value. + type: string + type: object + type: array + type: object + listeners: + description: Configures listeners to provide access to Kafka brokers. + items: + properties: + authentication: + description: Authentication configuration for this listener. + properties: + accessTokenIsJwt: + description: Configure whether the access token is treated as JWT. This must be set to `false` if the authorization server returns opaque tokens. Defaults to `true`. + type: boolean + checkAccessTokenType: + description: Configure whether the access token type check is performed or not. This should be set to `false` if the authorization server does not include 'typ' claim in JWT token. Defaults to `true`. + type: boolean + checkAudience: + description: Enable or disable audience checking. Audience checks identify the recipients of tokens. If audience checking is enabled, the OAuth Client ID also has to be configured using the `clientId` property. The Kafka broker will reject tokens that do not have its `clientId` in their `aud` (audience) claim.Default value is `false`. + type: boolean + checkIssuer: + description: Enable or disable issuer checking. By default issuer is checked using the value configured by `validIssuerUri`. Default value is `true`. + type: boolean + clientAudience: + description: The audience to use when making requests to the authorization server's token endpoint. Used for inter-broker authentication and for configuring OAuth 2.0 over PLAIN using the `clientId` and `secret` method. + type: string + clientId: + description: OAuth Client ID which the Kafka broker can use to authenticate against the authorization server and use the introspect endpoint URI. + type: string + clientScope: + description: The scope to use when making requests to the authorization server's token endpoint. Used for inter-broker authentication and for configuring OAuth 2.0 over PLAIN using the `clientId` and `secret` method. + type: string + clientSecret: + description: Link to Kubernetes Secret containing the OAuth client secret which the Kafka broker can use to authenticate against the authorization server and use the introspect endpoint URI. + properties: + key: + description: The key under which the secret value is stored in the Kubernetes Secret. + type: string + secretName: + description: The name of the Kubernetes Secret containing the secret value. + type: string + required: + - key + - secretName + type: object + connectTimeoutSeconds: + description: The connect timeout in seconds when connecting to authorization server. If not set, the effective connect timeout is 60 seconds. + type: integer + customClaimCheck: + description: JsonPath filter query to be applied to the JWT token or to the response of the introspection endpoint for additional token validation. Not set by default. + type: string + disableTlsHostnameVerification: + description: Enable or disable TLS hostname verification. Default value is `false`. + type: boolean + enableECDSA: + description: Enable or disable ECDSA support by installing BouncyCastle crypto provider. ECDSA support is always enabled. The BouncyCastle libraries are no longer packaged with Strimzi. Value is ignored. + type: boolean + enableMetrics: + description: Enable or disable OAuth metrics. Default value is `false`. + type: boolean + enableOauthBearer: + description: Enable or disable OAuth authentication over SASL_OAUTHBEARER. Default value is `true`. + type: boolean + enablePlain: + description: Enable or disable OAuth authentication over SASL_PLAIN. There is no re-authentication support when this mechanism is used. Default value is `false`. + type: boolean + failFast: + description: Enable or disable termination of Kafka broker processes due to potentially recoverable runtime errors during startup. Default value is `true`. + type: boolean + fallbackUserNameClaim: + description: The fallback username claim to be used for the user ID if the claim specified by `userNameClaim` is not present. This is useful when `client_credentials` authentication only results in the client ID being provided in another claim. It only takes effect if `userNameClaim` is set. + type: string + fallbackUserNamePrefix: + description: The prefix to use with the value of `fallbackUserNameClaim` to construct the user id. This only takes effect if `fallbackUserNameClaim` is true, and the value is present for the claim. Mapping usernames and client ids into the same user id space is useful in preventing name collisions. + type: string + groupsClaim: + description: JsonPath query used to extract groups for the user during authentication. Extracted groups can be used by a custom authorizer. By default no groups are extracted. + type: string + groupsClaimDelimiter: + description: A delimiter used to parse groups when they are extracted as a single String value rather than a JSON array. Default value is ',' (comma). + type: string + httpRetries: + description: The maximum number of retries to attempt if an initial HTTP request fails. If not set, the default is to not attempt any retries. + type: integer + httpRetryPauseMs: + description: The pause to take before retrying a failed HTTP request. If not set, the default is to not pause at all but to immediately repeat a request. + type: integer + includeAcceptHeader: + description: Whether the Accept header should be set in requests to the authorization servers. The default value is `true`. + type: boolean + introspectionEndpointUri: + description: URI of the token introspection endpoint which can be used to validate opaque non-JWT tokens. + type: string + jwksEndpointUri: + description: URI of the JWKS certificate endpoint, which can be used for local JWT validation. + type: string + jwksExpirySeconds: + description: Configures how often are the JWKS certificates considered valid. The expiry interval has to be at least 60 seconds longer then the refresh interval specified in `jwksRefreshSeconds`. Defaults to 360 seconds. + minimum: 1 + type: integer + jwksIgnoreKeyUse: + description: Flag to ignore the 'use' attribute of `key` declarations in a JWKS endpoint response. Default value is `false`. + type: boolean + jwksMinRefreshPauseSeconds: + description: The minimum pause between two consecutive refreshes. When an unknown signing key is encountered the refresh is scheduled immediately, but will always wait for this minimum pause. Defaults to 1 second. + minimum: 0 + type: integer + jwksRefreshSeconds: + description: Configures how often are the JWKS certificates refreshed. The refresh interval has to be at least 60 seconds shorter then the expiry interval specified in `jwksExpirySeconds`. Defaults to 300 seconds. + minimum: 1 + type: integer + listenerConfig: + description: Configuration to be used for a specific listener. All values are prefixed with `listener.name.`. + type: object + x-kubernetes-preserve-unknown-fields: true + maxSecondsWithoutReauthentication: + description: Maximum number of seconds the authenticated session remains valid without re-authentication. This enables Apache Kafka re-authentication feature, and causes sessions to expire when the access token expires. If the access token expires before max time or if max time is reached, the client has to re-authenticate, otherwise the server will drop the connection. Not set by default - the authenticated session does not expire when the access token expires. This option only applies to SASL_OAUTHBEARER authentication mechanism (when `enableOauthBearer` is `true`). + type: integer + readTimeoutSeconds: + description: The read timeout in seconds when connecting to authorization server. If not set, the effective read timeout is 60 seconds. + type: integer + sasl: + description: Enable or disable SASL on this listener. + type: boolean + secrets: + description: Secrets to be mounted to `/opt/kafka/custom-authn-secrets/custom-listener--/`. + items: + properties: + key: + description: The key under which the secret value is stored in the Kubernetes Secret. + type: string + secretName: + description: The name of the Kubernetes Secret containing the secret value. + type: string + required: + - key + - secretName + type: object + type: array + serverBearerTokenLocation: + description: Path to the file on the local filesystem that contains a bearer token to be used instead of client ID and secret when authenticating to authorization server. + type: string + tlsTrustedCertificates: + description: Trusted certificates for TLS connection to the OAuth server. + items: + oneOf: + - properties: + certificate: {} + required: + - certificate + - properties: + pattern: {} + required: + - pattern + properties: + certificate: + description: The name of the file certificate in the secret. + type: string + pattern: + description: Pattern for the certificate files in the secret. Use the link:https://en.wikipedia.org/wiki/Glob_(programming)[_glob syntax_] for the pattern. All files in the secret that match the pattern are used. + type: string + secretName: + description: The name of the Secret containing the certificate. + type: string + required: + - secretName + type: object + type: array + tokenEndpointUri: + description: URI of the Token Endpoint to use with SASL_PLAIN mechanism when the client authenticates with `clientId` and a `secret`. If set, the client can authenticate over SASL_PLAIN by either setting `username` to `clientId`, and setting `password` to client `secret`, or by setting `username` to account username, and `password` to access token prefixed with `$accessToken:`. If this option is not set, the `password` is always interpreted as an access token (without a prefix), and `username` as the account username (a so called 'no-client-credentials' mode). + type: string + type: + description: Authentication type. `oauth` type uses SASL OAUTHBEARER Authentication. `scram-sha-512` type uses SASL SCRAM-SHA-512 Authentication. `tls` type uses TLS Client Authentication. `tls` type is supported only on TLS listeners.`custom` type allows for any authentication type to be used. + enum: + - tls + - scram-sha-512 + - oauth + - custom + type: string + userInfoEndpointUri: + description: 'URI of the User Info Endpoint to use as a fallback to obtaining the user id when the Introspection Endpoint does not return information that can be used for the user id. ' + type: string + userNameClaim: + description: Name of the claim from the JWT authentication token, Introspection Endpoint response or User Info Endpoint response which will be used to extract the user id. Defaults to `sub`. + type: string + userNamePrefix: + description: The prefix to use with the value of `userNameClaim` to construct the user ID. This only takes effect if `userNameClaim` is specified and the value is present for the claim. When used in combination with `fallbackUserNameClaims`, it ensures consistent mapping of usernames and client IDs into the same user ID space and prevents name collisions. + type: string + validIssuerUri: + description: URI of the token issuer used for authentication. + type: string + validTokenType: + description: Valid value for the `token_type` attribute returned by the Introspection Endpoint. No default value, and not checked by default. + type: string + required: + - type + type: object + configuration: + description: Additional listener configuration. + properties: + advertisedHostTemplate: + description: Configures the template for generating the advertised hostnames of the individual brokers. Valid placeholders that you can use in the template are `{nodeId}` and `{nodePodName}`. + type: string + allocateLoadBalancerNodePorts: + description: |- + Configures whether to allocate NodePort automatically for the `Service` with type `LoadBalancer`. + This is a one to one with the `spec.allocateLoadBalancerNodePorts` configuration in the `Service` type + For `loadbalancer` listeners only. + type: boolean + bootstrap: + description: Bootstrap configuration. + properties: + alternativeNames: + description: Additional alternative names for the bootstrap service. The alternative names will be added to the list of subject alternative names of the TLS certificates. + items: + type: string + type: array + annotations: + additionalProperties: + type: string + description: Annotations added to `Ingress`, `Route`, or `Service` resources. You can use this property to configure DNS providers such as External DNS. For `loadbalancer`, `nodeport`, `route`, or `ingress` listeners only. + type: object + externalIPs: + description: External IPs associated to the nodeport service. These IPs are used by clients external to the Kubernetes cluster to access the Kafka brokers. This property is helpful when `nodeport` without `externalIP` is not sufficient. For example on bare-metal Kubernetes clusters that do not support Loadbalancer service types. For `nodeport` listeners only. + items: + type: string + type: array + host: + description: Specifies the hostname used for the bootstrap resource. For `route` (optional) or `ingress` (required) listeners only. Ensure the hostname resolves to the Ingress endpoints; no validation is performed by Strimzi. + type: string + labels: + additionalProperties: + type: string + description: Labels added to `Ingress`, `Route`, or `Service` resources. For `loadbalancer`, `nodeport`, `route`, or `ingress` listeners only. + type: object + loadBalancerIP: + description: The loadbalancer is requested with the IP address specified in this property. This feature depends on whether the underlying cloud provider supports specifying the `loadBalancerIP` when a load balancer is created. This property is ignored if the cloud provider does not support the feature. For `loadbalancer` listeners only. + type: string + nodePort: + description: Node port for the bootstrap service. For `nodeport` listeners only. + type: integer + type: object + brokerCertChainAndKey: + description: Reference to the `Secret` which holds the certificate and private key pair which will be used for this listener. The certificate can optionally contain the whole chain. This field can be used only with listeners with enabled TLS encryption. + properties: + certificate: + description: The name of the file certificate in the Secret. + type: string + key: + description: The name of the private key in the Secret. + type: string + secretName: + description: The name of the Secret containing the certificate. + type: string + required: + - secretName + - certificate + - key + type: object + brokers: + description: Per-broker configurations. + items: + properties: + advertisedHost: + description: The host name used in the brokers' `advertised.listeners`. + type: string + advertisedPort: + description: The port number used in the brokers' `advertised.listeners`. + type: integer + annotations: + additionalProperties: + type: string + description: Annotations that will be added to the `Ingress` or `Service` resource. You can use this field to configure DNS providers such as External DNS. This field can be used only with `loadbalancer`, `nodeport`, or `ingress` type listeners. + type: object + broker: + description: ID of the kafka broker (broker identifier). Broker IDs start from 0 and correspond to the number of broker replicas. + type: integer + externalIPs: + description: External IPs associated to the nodeport service. These IPs are used by clients external to the Kubernetes cluster to access the Kafka brokers. This field is helpful when `nodeport` without `externalIP` is not sufficient. For example on bare-metal Kubernetes clusters that do not support Loadbalancer service types. This field can only be used with `nodeport` type listener. + items: + type: string + type: array + host: + description: The broker host. This field will be used in the Ingress resource or in the Route resource to specify the desired hostname. This field can be used only with `route` (optional) or `ingress` (required) type listeners. + type: string + labels: + additionalProperties: + type: string + description: Labels that will be added to the `Ingress`, `Route`, or `Service` resource. This field can be used only with `loadbalancer`, `nodeport`, `route`, or `ingress` type listeners. + type: object + loadBalancerIP: + description: The loadbalancer is requested with the IP address specified in this field. This feature depends on whether the underlying cloud provider supports specifying the `loadBalancerIP` when a load balancer is created. This field is ignored if the cloud provider does not support the feature.This field can be used only with `loadbalancer` type listener. + type: string + nodePort: + description: Node port for the per-broker service. This field can be used only with `nodeport` type listener. + type: integer + required: + - broker + type: object + type: array + class: + description: |- + Configures a specific class for `Ingress` and `LoadBalancer` that defines which controller is used. If not specified, the default controller is used. + + * For an `ingress` listener, the operator uses this property to set the `ingressClassName` property in the `Ingress` resources. + * For a `loadbalancer` listener, the operator uses this property to set the `loadBalancerClass` property in the `Service` resources. + + For `ingress` and `loadbalancer` listeners only. + type: string + createBootstrapService: + description: Whether to create the bootstrap service or not. The bootstrap service is created by default (if not specified differently). This field can be used with the `loadbalancer` listener. + type: boolean + externalTrafficPolicy: + description: |- + Specifies whether the service routes external traffic to cluster-wide or node-local endpoints: + + * `Cluster` may cause a second hop to another node and obscures the client source IP. + * `Local` avoids a second hop for `LoadBalancer` and `Nodeport` type services and preserves the client source IP (when supported by the infrastructure). + + If unspecified, Kubernetes uses `Cluster` as the default. For `loadbalancer` or `nodeport` listeners only. + enum: + - Local + - Cluster + type: string + finalizers: + description: A list of finalizers configured for the `LoadBalancer` type services created for this listener. If supported by the platform, the finalizer `service.kubernetes.io/load-balancer-cleanup` to make sure that the external load balancer is deleted together with the service.For more information, see https://kubernetes.io/docs/tasks/access-application-cluster/create-external-load-balancer/#garbage-collecting-load-balancers. For `loadbalancer` listeners only. + items: + type: string + type: array + hostTemplate: + description: Configures the template for generating the hostnames of the individual brokers. Valid placeholders that you can use in the template are `{nodeId}` and `{nodePodName}`. + type: string + ipFamilies: + description: Specifies the IP Families used by the service. Available options are `IPv4` and `IPv6`. If unspecified, Kubernetes will choose the default value based on the `ipFamilyPolicy` setting. + items: + enum: + - IPv4 + - IPv6 + type: string + type: array + ipFamilyPolicy: + description: |- + Specifies the IP Family Policy used by the service. Available options are `SingleStack`, `PreferDualStack` and `RequireDualStack`: + + * `SingleStack` is for a single IP family. + * `PreferDualStack` is for two IP families on dual-stack configured clusters or a single IP family on single-stack clusters. + * `RequireDualStack` fails unless there are two IP families on dual-stack configured clusters. + + If unspecified, Kubernetes will choose the default value based on the service type. + enum: + - SingleStack + - PreferDualStack + - RequireDualStack + type: string + loadBalancerSourceRanges: + description: A list of CIDR ranges (for example `10.0.0.0/8` or `130.211.204.1/32`) from which clients can connect to loadbalancer listeners. If supported by the platform, traffic through the loadbalancer is restricted to the specified CIDR ranges. This field is applicable only for loadbalancer type services and is ignored if the cloud provider does not support the feature. For `loadbalancer` listeners only. + items: + type: string + type: array + maxConnectionCreationRate: + description: The maximum connection creation rate we allow in this listener at any time. New connections will be throttled if the limit is reached. + type: integer + maxConnections: + description: The maximum number of connections we allow for this listener in the broker at any time. New connections are blocked if the limit is reached. + type: integer + preferredNodePortAddressType: + description: |- + Defines which address type should be used as the node address. Available types are: `ExternalDNS`, `ExternalIP`, `InternalDNS`, `InternalIP` and `Hostname`. By default, the addresses are used in the following order (the first one found is used): + + * `ExternalDNS` + * `ExternalIP` + * `InternalDNS` + * `InternalIP` + * `Hostname` + + This property is used to select the preferred address type, which is checked first. If no address is found for this address type, the other types are checked in the default order.For `nodeport` listeners only. + enum: + - ExternalIP + - ExternalDNS + - InternalIP + - InternalDNS + - Hostname + type: string + publishNotReadyAddresses: + description: Configures whether the service endpoints are considered "ready" even if the Pods themselves are not. Defaults to `false`. This field can not be used with `internal` listeners. + type: boolean + useServiceDnsDomain: + description: |- + Configures whether the Kubernetes service DNS domain should be included in the generated addresses. + + * If set to `false`, the generated addresses do not contain the service DNS domain suffix. For example, `my-cluster-kafka-0.my-cluster-kafka-brokers.myproject.svc`. + * If set to `true`, the generated addresses contain the service DNS domain suffix. For example, `my-cluster-kafka-0.my-cluster-kafka-brokers.myproject.svc.cluster.local`. + + The default is `.cluster.local`, but this is customizable using the environment variable `KUBERNETES_SERVICE_DNS_DOMAIN`. For `internal` and `cluster-ip` listeners only. + type: boolean + type: object + name: + description: Name of the listener. The name will be used to identify the listener and the related Kubernetes objects. The name has to be unique within given a Kafka cluster. The name can consist of lowercase characters and numbers and be up to 11 characters long. + pattern: ^[a-z0-9]{1,11}$ + type: string + networkPolicyPeers: + description: List of peers which should be able to connect to this listener. Peers in this list are combined using a logical OR operation. If this field is empty or missing, all connections will be allowed for this listener. If this field is present and contains at least one item, the listener only allows the traffic which matches at least one item in this list. + items: + properties: + ipBlock: + properties: + cidr: + type: string + except: + items: + type: string + type: array + type: object + namespaceSelector: + properties: + matchExpressions: + items: + properties: + key: + type: string + operator: + type: string + values: + items: + type: string + type: array + type: object + type: array + matchLabels: + additionalProperties: + type: string + type: object + type: object + podSelector: + properties: + matchExpressions: + items: + properties: + key: + type: string + operator: + type: string + values: + items: + type: string + type: array + type: object + type: array + matchLabels: + additionalProperties: + type: string + type: object + type: object + type: object + type: array + port: + description: Port number used by the listener inside Kafka. The port number has to be unique within a given Kafka cluster. Allowed port numbers are 9092 and higher with the exception of ports 9404 and 9999, which are already used for Prometheus and JMX. Depending on the listener type, the port number might not be the same as the port number that connects Kafka clients. + minimum: 9092 + type: integer + tls: + description: Enables TLS encryption on the listener. This is a required property. For `route` and `ingress` type listeners, TLS encryption must be always enabled. + type: boolean + type: + description: "Type of the listener. The supported types are as follows: \n\n* `internal` type exposes Kafka internally only within the Kubernetes cluster.\n* `route` type uses OpenShift Routes to expose Kafka.\n* `loadbalancer` type uses LoadBalancer type services to expose Kafka.\n* `nodeport` type uses NodePort type services to expose Kafka.\n* `ingress` type uses Kubernetes Nginx Ingress to expose Kafka with TLS passthrough.\n* `cluster-ip` type uses a per-broker `ClusterIP` service.\n" + enum: + - internal + - route + - loadbalancer + - nodeport + - ingress + - cluster-ip + type: string + required: + - name + - port + - type + - tls + type: object + minItems: 1 + type: array + livenessProbe: + description: Pod liveness checking. + properties: + failureThreshold: + description: Minimum consecutive failures for the probe to be considered failed after having succeeded. Defaults to 3. Minimum value is 1. + minimum: 1 + type: integer + initialDelaySeconds: + description: The initial delay before first the health is first checked. Default to 15 seconds. Minimum value is 0. + minimum: 0 + type: integer + periodSeconds: + description: How often (in seconds) to perform the probe. Default to 10 seconds. Minimum value is 1. + minimum: 1 + type: integer + successThreshold: + description: Minimum consecutive successes for the probe to be considered successful after having failed. Defaults to 1. Must be 1 for liveness. Minimum value is 1. + minimum: 1 + type: integer + timeoutSeconds: + description: The timeout for each attempted health check. Default to 5 seconds. Minimum value is 1. + minimum: 1 + type: integer + type: object + logging: + description: Logging configuration for Kafka. + properties: + loggers: + additionalProperties: + type: string + description: A Map from logger name to logger level. + type: object + type: + description: Logging type, must be either 'inline' or 'external'. + enum: + - inline + - external + type: string + valueFrom: + description: '`ConfigMap` entry where the logging configuration is stored. ' + properties: + configMapKeyRef: + description: Reference to the key in the ConfigMap containing the configuration. + properties: + key: + type: string + name: + type: string + optional: + type: boolean + type: object + type: object + required: + - type + type: object + metadataVersion: + description: The KRaft metadata version used by the Kafka cluster. This property is ignored when running in ZooKeeper mode. If the property is not set, it defaults to the metadata version that corresponds to the `version` property. + type: string + metricsConfig: + description: Metrics configuration. + properties: + type: + description: Metrics type. Only 'jmxPrometheusExporter' supported currently. + enum: + - jmxPrometheusExporter + type: string + valueFrom: + description: 'ConfigMap entry where the Prometheus JMX Exporter configuration is stored. ' + properties: + configMapKeyRef: + description: Reference to the key in the ConfigMap containing the configuration. + properties: + key: + type: string + name: + type: string + optional: + type: boolean + type: object + type: object + required: + - type + - valueFrom + type: object + quotas: + description: Quotas plugin configuration for Kafka brokers allows setting quotas for disk usage, produce/fetch rates, and more. Supported plugin types include `kafka` (default) and `strimzi`. If not specified, the default `kafka` quotas plugin is used. + properties: + consumerByteRate: + description: A per-broker byte-rate quota for clients consuming from a broker, independent of their number. If clients consume at maximum speed, the quota is shared equally between all non-excluded consumers. Otherwise, the quota is divided based on each client's consumption rate. + minimum: 0 + type: integer + controllerMutationRate: + description: The default client quota on the rate at which mutations are accepted per second for create topic requests, create partition requests, and delete topic requests, defined for each broker. The mutations rate is measured by the number of partitions created or deleted. Applied on a per-broker basis. + minimum: 0 + type: number + excludedPrincipals: + description: List of principals that are excluded from the quota. The principals have to be prefixed with `User:`, for example `User:my-user;User:CN=my-other-user`. + items: + type: string + type: array + minAvailableBytesPerVolume: + description: Stop message production if the available size (in bytes) of the storage is lower than or equal to this specified value. This condition is mutually exclusive with `minAvailableRatioPerVolume`. + minimum: 0 + type: integer + minAvailableRatioPerVolume: + description: Stop message production if the percentage of available storage space falls below or equals the specified ratio (set as a decimal representing a percentage). This condition is mutually exclusive with `minAvailableBytesPerVolume`. + maximum: 1 + minimum: 0 + type: number + producerByteRate: + description: A per-broker byte-rate quota for clients producing to a broker, independent of their number. If clients produce at maximum speed, the quota is shared equally between all non-excluded producers. Otherwise, the quota is divided based on each client's production rate. + minimum: 0 + type: integer + requestPercentage: + description: The default client quota limits the maximum CPU utilization of each client as a percentage of the network and I/O threads of each broker. Applied on a per-broker basis. + minimum: 0 + type: integer + type: + description: Quotas plugin type. Currently, the supported types are `kafka` and `strimzi`. `kafka` quotas type uses Kafka's built-in quotas plugin. `strimzi` quotas type uses Strimzi quotas plugin. + enum: + - kafka + - strimzi + type: string + required: + - type + type: object + rack: + description: Configuration of the `broker.rack` broker config. + properties: + topologyKey: + description: A key that matches labels assigned to the Kubernetes cluster nodes. The value of the label is used to set a broker's `broker.rack` config, and the `client.rack` config for Kafka Connect or MirrorMaker 2. + example: topology.kubernetes.io/zone + type: string + required: + - topologyKey + type: object + readinessProbe: + description: Pod readiness checking. + properties: + failureThreshold: + description: Minimum consecutive failures for the probe to be considered failed after having succeeded. Defaults to 3. Minimum value is 1. + minimum: 1 + type: integer + initialDelaySeconds: + description: The initial delay before first the health is first checked. Default to 15 seconds. Minimum value is 0. + minimum: 0 + type: integer + periodSeconds: + description: How often (in seconds) to perform the probe. Default to 10 seconds. Minimum value is 1. + minimum: 1 + type: integer + successThreshold: + description: Minimum consecutive successes for the probe to be considered successful after having failed. Defaults to 1. Must be 1 for liveness. Minimum value is 1. + minimum: 1 + type: integer + timeoutSeconds: + description: The timeout for each attempted health check. Default to 5 seconds. Minimum value is 1. + minimum: 1 + type: integer + type: object + replicas: + description: The number of pods in the cluster. This property is required when node pools are not used. + minimum: 1 + type: integer + resources: + description: CPU and memory resources to reserve. + properties: + claims: + items: + properties: + name: + type: string + type: object + type: array + limits: + additionalProperties: + anyOf: + - type: integer + - type: string + pattern: ^(\+|-)?(([0-9]+(\.[0-9]*)?)|(\.[0-9]+))(([KMGTPE]i)|[numkMGTPE]|([eE](\+|-)?(([0-9]+(\.[0-9]*)?)|(\.[0-9]+))))?$ + x-kubernetes-int-or-string: true + type: object + requests: + additionalProperties: + anyOf: + - type: integer + - type: string + pattern: ^(\+|-)?(([0-9]+(\.[0-9]*)?)|(\.[0-9]+))(([KMGTPE]i)|[numkMGTPE]|([eE](\+|-)?(([0-9]+(\.[0-9]*)?)|(\.[0-9]+))))?$ + x-kubernetes-int-or-string: true + type: object + type: object + storage: + description: Storage configuration (disk). Cannot be updated. This property is required when node pools are not used. + properties: + class: + description: The storage class to use for dynamic volume allocation. + type: string + deleteClaim: + description: Specifies if the persistent volume claim has to be deleted when the cluster is un-deployed. + type: boolean + id: + description: Storage identification number. It is mandatory only for storage volumes defined in a storage of type 'jbod'. + minimum: 0 + type: integer + kraftMetadata: + description: Specifies whether this volume should be used for storing KRaft metadata. This property is optional. When set, the only currently supported value is `shared`. At most one volume can have this property set. + enum: + - shared + type: string + overrides: + description: Overrides for individual brokers. The `overrides` field allows you to specify a different configuration for different brokers. + items: + properties: + broker: + description: Id of the kafka broker (broker identifier). + type: integer + class: + description: The storage class to use for dynamic volume allocation for this broker. + type: string + type: object + type: array + selector: + additionalProperties: + type: string + description: Specifies a specific persistent volume to use. It contains key:value pairs representing labels for selecting such a volume. + type: object + size: + description: When `type=persistent-claim`, defines the size of the persistent volume claim, such as 100Gi. Mandatory when `type=persistent-claim`. + type: string + sizeLimit: + description: When type=ephemeral, defines the total amount of local storage required for this EmptyDir volume (for example 1Gi). + pattern: ^([0-9.]+)([eEinumkKMGTP]*[-+]?[0-9]*)$ + type: string + type: + description: Storage type, must be either 'ephemeral', 'persistent-claim', or 'jbod'. + enum: + - ephemeral + - persistent-claim + - jbod + type: string + volumes: + description: List of volumes as Storage objects representing the JBOD disks array. + items: + properties: + class: + description: The storage class to use for dynamic volume allocation. + type: string + deleteClaim: + description: Specifies if the persistent volume claim has to be deleted when the cluster is un-deployed. + type: boolean + id: + description: Storage identification number. Mandatory for storage volumes defined with a `jbod` storage type configuration. + minimum: 0 + type: integer + kraftMetadata: + description: Specifies whether this volume should be used for storing KRaft metadata. This property is optional. When set, the only currently supported value is `shared`. At most one volume can have this property set. + enum: + - shared + type: string + overrides: + description: Overrides for individual brokers. The `overrides` field allows you to specify a different configuration for different brokers. + items: + properties: + broker: + description: Id of the kafka broker (broker identifier). + type: integer + class: + description: The storage class to use for dynamic volume allocation for this broker. + type: string + type: object + type: array + selector: + additionalProperties: + type: string + description: Specifies a specific persistent volume to use. It contains key:value pairs representing labels for selecting such a volume. + type: object + size: + description: When `type=persistent-claim`, defines the size of the persistent volume claim, such as 100Gi. Mandatory when `type=persistent-claim`. + type: string + sizeLimit: + description: When type=ephemeral, defines the total amount of local storage required for this EmptyDir volume (for example 1Gi). + pattern: ^([0-9.]+)([eEinumkKMGTP]*[-+]?[0-9]*)$ + type: string + type: + description: Storage type, must be either 'ephemeral' or 'persistent-claim'. + enum: + - ephemeral + - persistent-claim + type: string + required: + - type + type: object + type: array + required: + - type + type: object + template: + description: Template for Kafka cluster resources. The template allows users to specify how the Kubernetes resources are generated. + properties: + bootstrapService: + description: Template for Kafka bootstrap `Service`. + properties: + ipFamilies: + description: Specifies the IP Families used by the service. Available options are `IPv4` and `IPv6`. If unspecified, Kubernetes will choose the default value based on the `ipFamilyPolicy` setting. + items: + enum: + - IPv4 + - IPv6 + type: string + type: array + ipFamilyPolicy: + description: Specifies the IP Family Policy used by the service. Available options are `SingleStack`, `PreferDualStack` and `RequireDualStack`. `SingleStack` is for a single IP family. `PreferDualStack` is for two IP families on dual-stack configured clusters or a single IP family on single-stack clusters. `RequireDualStack` fails unless there are two IP families on dual-stack configured clusters. If unspecified, Kubernetes will choose the default value based on the service type. + enum: + - SingleStack + - PreferDualStack + - RequireDualStack + type: string + metadata: + description: Metadata applied to the resource. + properties: + annotations: + additionalProperties: + type: string + description: Annotations added to the Kubernetes resource. + type: object + labels: + additionalProperties: + type: string + description: Labels added to the Kubernetes resource. + type: object + type: object + type: object + brokersService: + description: Template for Kafka broker `Service`. + properties: + ipFamilies: + description: Specifies the IP Families used by the service. Available options are `IPv4` and `IPv6`. If unspecified, Kubernetes will choose the default value based on the `ipFamilyPolicy` setting. + items: + enum: + - IPv4 + - IPv6 + type: string + type: array + ipFamilyPolicy: + description: Specifies the IP Family Policy used by the service. Available options are `SingleStack`, `PreferDualStack` and `RequireDualStack`. `SingleStack` is for a single IP family. `PreferDualStack` is for two IP families on dual-stack configured clusters or a single IP family on single-stack clusters. `RequireDualStack` fails unless there are two IP families on dual-stack configured clusters. If unspecified, Kubernetes will choose the default value based on the service type. + enum: + - SingleStack + - PreferDualStack + - RequireDualStack + type: string + metadata: + description: Metadata applied to the resource. + properties: + annotations: + additionalProperties: + type: string + description: Annotations added to the Kubernetes resource. + type: object + labels: + additionalProperties: + type: string + description: Labels added to the Kubernetes resource. + type: object + type: object + type: object + clusterCaCert: + description: Template for Secret with Kafka Cluster certificate public key. + properties: + metadata: + description: Metadata applied to the resource. + properties: + annotations: + additionalProperties: + type: string + description: Annotations added to the Kubernetes resource. + type: object + labels: + additionalProperties: + type: string + description: Labels added to the Kubernetes resource. + type: object + type: object + type: object + clusterRoleBinding: + description: Template for the Kafka ClusterRoleBinding. + properties: + metadata: + description: Metadata applied to the resource. + properties: + annotations: + additionalProperties: + type: string + description: Annotations added to the Kubernetes resource. + type: object + labels: + additionalProperties: + type: string + description: Labels added to the Kubernetes resource. + type: object + type: object + type: object + externalBootstrapIngress: + description: Template for Kafka external bootstrap `Ingress`. + properties: + metadata: + description: Metadata applied to the resource. + properties: + annotations: + additionalProperties: + type: string + description: Annotations added to the Kubernetes resource. + type: object + labels: + additionalProperties: + type: string + description: Labels added to the Kubernetes resource. + type: object + type: object + type: object + externalBootstrapRoute: + description: Template for Kafka external bootstrap `Route`. + properties: + metadata: + description: Metadata applied to the resource. + properties: + annotations: + additionalProperties: + type: string + description: Annotations added to the Kubernetes resource. + type: object + labels: + additionalProperties: + type: string + description: Labels added to the Kubernetes resource. + type: object + type: object + type: object + externalBootstrapService: + description: Template for Kafka external bootstrap `Service`. + properties: + metadata: + description: Metadata applied to the resource. + properties: + annotations: + additionalProperties: + type: string + description: Annotations added to the Kubernetes resource. + type: object + labels: + additionalProperties: + type: string + description: Labels added to the Kubernetes resource. + type: object + type: object + type: object + initContainer: + description: Template for the Kafka init container. + properties: + env: + description: Environment variables which should be applied to the container. + items: + oneOf: + - properties: + value: {} + required: + - value + - properties: + valueFrom: {} + required: + - valueFrom + properties: + name: + description: The environment variable key. + type: string + value: + description: The environment variable value. + type: string + valueFrom: + description: Reference to the secret or config map property to which the environment variable is set. + oneOf: + - properties: + secretKeyRef: {} + required: + - secretKeyRef + - properties: + configMapKeyRef: {} + required: + - configMapKeyRef + properties: + configMapKeyRef: + description: Reference to a key in a config map. + properties: + key: + type: string + name: + type: string + optional: + type: boolean + type: object + secretKeyRef: + description: Reference to a key in a secret. + properties: + key: + type: string + name: + type: string + optional: + type: boolean + type: object + type: object + type: object + type: array + securityContext: + description: Security context for the container. + properties: + allowPrivilegeEscalation: + type: boolean + appArmorProfile: + properties: + localhostProfile: + type: string + type: + type: string + type: object + capabilities: + properties: + add: + items: + type: string + type: array + drop: + items: + type: string + type: array + type: object + privileged: + type: boolean + procMount: + type: string + readOnlyRootFilesystem: + type: boolean + runAsGroup: + type: integer + runAsNonRoot: + type: boolean + runAsUser: + type: integer + seLinuxOptions: + properties: + level: + type: string + role: + type: string + type: + type: string + user: + type: string + type: object + seccompProfile: + properties: + localhostProfile: + type: string + type: + type: string + type: object + windowsOptions: + properties: + gmsaCredentialSpec: + type: string + gmsaCredentialSpecName: + type: string + hostProcess: + type: boolean + runAsUserName: + type: string + type: object + type: object + volumeMounts: + description: Additional volume mounts which should be applied to the container. + items: + properties: + mountPath: + type: string + mountPropagation: + type: string + name: + type: string + readOnly: + type: boolean + recursiveReadOnly: + type: string + subPath: + type: string + subPathExpr: + type: string + type: object + type: array + type: object + jmxSecret: + description: Template for Secret of the Kafka Cluster JMX authentication. + properties: + metadata: + description: Metadata applied to the resource. + properties: + annotations: + additionalProperties: + type: string + description: Annotations added to the Kubernetes resource. + type: object + labels: + additionalProperties: + type: string + description: Labels added to the Kubernetes resource. + type: object + type: object + type: object + kafkaContainer: + description: Template for the Kafka broker container. + properties: + env: + description: Environment variables which should be applied to the container. + items: + oneOf: + - properties: + value: {} + required: + - value + - properties: + valueFrom: {} + required: + - valueFrom + properties: + name: + description: The environment variable key. + type: string + value: + description: The environment variable value. + type: string + valueFrom: + description: Reference to the secret or config map property to which the environment variable is set. + oneOf: + - properties: + secretKeyRef: {} + required: + - secretKeyRef + - properties: + configMapKeyRef: {} + required: + - configMapKeyRef + properties: + configMapKeyRef: + description: Reference to a key in a config map. + properties: + key: + type: string + name: + type: string + optional: + type: boolean + type: object + secretKeyRef: + description: Reference to a key in a secret. + properties: + key: + type: string + name: + type: string + optional: + type: boolean + type: object + type: object + type: object + type: array + securityContext: + description: Security context for the container. + properties: + allowPrivilegeEscalation: + type: boolean + appArmorProfile: + properties: + localhostProfile: + type: string + type: + type: string + type: object + capabilities: + properties: + add: + items: + type: string + type: array + drop: + items: + type: string + type: array + type: object + privileged: + type: boolean + procMount: + type: string + readOnlyRootFilesystem: + type: boolean + runAsGroup: + type: integer + runAsNonRoot: + type: boolean + runAsUser: + type: integer + seLinuxOptions: + properties: + level: + type: string + role: + type: string + type: + type: string + user: + type: string + type: object + seccompProfile: + properties: + localhostProfile: + type: string + type: + type: string + type: object + windowsOptions: + properties: + gmsaCredentialSpec: + type: string + gmsaCredentialSpecName: + type: string + hostProcess: + type: boolean + runAsUserName: + type: string + type: object + type: object + volumeMounts: + description: Additional volume mounts which should be applied to the container. + items: + properties: + mountPath: + type: string + mountPropagation: + type: string + name: + type: string + readOnly: + type: boolean + recursiveReadOnly: + type: string + subPath: + type: string + subPathExpr: + type: string + type: object + type: array + type: object + perPodIngress: + description: Template for Kafka per-pod `Ingress` used for access from outside of Kubernetes. + properties: + metadata: + description: Metadata applied to the resource. + properties: + annotations: + additionalProperties: + type: string + description: Annotations added to the Kubernetes resource. + type: object + labels: + additionalProperties: + type: string + description: Labels added to the Kubernetes resource. + type: object + type: object + type: object + perPodRoute: + description: Template for Kafka per-pod `Routes` used for access from outside of OpenShift. + properties: + metadata: + description: Metadata applied to the resource. + properties: + annotations: + additionalProperties: + type: string + description: Annotations added to the Kubernetes resource. + type: object + labels: + additionalProperties: + type: string + description: Labels added to the Kubernetes resource. + type: object + type: object + type: object + perPodService: + description: Template for Kafka per-pod `Services` used for access from outside of Kubernetes. + properties: + metadata: + description: Metadata applied to the resource. + properties: + annotations: + additionalProperties: + type: string + description: Annotations added to the Kubernetes resource. + type: object + labels: + additionalProperties: + type: string + description: Labels added to the Kubernetes resource. + type: object + type: object + type: object + persistentVolumeClaim: + description: Template for all Kafka `PersistentVolumeClaims`. + properties: + metadata: + description: Metadata applied to the resource. + properties: + annotations: + additionalProperties: + type: string + description: Annotations added to the Kubernetes resource. + type: object + labels: + additionalProperties: + type: string + description: Labels added to the Kubernetes resource. + type: object + type: object + type: object + pod: + description: Template for Kafka `Pods`. + properties: + affinity: + description: The pod's affinity rules. + properties: + nodeAffinity: + properties: + preferredDuringSchedulingIgnoredDuringExecution: + items: + properties: + preference: + properties: + matchExpressions: + items: + properties: + key: + type: string + operator: + type: string + values: + items: + type: string + type: array + type: object + type: array + matchFields: + items: + properties: + key: + type: string + operator: + type: string + values: + items: + type: string + type: array + type: object + type: array + type: object + weight: + type: integer + type: object + type: array + requiredDuringSchedulingIgnoredDuringExecution: + properties: + nodeSelectorTerms: + items: + properties: + matchExpressions: + items: + properties: + key: + type: string + operator: + type: string + values: + items: + type: string + type: array + type: object + type: array + matchFields: + items: + properties: + key: + type: string + operator: + type: string + values: + items: + type: string + type: array + type: object + type: array + type: object + type: array + type: object + type: object + podAffinity: + properties: + preferredDuringSchedulingIgnoredDuringExecution: + items: + properties: + podAffinityTerm: + properties: + labelSelector: + properties: + matchExpressions: + items: + properties: + key: + type: string + operator: + type: string + values: + items: + type: string + type: array + type: object + type: array + matchLabels: + additionalProperties: + type: string + type: object + type: object + matchLabelKeys: + items: + type: string + type: array + mismatchLabelKeys: + items: + type: string + type: array + namespaceSelector: + properties: + matchExpressions: + items: + properties: + key: + type: string + operator: + type: string + values: + items: + type: string + type: array + type: object + type: array + matchLabels: + additionalProperties: + type: string + type: object + type: object + namespaces: + items: + type: string + type: array + topologyKey: + type: string + type: object + weight: + type: integer + type: object + type: array + requiredDuringSchedulingIgnoredDuringExecution: + items: + properties: + labelSelector: + properties: + matchExpressions: + items: + properties: + key: + type: string + operator: + type: string + values: + items: + type: string + type: array + type: object + type: array + matchLabels: + additionalProperties: + type: string + type: object + type: object + matchLabelKeys: + items: + type: string + type: array + mismatchLabelKeys: + items: + type: string + type: array + namespaceSelector: + properties: + matchExpressions: + items: + properties: + key: + type: string + operator: + type: string + values: + items: + type: string + type: array + type: object + type: array + matchLabels: + additionalProperties: + type: string + type: object + type: object + namespaces: + items: + type: string + type: array + topologyKey: + type: string + type: object + type: array + type: object + podAntiAffinity: + properties: + preferredDuringSchedulingIgnoredDuringExecution: + items: + properties: + podAffinityTerm: + properties: + labelSelector: + properties: + matchExpressions: + items: + properties: + key: + type: string + operator: + type: string + values: + items: + type: string + type: array + type: object + type: array + matchLabels: + additionalProperties: + type: string + type: object + type: object + matchLabelKeys: + items: + type: string + type: array + mismatchLabelKeys: + items: + type: string + type: array + namespaceSelector: + properties: + matchExpressions: + items: + properties: + key: + type: string + operator: + type: string + values: + items: + type: string + type: array + type: object + type: array + matchLabels: + additionalProperties: + type: string + type: object + type: object + namespaces: + items: + type: string + type: array + topologyKey: + type: string + type: object + weight: + type: integer + type: object + type: array + requiredDuringSchedulingIgnoredDuringExecution: + items: + properties: + labelSelector: + properties: + matchExpressions: + items: + properties: + key: + type: string + operator: + type: string + values: + items: + type: string + type: array + type: object + type: array + matchLabels: + additionalProperties: + type: string + type: object + type: object + matchLabelKeys: + items: + type: string + type: array + mismatchLabelKeys: + items: + type: string + type: array + namespaceSelector: + properties: + matchExpressions: + items: + properties: + key: + type: string + operator: + type: string + values: + items: + type: string + type: array + type: object + type: array + matchLabels: + additionalProperties: + type: string + type: object + type: object + namespaces: + items: + type: string + type: array + topologyKey: + type: string + type: object + type: array + type: object + type: object + enableServiceLinks: + description: Indicates whether information about services should be injected into Pod's environment variables. + type: boolean + hostAliases: + description: The pod's HostAliases. HostAliases is an optional list of hosts and IPs that will be injected into the Pod's hosts file if specified. + items: + properties: + hostnames: + items: + type: string + type: array + ip: + type: string + type: object + type: array + imagePullSecrets: + description: List of references to secrets in the same namespace to use for pulling any of the images used by this Pod. When the `STRIMZI_IMAGE_PULL_SECRETS` environment variable in Cluster Operator and the `imagePullSecrets` option are specified, only the `imagePullSecrets` variable is used and the `STRIMZI_IMAGE_PULL_SECRETS` variable is ignored. + items: + properties: + name: + type: string + type: object + type: array + metadata: + description: Metadata applied to the resource. + properties: + annotations: + additionalProperties: + type: string + description: Annotations added to the Kubernetes resource. + type: object + labels: + additionalProperties: + type: string + description: Labels added to the Kubernetes resource. + type: object + type: object + priorityClassName: + description: 'The name of the priority class used to assign priority to the pods. ' + type: string + schedulerName: + description: The name of the scheduler used to dispatch this `Pod`. If not specified, the default scheduler will be used. + type: string + securityContext: + description: Configures pod-level security attributes and common container settings. + properties: + appArmorProfile: + properties: + localhostProfile: + type: string + type: + type: string + type: object + fsGroup: + type: integer + fsGroupChangePolicy: + type: string + runAsGroup: + type: integer + runAsNonRoot: + type: boolean + runAsUser: + type: integer + seLinuxOptions: + properties: + level: + type: string + role: + type: string + type: + type: string + user: + type: string + type: object + seccompProfile: + properties: + localhostProfile: + type: string + type: + type: string + type: object + supplementalGroups: + items: + type: integer + type: array + sysctls: + items: + properties: + name: + type: string + value: + type: string + type: object + type: array + windowsOptions: + properties: + gmsaCredentialSpec: + type: string + gmsaCredentialSpecName: + type: string + hostProcess: + type: boolean + runAsUserName: + type: string + type: object + type: object + terminationGracePeriodSeconds: + description: The grace period is the duration in seconds after the processes running in the pod are sent a termination signal, and the time when the processes are forcibly halted with a kill signal. Set this value to longer than the expected cleanup time for your process. Value must be a non-negative integer. A zero value indicates delete immediately. You might need to increase the grace period for very large Kafka clusters, so that the Kafka brokers have enough time to transfer their work to another broker before they are terminated. Defaults to 30 seconds. + minimum: 0 + type: integer + tmpDirSizeLimit: + description: Defines the total amount of pod memory allocated for the temporary `EmptyDir` volume `/tmp`. Specify the allocation in memory units, for example, `100Mi` for 100 mebibytes. Default value is `5Mi`. The `/tmp` volume is backed by pod memory, not disk storage, so avoid setting a high value as it consumes pod memory resources. + pattern: ^([0-9.]+)([eEinumkKMGTP]*[-+]?[0-9]*)$ + type: string + tolerations: + description: The pod's tolerations. + items: + properties: + effect: + type: string + key: + type: string + operator: + type: string + tolerationSeconds: + type: integer + value: + type: string + type: object + type: array + topologySpreadConstraints: + description: The pod's topology spread constraints. + items: + properties: + labelSelector: + properties: + matchExpressions: + items: + properties: + key: + type: string + operator: + type: string + values: + items: + type: string + type: array + type: object + type: array + matchLabels: + additionalProperties: + type: string + type: object + type: object + matchLabelKeys: + items: + type: string + type: array + maxSkew: + type: integer + minDomains: + type: integer + nodeAffinityPolicy: + type: string + nodeTaintsPolicy: + type: string + topologyKey: + type: string + whenUnsatisfiable: + type: string + type: object + type: array + volumes: + description: Additional volumes that can be mounted to the pod. + items: + oneOf: + - properties: + configMap: {} + csi: {} + emptyDir: {} + persistentVolumeClaim: {} + secret: {} + properties: + configMap: + description: ConfigMap to use to populate the volume. + properties: + defaultMode: + type: integer + items: + items: + properties: + key: + type: string + mode: + type: integer + path: + type: string + type: object + type: array + name: + type: string + optional: + type: boolean + type: object + csi: + description: CSIVolumeSource object to use to populate the volume. + properties: + driver: + type: string + fsType: + type: string + nodePublishSecretRef: + properties: + name: + type: string + type: object + readOnly: + type: boolean + volumeAttributes: + additionalProperties: + type: string + type: object + type: object + emptyDir: + description: EmptyDir to use to populate the volume. + properties: + medium: + type: string + sizeLimit: + properties: + amount: + type: string + format: + type: string + type: object + type: object + name: + description: Name to use for the volume. Required. + type: string + persistentVolumeClaim: + description: PersistentVolumeClaim object to use to populate the volume. + properties: + claimName: + type: string + readOnly: + type: boolean + type: object + secret: + description: Secret to use populate the volume. + properties: + defaultMode: + type: integer + items: + items: + properties: + key: + type: string + mode: + type: integer + path: + type: string + type: object + type: array + optional: + type: boolean + secretName: + type: string + type: object + type: object + type: array + type: object + podDisruptionBudget: + description: Template for Kafka `PodDisruptionBudget`. + properties: + maxUnavailable: + description: Maximum number of unavailable pods to allow automatic Pod eviction. A Pod eviction is allowed when the `maxUnavailable` number of pods or fewer are unavailable after the eviction. Setting this value to 0 prevents all voluntary evictions, so the pods must be evicted manually. Defaults to 1. + minimum: 0 + type: integer + metadata: + description: Metadata to apply to the `PodDisruptionBudgetTemplate` resource. + properties: + annotations: + additionalProperties: + type: string + description: Annotations added to the Kubernetes resource. + type: object + labels: + additionalProperties: + type: string + description: Labels added to the Kubernetes resource. + type: object + type: object + type: object + podSet: + description: Template for Kafka `StrimziPodSet` resource. + properties: + metadata: + description: Metadata applied to the resource. + properties: + annotations: + additionalProperties: + type: string + description: Annotations added to the Kubernetes resource. + type: object + labels: + additionalProperties: + type: string + description: Labels added to the Kubernetes resource. + type: object + type: object + type: object + serviceAccount: + description: Template for the Kafka service account. + properties: + metadata: + description: Metadata applied to the resource. + properties: + annotations: + additionalProperties: + type: string + description: Annotations added to the Kubernetes resource. + type: object + labels: + additionalProperties: + type: string + description: Labels added to the Kubernetes resource. + type: object + type: object + type: object + statefulset: + description: Template for Kafka `StatefulSet`. + properties: + metadata: + description: Metadata applied to the resource. + properties: + annotations: + additionalProperties: + type: string + description: Annotations added to the Kubernetes resource. + type: object + labels: + additionalProperties: + type: string + description: Labels added to the Kubernetes resource. + type: object + type: object + podManagementPolicy: + description: PodManagementPolicy which will be used for this StatefulSet. Valid values are `Parallel` and `OrderedReady`. Defaults to `Parallel`. + enum: + - OrderedReady + - Parallel + type: string + type: object + type: object + tieredStorage: + description: Configure the tiered storage feature for Kafka brokers. + properties: + remoteStorageManager: + description: Configuration for the Remote Storage Manager. + properties: + className: + description: The class name for the `RemoteStorageManager` implementation. + type: string + classPath: + description: The class path for the `RemoteStorageManager` implementation. + type: string + config: + additionalProperties: + type: string + description: The additional configuration map for the `RemoteStorageManager` implementation. Keys will be automatically prefixed with `rsm.config.`, and added to Kafka broker configuration. + type: object + type: object + type: + description: Storage type, only 'custom' is supported at the moment. + enum: + - custom + type: string + required: + - type + type: object + version: + description: The Kafka broker version. Defaults to the latest version. Consult the user documentation to understand the process required to upgrade or downgrade the version. + type: string + required: + - listeners + type: object + kafkaExporter: + description: Configuration of the Kafka Exporter. Kafka Exporter can provide additional metrics, for example lag of consumer group at topic/partition. + properties: + enableSaramaLogging: + description: Enable Sarama logging, a Go client library used by the Kafka Exporter. + type: boolean + groupExcludeRegex: + description: Regular expression to specify which consumer groups to exclude. + type: string + groupRegex: + description: Regular expression to specify which consumer groups to collect. Default value is `.*`. + type: string + image: + description: The container image used for the Kafka Exporter pods. If no image name is explicitly specified, the image name corresponds to the version specified in the Cluster Operator configuration. If an image name is not defined in the Cluster Operator configuration, a default value is used. + type: string + livenessProbe: + description: Pod liveness check. + properties: + failureThreshold: + description: Minimum consecutive failures for the probe to be considered failed after having succeeded. Defaults to 3. Minimum value is 1. + minimum: 1 + type: integer + initialDelaySeconds: + description: The initial delay before first the health is first checked. Default to 15 seconds. Minimum value is 0. + minimum: 0 + type: integer + periodSeconds: + description: How often (in seconds) to perform the probe. Default to 10 seconds. Minimum value is 1. + minimum: 1 + type: integer + successThreshold: + description: Minimum consecutive successes for the probe to be considered successful after having failed. Defaults to 1. Must be 1 for liveness. Minimum value is 1. + minimum: 1 + type: integer + timeoutSeconds: + description: The timeout for each attempted health check. Default to 5 seconds. Minimum value is 1. + minimum: 1 + type: integer + type: object + logging: + description: 'Only log messages with the given severity or above. Valid levels: [`info`, `debug`, `trace`]. Default log level is `info`.' + type: string + readinessProbe: + description: Pod readiness check. + properties: + failureThreshold: + description: Minimum consecutive failures for the probe to be considered failed after having succeeded. Defaults to 3. Minimum value is 1. + minimum: 1 + type: integer + initialDelaySeconds: + description: The initial delay before first the health is first checked. Default to 15 seconds. Minimum value is 0. + minimum: 0 + type: integer + periodSeconds: + description: How often (in seconds) to perform the probe. Default to 10 seconds. Minimum value is 1. + minimum: 1 + type: integer + successThreshold: + description: Minimum consecutive successes for the probe to be considered successful after having failed. Defaults to 1. Must be 1 for liveness. Minimum value is 1. + minimum: 1 + type: integer + timeoutSeconds: + description: The timeout for each attempted health check. Default to 5 seconds. Minimum value is 1. + minimum: 1 + type: integer + type: object + resources: + description: CPU and memory resources to reserve. + properties: + claims: + items: + properties: + name: + type: string + type: object + type: array + limits: + additionalProperties: + anyOf: + - type: integer + - type: string + pattern: ^(\+|-)?(([0-9]+(\.[0-9]*)?)|(\.[0-9]+))(([KMGTPE]i)|[numkMGTPE]|([eE](\+|-)?(([0-9]+(\.[0-9]*)?)|(\.[0-9]+))))?$ + x-kubernetes-int-or-string: true + type: object + requests: + additionalProperties: + anyOf: + - type: integer + - type: string + pattern: ^(\+|-)?(([0-9]+(\.[0-9]*)?)|(\.[0-9]+))(([KMGTPE]i)|[numkMGTPE]|([eE](\+|-)?(([0-9]+(\.[0-9]*)?)|(\.[0-9]+))))?$ + x-kubernetes-int-or-string: true + type: object + type: object + showAllOffsets: + description: Whether show the offset/lag for all consumer group, otherwise, only show connected consumer groups. + type: boolean + template: + description: Customization of deployment templates and pods. + properties: + container: + description: Template for the Kafka Exporter container. + properties: + env: + description: Environment variables which should be applied to the container. + items: + oneOf: + - properties: + value: {} + required: + - value + - properties: + valueFrom: {} + required: + - valueFrom + properties: + name: + description: The environment variable key. + type: string + value: + description: The environment variable value. + type: string + valueFrom: + description: Reference to the secret or config map property to which the environment variable is set. + oneOf: + - properties: + secretKeyRef: {} + required: + - secretKeyRef + - properties: + configMapKeyRef: {} + required: + - configMapKeyRef + properties: + configMapKeyRef: + description: Reference to a key in a config map. + properties: + key: + type: string + name: + type: string + optional: + type: boolean + type: object + secretKeyRef: + description: Reference to a key in a secret. + properties: + key: + type: string + name: + type: string + optional: + type: boolean + type: object + type: object + type: object + type: array + securityContext: + description: Security context for the container. + properties: + allowPrivilegeEscalation: + type: boolean + appArmorProfile: + properties: + localhostProfile: + type: string + type: + type: string + type: object + capabilities: + properties: + add: + items: + type: string + type: array + drop: + items: + type: string + type: array + type: object + privileged: + type: boolean + procMount: + type: string + readOnlyRootFilesystem: + type: boolean + runAsGroup: + type: integer + runAsNonRoot: + type: boolean + runAsUser: + type: integer + seLinuxOptions: + properties: + level: + type: string + role: + type: string + type: + type: string + user: + type: string + type: object + seccompProfile: + properties: + localhostProfile: + type: string + type: + type: string + type: object + windowsOptions: + properties: + gmsaCredentialSpec: + type: string + gmsaCredentialSpecName: + type: string + hostProcess: + type: boolean + runAsUserName: + type: string + type: object + type: object + volumeMounts: + description: Additional volume mounts which should be applied to the container. + items: + properties: + mountPath: + type: string + mountPropagation: + type: string + name: + type: string + readOnly: + type: boolean + recursiveReadOnly: + type: string + subPath: + type: string + subPathExpr: + type: string + type: object + type: array + type: object + deployment: + description: Template for Kafka Exporter `Deployment`. + properties: + deploymentStrategy: + description: Pod replacement strategy for deployment configuration changes. Valid values are `RollingUpdate` and `Recreate`. Defaults to `RollingUpdate`. + enum: + - RollingUpdate + - Recreate + type: string + metadata: + description: Metadata applied to the resource. + properties: + annotations: + additionalProperties: + type: string + description: Annotations added to the Kubernetes resource. + type: object + labels: + additionalProperties: + type: string + description: Labels added to the Kubernetes resource. + type: object + type: object + type: object + pod: + description: Template for Kafka Exporter `Pods`. + properties: + affinity: + description: The pod's affinity rules. + properties: + nodeAffinity: + properties: + preferredDuringSchedulingIgnoredDuringExecution: + items: + properties: + preference: + properties: + matchExpressions: + items: + properties: + key: + type: string + operator: + type: string + values: + items: + type: string + type: array + type: object + type: array + matchFields: + items: + properties: + key: + type: string + operator: + type: string + values: + items: + type: string + type: array + type: object + type: array + type: object + weight: + type: integer + type: object + type: array + requiredDuringSchedulingIgnoredDuringExecution: + properties: + nodeSelectorTerms: + items: + properties: + matchExpressions: + items: + properties: + key: + type: string + operator: + type: string + values: + items: + type: string + type: array + type: object + type: array + matchFields: + items: + properties: + key: + type: string + operator: + type: string + values: + items: + type: string + type: array + type: object + type: array + type: object + type: array + type: object + type: object + podAffinity: + properties: + preferredDuringSchedulingIgnoredDuringExecution: + items: + properties: + podAffinityTerm: + properties: + labelSelector: + properties: + matchExpressions: + items: + properties: + key: + type: string + operator: + type: string + values: + items: + type: string + type: array + type: object + type: array + matchLabels: + additionalProperties: + type: string + type: object + type: object + matchLabelKeys: + items: + type: string + type: array + mismatchLabelKeys: + items: + type: string + type: array + namespaceSelector: + properties: + matchExpressions: + items: + properties: + key: + type: string + operator: + type: string + values: + items: + type: string + type: array + type: object + type: array + matchLabels: + additionalProperties: + type: string + type: object + type: object + namespaces: + items: + type: string + type: array + topologyKey: + type: string + type: object + weight: + type: integer + type: object + type: array + requiredDuringSchedulingIgnoredDuringExecution: + items: + properties: + labelSelector: + properties: + matchExpressions: + items: + properties: + key: + type: string + operator: + type: string + values: + items: + type: string + type: array + type: object + type: array + matchLabels: + additionalProperties: + type: string + type: object + type: object + matchLabelKeys: + items: + type: string + type: array + mismatchLabelKeys: + items: + type: string + type: array + namespaceSelector: + properties: + matchExpressions: + items: + properties: + key: + type: string + operator: + type: string + values: + items: + type: string + type: array + type: object + type: array + matchLabels: + additionalProperties: + type: string + type: object + type: object + namespaces: + items: + type: string + type: array + topologyKey: + type: string + type: object + type: array + type: object + podAntiAffinity: + properties: + preferredDuringSchedulingIgnoredDuringExecution: + items: + properties: + podAffinityTerm: + properties: + labelSelector: + properties: + matchExpressions: + items: + properties: + key: + type: string + operator: + type: string + values: + items: + type: string + type: array + type: object + type: array + matchLabels: + additionalProperties: + type: string + type: object + type: object + matchLabelKeys: + items: + type: string + type: array + mismatchLabelKeys: + items: + type: string + type: array + namespaceSelector: + properties: + matchExpressions: + items: + properties: + key: + type: string + operator: + type: string + values: + items: + type: string + type: array + type: object + type: array + matchLabels: + additionalProperties: + type: string + type: object + type: object + namespaces: + items: + type: string + type: array + topologyKey: + type: string + type: object + weight: + type: integer + type: object + type: array + requiredDuringSchedulingIgnoredDuringExecution: + items: + properties: + labelSelector: + properties: + matchExpressions: + items: + properties: + key: + type: string + operator: + type: string + values: + items: + type: string + type: array + type: object + type: array + matchLabels: + additionalProperties: + type: string + type: object + type: object + matchLabelKeys: + items: + type: string + type: array + mismatchLabelKeys: + items: + type: string + type: array + namespaceSelector: + properties: + matchExpressions: + items: + properties: + key: + type: string + operator: + type: string + values: + items: + type: string + type: array + type: object + type: array + matchLabels: + additionalProperties: + type: string + type: object + type: object + namespaces: + items: + type: string + type: array + topologyKey: + type: string + type: object + type: array + type: object + type: object + enableServiceLinks: + description: Indicates whether information about services should be injected into Pod's environment variables. + type: boolean + hostAliases: + description: The pod's HostAliases. HostAliases is an optional list of hosts and IPs that will be injected into the Pod's hosts file if specified. + items: + properties: + hostnames: + items: + type: string + type: array + ip: + type: string + type: object + type: array + imagePullSecrets: + description: List of references to secrets in the same namespace to use for pulling any of the images used by this Pod. When the `STRIMZI_IMAGE_PULL_SECRETS` environment variable in Cluster Operator and the `imagePullSecrets` option are specified, only the `imagePullSecrets` variable is used and the `STRIMZI_IMAGE_PULL_SECRETS` variable is ignored. + items: + properties: + name: + type: string + type: object + type: array + metadata: + description: Metadata applied to the resource. + properties: + annotations: + additionalProperties: + type: string + description: Annotations added to the Kubernetes resource. + type: object + labels: + additionalProperties: + type: string + description: Labels added to the Kubernetes resource. + type: object + type: object + priorityClassName: + description: 'The name of the priority class used to assign priority to the pods. ' + type: string + schedulerName: + description: The name of the scheduler used to dispatch this `Pod`. If not specified, the default scheduler will be used. + type: string + securityContext: + description: Configures pod-level security attributes and common container settings. + properties: + appArmorProfile: + properties: + localhostProfile: + type: string + type: + type: string + type: object + fsGroup: + type: integer + fsGroupChangePolicy: + type: string + runAsGroup: + type: integer + runAsNonRoot: + type: boolean + runAsUser: + type: integer + seLinuxOptions: + properties: + level: + type: string + role: + type: string + type: + type: string + user: + type: string + type: object + seccompProfile: + properties: + localhostProfile: + type: string + type: + type: string + type: object + supplementalGroups: + items: + type: integer + type: array + sysctls: + items: + properties: + name: + type: string + value: + type: string + type: object + type: array + windowsOptions: + properties: + gmsaCredentialSpec: + type: string + gmsaCredentialSpecName: + type: string + hostProcess: + type: boolean + runAsUserName: + type: string + type: object + type: object + terminationGracePeriodSeconds: + description: The grace period is the duration in seconds after the processes running in the pod are sent a termination signal, and the time when the processes are forcibly halted with a kill signal. Set this value to longer than the expected cleanup time for your process. Value must be a non-negative integer. A zero value indicates delete immediately. You might need to increase the grace period for very large Kafka clusters, so that the Kafka brokers have enough time to transfer their work to another broker before they are terminated. Defaults to 30 seconds. + minimum: 0 + type: integer + tmpDirSizeLimit: + description: Defines the total amount of pod memory allocated for the temporary `EmptyDir` volume `/tmp`. Specify the allocation in memory units, for example, `100Mi` for 100 mebibytes. Default value is `5Mi`. The `/tmp` volume is backed by pod memory, not disk storage, so avoid setting a high value as it consumes pod memory resources. + pattern: ^([0-9.]+)([eEinumkKMGTP]*[-+]?[0-9]*)$ + type: string + tolerations: + description: The pod's tolerations. + items: + properties: + effect: + type: string + key: + type: string + operator: + type: string + tolerationSeconds: + type: integer + value: + type: string + type: object + type: array + topologySpreadConstraints: + description: The pod's topology spread constraints. + items: + properties: + labelSelector: + properties: + matchExpressions: + items: + properties: + key: + type: string + operator: + type: string + values: + items: + type: string + type: array + type: object + type: array + matchLabels: + additionalProperties: + type: string + type: object + type: object + matchLabelKeys: + items: + type: string + type: array + maxSkew: + type: integer + minDomains: + type: integer + nodeAffinityPolicy: + type: string + nodeTaintsPolicy: + type: string + topologyKey: + type: string + whenUnsatisfiable: + type: string + type: object + type: array + volumes: + description: Additional volumes that can be mounted to the pod. + items: + oneOf: + - properties: + configMap: {} + csi: {} + emptyDir: {} + persistentVolumeClaim: {} + secret: {} + properties: + configMap: + description: ConfigMap to use to populate the volume. + properties: + defaultMode: + type: integer + items: + items: + properties: + key: + type: string + mode: + type: integer + path: + type: string + type: object + type: array + name: + type: string + optional: + type: boolean + type: object + csi: + description: CSIVolumeSource object to use to populate the volume. + properties: + driver: + type: string + fsType: + type: string + nodePublishSecretRef: + properties: + name: + type: string + type: object + readOnly: + type: boolean + volumeAttributes: + additionalProperties: + type: string + type: object + type: object + emptyDir: + description: EmptyDir to use to populate the volume. + properties: + medium: + type: string + sizeLimit: + properties: + amount: + type: string + format: + type: string + type: object + type: object + name: + description: Name to use for the volume. Required. + type: string + persistentVolumeClaim: + description: PersistentVolumeClaim object to use to populate the volume. + properties: + claimName: + type: string + readOnly: + type: boolean + type: object + secret: + description: Secret to use populate the volume. + properties: + defaultMode: + type: integer + items: + items: + properties: + key: + type: string + mode: + type: integer + path: + type: string + type: object + type: array + optional: + type: boolean + secretName: + type: string + type: object + type: object + type: array + type: object + service: + description: Template for Kafka Exporter `Service`. + properties: + metadata: + description: Metadata applied to the resource. + properties: + annotations: + additionalProperties: + type: string + description: Annotations added to the Kubernetes resource. + type: object + labels: + additionalProperties: + type: string + description: Labels added to the Kubernetes resource. + type: object + type: object + type: object + serviceAccount: + description: Template for the Kafka Exporter service account. + properties: + metadata: + description: Metadata applied to the resource. + properties: + annotations: + additionalProperties: + type: string + description: Annotations added to the Kubernetes resource. + type: object + labels: + additionalProperties: + type: string + description: Labels added to the Kubernetes resource. + type: object + type: object + type: object + type: object + topicExcludeRegex: + description: Regular expression to specify which topics to exclude. + type: string + topicRegex: + description: Regular expression to specify which topics to collect. Default value is `.*`. + type: string + type: object + maintenanceTimeWindows: + description: A list of time windows for maintenance tasks (that is, certificates renewal). Each time window is defined by a cron expression. + items: + type: string + type: array + zookeeper: + description: Configuration of the ZooKeeper cluster. This section is required when running a ZooKeeper-based Apache Kafka cluster. + properties: + config: + description: 'The ZooKeeper broker config. Properties with the following prefixes cannot be set: server., dataDir, dataLogDir, clientPort, authProvider, quorum.auth, requireClientAuthScheme, snapshot.trust.empty, standaloneEnabled, reconfigEnabled, 4lw.commands.whitelist, secureClientPort, ssl., serverCnxnFactory, sslQuorum (with the exception of: ssl.protocol, ssl.quorum.protocol, ssl.enabledProtocols, ssl.quorum.enabledProtocols, ssl.ciphersuites, ssl.quorum.ciphersuites, ssl.hostnameVerification, ssl.quorum.hostnameVerification).' + type: object + x-kubernetes-preserve-unknown-fields: true + image: + description: The container image used for ZooKeeper pods. If no image name is explicitly specified, it is determined based on the Kafka version set in `spec.kafka.version`. The image names are specifically mapped to corresponding versions in the Cluster Operator configuration. + type: string + jmxOptions: + description: JMX Options for Zookeeper nodes. + properties: + authentication: + description: Authentication configuration for connecting to the JMX port. + properties: + type: + description: Authentication type. Currently the only supported types are `password`.`password` type creates a username and protected port with no TLS. + enum: + - password + type: string + required: + - type + type: object + type: object + jvmOptions: + description: JVM Options for pods. + properties: + -XX: + additionalProperties: + type: string + description: A map of -XX options to the JVM. + type: object + -Xms: + description: -Xms option to to the JVM. + pattern: ^[0-9]+[mMgG]?$ + type: string + -Xmx: + description: -Xmx option to to the JVM. + pattern: ^[0-9]+[mMgG]?$ + type: string + gcLoggingEnabled: + description: Specifies whether the Garbage Collection logging is enabled. The default is false. + type: boolean + javaSystemProperties: + description: A map of additional system properties which will be passed using the `-D` option to the JVM. + items: + properties: + name: + description: The system property name. + type: string + value: + description: The system property value. + type: string + type: object + type: array + type: object + livenessProbe: + description: Pod liveness checking. + properties: + failureThreshold: + description: Minimum consecutive failures for the probe to be considered failed after having succeeded. Defaults to 3. Minimum value is 1. + minimum: 1 + type: integer + initialDelaySeconds: + description: The initial delay before first the health is first checked. Default to 15 seconds. Minimum value is 0. + minimum: 0 + type: integer + periodSeconds: + description: How often (in seconds) to perform the probe. Default to 10 seconds. Minimum value is 1. + minimum: 1 + type: integer + successThreshold: + description: Minimum consecutive successes for the probe to be considered successful after having failed. Defaults to 1. Must be 1 for liveness. Minimum value is 1. + minimum: 1 + type: integer + timeoutSeconds: + description: The timeout for each attempted health check. Default to 5 seconds. Minimum value is 1. + minimum: 1 + type: integer + type: object + logging: + description: Logging configuration for ZooKeeper. + properties: + loggers: + additionalProperties: + type: string + description: A Map from logger name to logger level. + type: object + type: + description: Logging type, must be either 'inline' or 'external'. + enum: + - inline + - external + type: string + valueFrom: + description: '`ConfigMap` entry where the logging configuration is stored. ' + properties: + configMapKeyRef: + description: Reference to the key in the ConfigMap containing the configuration. + properties: + key: + type: string + name: + type: string + optional: + type: boolean + type: object + type: object + required: + - type + type: object + metricsConfig: + description: Metrics configuration. + properties: + type: + description: Metrics type. Only 'jmxPrometheusExporter' supported currently. + enum: + - jmxPrometheusExporter + type: string + valueFrom: + description: 'ConfigMap entry where the Prometheus JMX Exporter configuration is stored. ' + properties: + configMapKeyRef: + description: Reference to the key in the ConfigMap containing the configuration. + properties: + key: + type: string + name: + type: string + optional: + type: boolean + type: object + type: object + required: + - type + - valueFrom + type: object + readinessProbe: + description: Pod readiness checking. + properties: + failureThreshold: + description: Minimum consecutive failures for the probe to be considered failed after having succeeded. Defaults to 3. Minimum value is 1. + minimum: 1 + type: integer + initialDelaySeconds: + description: The initial delay before first the health is first checked. Default to 15 seconds. Minimum value is 0. + minimum: 0 + type: integer + periodSeconds: + description: How often (in seconds) to perform the probe. Default to 10 seconds. Minimum value is 1. + minimum: 1 + type: integer + successThreshold: + description: Minimum consecutive successes for the probe to be considered successful after having failed. Defaults to 1. Must be 1 for liveness. Minimum value is 1. + minimum: 1 + type: integer + timeoutSeconds: + description: The timeout for each attempted health check. Default to 5 seconds. Minimum value is 1. + minimum: 1 + type: integer + type: object + replicas: + description: The number of pods in the cluster. + minimum: 1 + type: integer + resources: + description: CPU and memory resources to reserve. + properties: + claims: + items: + properties: + name: + type: string + type: object + type: array + limits: + additionalProperties: + anyOf: + - type: integer + - type: string + pattern: ^(\+|-)?(([0-9]+(\.[0-9]*)?)|(\.[0-9]+))(([KMGTPE]i)|[numkMGTPE]|([eE](\+|-)?(([0-9]+(\.[0-9]*)?)|(\.[0-9]+))))?$ + x-kubernetes-int-or-string: true + type: object + requests: + additionalProperties: + anyOf: + - type: integer + - type: string + pattern: ^(\+|-)?(([0-9]+(\.[0-9]*)?)|(\.[0-9]+))(([KMGTPE]i)|[numkMGTPE]|([eE](\+|-)?(([0-9]+(\.[0-9]*)?)|(\.[0-9]+))))?$ + x-kubernetes-int-or-string: true + type: object + type: object + storage: + description: Storage configuration (disk). Cannot be updated. + properties: + class: + description: The storage class to use for dynamic volume allocation. + type: string + deleteClaim: + description: Specifies if the persistent volume claim has to be deleted when the cluster is un-deployed. + type: boolean + id: + description: Storage identification number. Mandatory for storage volumes defined with a `jbod` storage type configuration. + minimum: 0 + type: integer + kraftMetadata: + description: Specifies whether this volume should be used for storing KRaft metadata. This property is optional. When set, the only currently supported value is `shared`. At most one volume can have this property set. + enum: + - shared + type: string + overrides: + description: Overrides for individual brokers. The `overrides` field allows you to specify a different configuration for different brokers. + items: + properties: + broker: + description: Id of the kafka broker (broker identifier). + type: integer + class: + description: The storage class to use for dynamic volume allocation for this broker. + type: string + type: object + type: array + selector: + additionalProperties: + type: string + description: Specifies a specific persistent volume to use. It contains key:value pairs representing labels for selecting such a volume. + type: object + size: + description: When `type=persistent-claim`, defines the size of the persistent volume claim, such as 100Gi. Mandatory when `type=persistent-claim`. + type: string + sizeLimit: + description: When type=ephemeral, defines the total amount of local storage required for this EmptyDir volume (for example 1Gi). + pattern: ^([0-9.]+)([eEinumkKMGTP]*[-+]?[0-9]*)$ + type: string + type: + description: Storage type, must be either 'ephemeral' or 'persistent-claim'. + enum: + - ephemeral + - persistent-claim + type: string + required: + - type + type: object + template: + description: Template for ZooKeeper cluster resources. The template allows users to specify how the Kubernetes resources are generated. + properties: + clientService: + description: Template for ZooKeeper client `Service`. + properties: + ipFamilies: + description: Specifies the IP Families used by the service. Available options are `IPv4` and `IPv6`. If unspecified, Kubernetes will choose the default value based on the `ipFamilyPolicy` setting. + items: + enum: + - IPv4 + - IPv6 + type: string + type: array + ipFamilyPolicy: + description: Specifies the IP Family Policy used by the service. Available options are `SingleStack`, `PreferDualStack` and `RequireDualStack`. `SingleStack` is for a single IP family. `PreferDualStack` is for two IP families on dual-stack configured clusters or a single IP family on single-stack clusters. `RequireDualStack` fails unless there are two IP families on dual-stack configured clusters. If unspecified, Kubernetes will choose the default value based on the service type. + enum: + - SingleStack + - PreferDualStack + - RequireDualStack + type: string + metadata: + description: Metadata applied to the resource. + properties: + annotations: + additionalProperties: + type: string + description: Annotations added to the Kubernetes resource. + type: object + labels: + additionalProperties: + type: string + description: Labels added to the Kubernetes resource. + type: object + type: object + type: object + jmxSecret: + description: Template for Secret of the Zookeeper Cluster JMX authentication. + properties: + metadata: + description: Metadata applied to the resource. + properties: + annotations: + additionalProperties: + type: string + description: Annotations added to the Kubernetes resource. + type: object + labels: + additionalProperties: + type: string + description: Labels added to the Kubernetes resource. + type: object + type: object + type: object + nodesService: + description: Template for ZooKeeper nodes `Service`. + properties: + ipFamilies: + description: Specifies the IP Families used by the service. Available options are `IPv4` and `IPv6`. If unspecified, Kubernetes will choose the default value based on the `ipFamilyPolicy` setting. + items: + enum: + - IPv4 + - IPv6 + type: string + type: array + ipFamilyPolicy: + description: Specifies the IP Family Policy used by the service. Available options are `SingleStack`, `PreferDualStack` and `RequireDualStack`. `SingleStack` is for a single IP family. `PreferDualStack` is for two IP families on dual-stack configured clusters or a single IP family on single-stack clusters. `RequireDualStack` fails unless there are two IP families on dual-stack configured clusters. If unspecified, Kubernetes will choose the default value based on the service type. + enum: + - SingleStack + - PreferDualStack + - RequireDualStack + type: string + metadata: + description: Metadata applied to the resource. + properties: + annotations: + additionalProperties: + type: string + description: Annotations added to the Kubernetes resource. + type: object + labels: + additionalProperties: + type: string + description: Labels added to the Kubernetes resource. + type: object + type: object + type: object + persistentVolumeClaim: + description: Template for all ZooKeeper `PersistentVolumeClaims`. + properties: + metadata: + description: Metadata applied to the resource. + properties: + annotations: + additionalProperties: + type: string + description: Annotations added to the Kubernetes resource. + type: object + labels: + additionalProperties: + type: string + description: Labels added to the Kubernetes resource. + type: object + type: object + type: object + pod: + description: Template for ZooKeeper `Pods`. + properties: + affinity: + description: The pod's affinity rules. + properties: + nodeAffinity: + properties: + preferredDuringSchedulingIgnoredDuringExecution: + items: + properties: + preference: + properties: + matchExpressions: + items: + properties: + key: + type: string + operator: + type: string + values: + items: + type: string + type: array + type: object + type: array + matchFields: + items: + properties: + key: + type: string + operator: + type: string + values: + items: + type: string + type: array + type: object + type: array + type: object + weight: + type: integer + type: object + type: array + requiredDuringSchedulingIgnoredDuringExecution: + properties: + nodeSelectorTerms: + items: + properties: + matchExpressions: + items: + properties: + key: + type: string + operator: + type: string + values: + items: + type: string + type: array + type: object + type: array + matchFields: + items: + properties: + key: + type: string + operator: + type: string + values: + items: + type: string + type: array + type: object + type: array + type: object + type: array + type: object + type: object + podAffinity: + properties: + preferredDuringSchedulingIgnoredDuringExecution: + items: + properties: + podAffinityTerm: + properties: + labelSelector: + properties: + matchExpressions: + items: + properties: + key: + type: string + operator: + type: string + values: + items: + type: string + type: array + type: object + type: array + matchLabels: + additionalProperties: + type: string + type: object + type: object + matchLabelKeys: + items: + type: string + type: array + mismatchLabelKeys: + items: + type: string + type: array + namespaceSelector: + properties: + matchExpressions: + items: + properties: + key: + type: string + operator: + type: string + values: + items: + type: string + type: array + type: object + type: array + matchLabels: + additionalProperties: + type: string + type: object + type: object + namespaces: + items: + type: string + type: array + topologyKey: + type: string + type: object + weight: + type: integer + type: object + type: array + requiredDuringSchedulingIgnoredDuringExecution: + items: + properties: + labelSelector: + properties: + matchExpressions: + items: + properties: + key: + type: string + operator: + type: string + values: + items: + type: string + type: array + type: object + type: array + matchLabels: + additionalProperties: + type: string + type: object + type: object + matchLabelKeys: + items: + type: string + type: array + mismatchLabelKeys: + items: + type: string + type: array + namespaceSelector: + properties: + matchExpressions: + items: + properties: + key: + type: string + operator: + type: string + values: + items: + type: string + type: array + type: object + type: array + matchLabels: + additionalProperties: + type: string + type: object + type: object + namespaces: + items: + type: string + type: array + topologyKey: + type: string + type: object + type: array + type: object + podAntiAffinity: + properties: + preferredDuringSchedulingIgnoredDuringExecution: + items: + properties: + podAffinityTerm: + properties: + labelSelector: + properties: + matchExpressions: + items: + properties: + key: + type: string + operator: + type: string + values: + items: + type: string + type: array + type: object + type: array + matchLabels: + additionalProperties: + type: string + type: object + type: object + matchLabelKeys: + items: + type: string + type: array + mismatchLabelKeys: + items: + type: string + type: array + namespaceSelector: + properties: + matchExpressions: + items: + properties: + key: + type: string + operator: + type: string + values: + items: + type: string + type: array + type: object + type: array + matchLabels: + additionalProperties: + type: string + type: object + type: object + namespaces: + items: + type: string + type: array + topologyKey: + type: string + type: object + weight: + type: integer + type: object + type: array + requiredDuringSchedulingIgnoredDuringExecution: + items: + properties: + labelSelector: + properties: + matchExpressions: + items: + properties: + key: + type: string + operator: + type: string + values: + items: + type: string + type: array + type: object + type: array + matchLabels: + additionalProperties: + type: string + type: object + type: object + matchLabelKeys: + items: + type: string + type: array + mismatchLabelKeys: + items: + type: string + type: array + namespaceSelector: + properties: + matchExpressions: + items: + properties: + key: + type: string + operator: + type: string + values: + items: + type: string + type: array + type: object + type: array + matchLabels: + additionalProperties: + type: string + type: object + type: object + namespaces: + items: + type: string + type: array + topologyKey: + type: string + type: object + type: array + type: object + type: object + enableServiceLinks: + description: Indicates whether information about services should be injected into Pod's environment variables. + type: boolean + hostAliases: + description: The pod's HostAliases. HostAliases is an optional list of hosts and IPs that will be injected into the Pod's hosts file if specified. + items: + properties: + hostnames: + items: + type: string + type: array + ip: + type: string + type: object + type: array + imagePullSecrets: + description: List of references to secrets in the same namespace to use for pulling any of the images used by this Pod. When the `STRIMZI_IMAGE_PULL_SECRETS` environment variable in Cluster Operator and the `imagePullSecrets` option are specified, only the `imagePullSecrets` variable is used and the `STRIMZI_IMAGE_PULL_SECRETS` variable is ignored. + items: + properties: + name: + type: string + type: object + type: array + metadata: + description: Metadata applied to the resource. + properties: + annotations: + additionalProperties: + type: string + description: Annotations added to the Kubernetes resource. + type: object + labels: + additionalProperties: + type: string + description: Labels added to the Kubernetes resource. + type: object + type: object + priorityClassName: + description: 'The name of the priority class used to assign priority to the pods. ' + type: string + schedulerName: + description: The name of the scheduler used to dispatch this `Pod`. If not specified, the default scheduler will be used. + type: string + securityContext: + description: Configures pod-level security attributes and common container settings. + properties: + appArmorProfile: + properties: + localhostProfile: + type: string + type: + type: string + type: object + fsGroup: + type: integer + fsGroupChangePolicy: + type: string + runAsGroup: + type: integer + runAsNonRoot: + type: boolean + runAsUser: + type: integer + seLinuxOptions: + properties: + level: + type: string + role: + type: string + type: + type: string + user: + type: string + type: object + seccompProfile: + properties: + localhostProfile: + type: string + type: + type: string + type: object + supplementalGroups: + items: + type: integer + type: array + sysctls: + items: + properties: + name: + type: string + value: + type: string + type: object + type: array + windowsOptions: + properties: + gmsaCredentialSpec: + type: string + gmsaCredentialSpecName: + type: string + hostProcess: + type: boolean + runAsUserName: + type: string + type: object + type: object + terminationGracePeriodSeconds: + description: The grace period is the duration in seconds after the processes running in the pod are sent a termination signal, and the time when the processes are forcibly halted with a kill signal. Set this value to longer than the expected cleanup time for your process. Value must be a non-negative integer. A zero value indicates delete immediately. You might need to increase the grace period for very large Kafka clusters, so that the Kafka brokers have enough time to transfer their work to another broker before they are terminated. Defaults to 30 seconds. + minimum: 0 + type: integer + tmpDirSizeLimit: + description: Defines the total amount of pod memory allocated for the temporary `EmptyDir` volume `/tmp`. Specify the allocation in memory units, for example, `100Mi` for 100 mebibytes. Default value is `5Mi`. The `/tmp` volume is backed by pod memory, not disk storage, so avoid setting a high value as it consumes pod memory resources. + pattern: ^([0-9.]+)([eEinumkKMGTP]*[-+]?[0-9]*)$ + type: string + tolerations: + description: The pod's tolerations. + items: + properties: + effect: + type: string + key: + type: string + operator: + type: string + tolerationSeconds: + type: integer + value: + type: string + type: object + type: array + topologySpreadConstraints: + description: The pod's topology spread constraints. + items: + properties: + labelSelector: + properties: + matchExpressions: + items: + properties: + key: + type: string + operator: + type: string + values: + items: + type: string + type: array + type: object + type: array + matchLabels: + additionalProperties: + type: string + type: object + type: object + matchLabelKeys: + items: + type: string + type: array + maxSkew: + type: integer + minDomains: + type: integer + nodeAffinityPolicy: + type: string + nodeTaintsPolicy: + type: string + topologyKey: + type: string + whenUnsatisfiable: + type: string + type: object + type: array + volumes: + description: Additional volumes that can be mounted to the pod. + items: + oneOf: + - properties: + configMap: {} + csi: {} + emptyDir: {} + persistentVolumeClaim: {} + secret: {} + properties: + configMap: + description: ConfigMap to use to populate the volume. + properties: + defaultMode: + type: integer + items: + items: + properties: + key: + type: string + mode: + type: integer + path: + type: string + type: object + type: array + name: + type: string + optional: + type: boolean + type: object + csi: + description: CSIVolumeSource object to use to populate the volume. + properties: + driver: + type: string + fsType: + type: string + nodePublishSecretRef: + properties: + name: + type: string + type: object + readOnly: + type: boolean + volumeAttributes: + additionalProperties: + type: string + type: object + type: object + emptyDir: + description: EmptyDir to use to populate the volume. + properties: + medium: + type: string + sizeLimit: + properties: + amount: + type: string + format: + type: string + type: object + type: object + name: + description: Name to use for the volume. Required. + type: string + persistentVolumeClaim: + description: PersistentVolumeClaim object to use to populate the volume. + properties: + claimName: + type: string + readOnly: + type: boolean + type: object + secret: + description: Secret to use populate the volume. + properties: + defaultMode: + type: integer + items: + items: + properties: + key: + type: string + mode: + type: integer + path: + type: string + type: object + type: array + optional: + type: boolean + secretName: + type: string + type: object + type: object + type: array + type: object + podDisruptionBudget: + description: Template for ZooKeeper `PodDisruptionBudget`. + properties: + maxUnavailable: + description: Maximum number of unavailable pods to allow automatic Pod eviction. A Pod eviction is allowed when the `maxUnavailable` number of pods or fewer are unavailable after the eviction. Setting this value to 0 prevents all voluntary evictions, so the pods must be evicted manually. Defaults to 1. + minimum: 0 + type: integer + metadata: + description: Metadata to apply to the `PodDisruptionBudgetTemplate` resource. + properties: + annotations: + additionalProperties: + type: string + description: Annotations added to the Kubernetes resource. + type: object + labels: + additionalProperties: + type: string + description: Labels added to the Kubernetes resource. + type: object + type: object + type: object + podSet: + description: Template for ZooKeeper `StrimziPodSet` resource. + properties: + metadata: + description: Metadata applied to the resource. + properties: + annotations: + additionalProperties: + type: string + description: Annotations added to the Kubernetes resource. + type: object + labels: + additionalProperties: + type: string + description: Labels added to the Kubernetes resource. + type: object + type: object + type: object + serviceAccount: + description: Template for the ZooKeeper service account. + properties: + metadata: + description: Metadata applied to the resource. + properties: + annotations: + additionalProperties: + type: string + description: Annotations added to the Kubernetes resource. + type: object + labels: + additionalProperties: + type: string + description: Labels added to the Kubernetes resource. + type: object + type: object + type: object + statefulset: + description: Template for ZooKeeper `StatefulSet`. + properties: + metadata: + description: Metadata applied to the resource. + properties: + annotations: + additionalProperties: + type: string + description: Annotations added to the Kubernetes resource. + type: object + labels: + additionalProperties: + type: string + description: Labels added to the Kubernetes resource. + type: object + type: object + podManagementPolicy: + description: PodManagementPolicy which will be used for this StatefulSet. Valid values are `Parallel` and `OrderedReady`. Defaults to `Parallel`. + enum: + - OrderedReady + - Parallel + type: string + type: object + zookeeperContainer: + description: Template for the ZooKeeper container. + properties: + env: + description: Environment variables which should be applied to the container. + items: + oneOf: + - properties: + value: {} + required: + - value + - properties: + valueFrom: {} + required: + - valueFrom + properties: + name: + description: The environment variable key. + type: string + value: + description: The environment variable value. + type: string + valueFrom: + description: Reference to the secret or config map property to which the environment variable is set. + oneOf: + - properties: + secretKeyRef: {} + required: + - secretKeyRef + - properties: + configMapKeyRef: {} + required: + - configMapKeyRef + properties: + configMapKeyRef: + description: Reference to a key in a config map. + properties: + key: + type: string + name: + type: string + optional: + type: boolean + type: object + secretKeyRef: + description: Reference to a key in a secret. + properties: + key: + type: string + name: + type: string + optional: + type: boolean + type: object + type: object + type: object + type: array + securityContext: + description: Security context for the container. + properties: + allowPrivilegeEscalation: + type: boolean + appArmorProfile: + properties: + localhostProfile: + type: string + type: + type: string + type: object + capabilities: + properties: + add: + items: + type: string + type: array + drop: + items: + type: string + type: array + type: object + privileged: + type: boolean + procMount: + type: string + readOnlyRootFilesystem: + type: boolean + runAsGroup: + type: integer + runAsNonRoot: + type: boolean + runAsUser: + type: integer + seLinuxOptions: + properties: + level: + type: string + role: + type: string + type: + type: string + user: + type: string + type: object + seccompProfile: + properties: + localhostProfile: + type: string + type: + type: string + type: object + windowsOptions: + properties: + gmsaCredentialSpec: + type: string + gmsaCredentialSpecName: + type: string + hostProcess: + type: boolean + runAsUserName: + type: string + type: object + type: object + volumeMounts: + description: Additional volume mounts which should be applied to the container. + items: + properties: + mountPath: + type: string + mountPropagation: + type: string + name: + type: string + readOnly: + type: boolean + recursiveReadOnly: + type: string + subPath: + type: string + subPathExpr: + type: string + type: object + type: array + type: object + type: object + required: + - replicas + - storage + type: object + required: + - kafka + type: object + status: + description: The status of the Kafka and ZooKeeper clusters, and Topic Operator. + properties: + autoRebalance: + description: The status of an auto-rebalancing triggered by a cluster scaling request. + properties: + lastTransitionTime: + description: The timestamp of the latest auto-rebalancing state update. + type: string + modes: + description: "List of modes where an auto-rebalancing operation is either running or queued. \nEach mode entry (`add-brokers` or `remove-brokers`) includes one of the following: \n\n* Broker IDs for a current auto-rebalance. \n* Broker IDs for a queued auto-rebalance (if a previous rebalance is still in progress)." + items: + properties: + brokers: + description: "List of broker IDs involved in an auto-rebalancing operation related to the current mode. \nThe list contains one of the following: \n\n* Broker IDs for a current auto-rebalance. \n* Broker IDs for a queued auto-rebalance (if a previous auto-rebalance is still in progress). \n" + items: + type: integer + type: array + mode: + description: Mode for which there is an auto-rebalancing operation in progress or queued, when brokers are added or removed. The possible modes are `add-brokers` and `remove-brokers`. + enum: + - add-brokers + - remove-brokers + type: string + type: object + type: array + state: + description: "The current state of an auto-rebalancing operation. Possible values are: \n\n* `Idle` as the initial state when an auto-rebalancing is requested or as final state when it completes or fails.\n* `RebalanceOnScaleDown` if an auto-rebalance related to a scale-down operation is running.\n* `RebalanceOnScaleUp` if an auto-rebalance related to a scale-up operation is running." + enum: + - Idle + - RebalanceOnScaleDown + - RebalanceOnScaleUp + type: string + type: object + clusterId: + description: Kafka cluster Id. + type: string + conditions: + description: List of status conditions. + items: + properties: + lastTransitionTime: + description: Last time the condition of a type changed from one status to another. The required format is 'yyyy-MM-ddTHH:mm:ssZ', in the UTC time zone. + type: string + message: + description: Human-readable message indicating details about the condition's last transition. + type: string + reason: + description: The reason for the condition's last transition (a single word in CamelCase). + type: string + status: + description: The status of the condition, either True, False or Unknown. + type: string + type: + description: The unique identifier of a condition, used to distinguish between other conditions in the resource. + type: string + type: object + type: array + kafkaMetadataState: + description: 'Defines where cluster metadata are stored. Possible values are: ZooKeeper if the metadata are stored in ZooKeeper; KRaftMigration if the controllers are connected to ZooKeeper, brokers are being rolled with Zookeeper migration enabled and connection information to controllers, and the metadata migration process is running; KRaftDualWriting if the metadata migration process finished and the cluster is in dual-write mode; KRaftPostMigration if the brokers are fully KRaft-based but controllers being rolled to disconnect from ZooKeeper; PreKRaft if brokers and controller are fully KRaft-based, metadata are stored in KRaft, but ZooKeeper must be deleted; KRaft if the metadata are stored in KRaft.' + enum: + - ZooKeeper + - KRaftMigration + - KRaftDualWriting + - KRaftPostMigration + - PreKRaft + - KRaft + type: string + kafkaMetadataVersion: + description: The KRaft metadata.version currently used by the Kafka cluster. + type: string + kafkaNodePools: + description: List of the KafkaNodePools used by this Kafka cluster. + items: + properties: + name: + description: The name of the KafkaNodePool used by this Kafka resource. + type: string + type: object + type: array + kafkaVersion: + description: The version of Kafka currently deployed in the cluster. + type: string + listeners: + description: Addresses of the internal and external listeners. + items: + properties: + addresses: + description: A list of the addresses for this listener. + items: + properties: + host: + description: The DNS name or IP address of the Kafka bootstrap service. + type: string + port: + description: The port of the Kafka bootstrap service. + type: integer + type: object + type: array + bootstrapServers: + description: A comma-separated list of `host:port` pairs for connecting to the Kafka cluster using this listener. + type: string + certificates: + description: A list of TLS certificates which can be used to verify the identity of the server when connecting to the given listener. Set only for `tls` and `external` listeners. + items: + type: string + type: array + name: + description: The name of the listener. + type: string + type: + description: The name of the listener. + type: string + type: object + type: array + observedGeneration: + description: The generation of the CRD that was last reconciled by the operator. + type: integer + operatorLastSuccessfulVersion: + description: The version of the Strimzi Cluster Operator which performed the last successful reconciliation. + type: string + registeredNodeIds: + description: Registered node IDs used by this Kafka cluster. This field is used for internal purposes only and will be removed in the future. + items: + type: integer + type: array + type: object + type: object + served: true + storage: true + subresources: + status: {} +--- +apiVersion: apiextensions.k8s.io/v1 +kind: CustomResourceDefinition +metadata: + labels: + app: strimzi + strimzi.io/crd-install: "true" + name: kafkabridges.kafka.strimzi.io +spec: + conversion: + strategy: None + group: kafka.strimzi.io + names: + categories: + - strimzi + kind: KafkaBridge + listKind: KafkaBridgeList + plural: kafkabridges + shortNames: + - kb + singular: kafkabridge + scope: Namespaced + versions: + - additionalPrinterColumns: + - description: The desired number of Kafka Bridge replicas + jsonPath: .spec.replicas + name: Desired replicas + type: integer + - description: The boostrap servers + jsonPath: .spec.bootstrapServers + name: Bootstrap Servers + priority: 1 + type: string + - description: The state of the custom resource + jsonPath: .status.conditions[?(@.type=="Ready")].status + name: Ready + type: string + name: v1beta2 + schema: + openAPIV3Schema: + properties: + apiVersion: + description: 'APIVersion defines the versioned schema of this representation of an object. Servers should convert recognized schemas to the latest internal value, and may reject unrecognized values. More info: https://git.k8s.io/community/contributors/devel/sig-architecture/api-conventions.md#resources' + type: string + kind: + description: 'Kind is a string value representing the REST resource this object represents. Servers may infer this from the endpoint the client submits requests to. Cannot be updated. In CamelCase. More info: https://git.k8s.io/community/contributors/devel/sig-architecture/api-conventions.md#types-kinds' + type: string + metadata: + type: object + spec: + description: The specification of the Kafka Bridge. + properties: + adminClient: + description: Kafka AdminClient related configuration. + properties: + config: + description: The Kafka AdminClient configuration used for AdminClient instances created by the bridge. + type: object + x-kubernetes-preserve-unknown-fields: true + type: object + authentication: + description: Authentication configuration for connecting to the cluster. + properties: + accessToken: + description: Link to Kubernetes Secret containing the access token which was obtained from the authorization server. + properties: + key: + description: The key under which the secret value is stored in the Kubernetes Secret. + type: string + secretName: + description: The name of the Kubernetes Secret containing the secret value. + type: string + required: + - key + - secretName + type: object + accessTokenIsJwt: + description: Configure whether access token should be treated as JWT. This should be set to `false` if the authorization server returns opaque tokens. Defaults to `true`. + type: boolean + accessTokenLocation: + description: Path to the token file containing an access token to be used for authentication. + type: string + audience: + description: OAuth audience to use when authenticating against the authorization server. Some authorization servers require the audience to be explicitly set. The possible values depend on how the authorization server is configured. By default, `audience` is not specified when performing the token endpoint request. + type: string + certificateAndKey: + description: Reference to the `Secret` which holds the certificate and private key pair. + properties: + certificate: + description: The name of the file certificate in the Secret. + type: string + key: + description: The name of the private key in the Secret. + type: string + secretName: + description: The name of the Secret containing the certificate. + type: string + required: + - secretName + - certificate + - key + type: object + clientAssertion: + description: Link to Kubernetes secret containing the client assertion which was manually configured for the client. + properties: + key: + description: The key under which the secret value is stored in the Kubernetes Secret. + type: string + secretName: + description: The name of the Kubernetes Secret containing the secret value. + type: string + required: + - key + - secretName + type: object + clientAssertionLocation: + description: Path to the file containing the client assertion to be used for authentication. + type: string + clientAssertionType: + description: The client assertion type. If not set, and either `clientAssertion` or `clientAssertionLocation` is configured, this value defaults to `urn:ietf:params:oauth:client-assertion-type:jwt-bearer`. + type: string + clientId: + description: OAuth Client ID which the Kafka client can use to authenticate against the OAuth server and use the token endpoint URI. + type: string + clientSecret: + description: Link to Kubernetes Secret containing the OAuth client secret which the Kafka client can use to authenticate against the OAuth server and use the token endpoint URI. + properties: + key: + description: The key under which the secret value is stored in the Kubernetes Secret. + type: string + secretName: + description: The name of the Kubernetes Secret containing the secret value. + type: string + required: + - key + - secretName + type: object + connectTimeoutSeconds: + description: The connect timeout in seconds when connecting to authorization server. If not set, the effective connect timeout is 60 seconds. + type: integer + disableTlsHostnameVerification: + description: Enable or disable TLS hostname verification. Default value is `false`. + type: boolean + enableMetrics: + description: Enable or disable OAuth metrics. Default value is `false`. + type: boolean + httpRetries: + description: The maximum number of retries to attempt if an initial HTTP request fails. If not set, the default is to not attempt any retries. + type: integer + httpRetryPauseMs: + description: The pause to take before retrying a failed HTTP request. If not set, the default is to not pause at all but to immediately repeat a request. + type: integer + includeAcceptHeader: + description: Whether the Accept header should be set in requests to the authorization servers. The default value is `true`. + type: boolean + maxTokenExpirySeconds: + description: Set or limit time-to-live of the access tokens to the specified number of seconds. This should be set if the authorization server returns opaque tokens. + type: integer + passwordSecret: + description: Reference to the `Secret` which holds the password. + properties: + password: + description: The name of the key in the Secret under which the password is stored. + type: string + secretName: + description: The name of the Secret containing the password. + type: string + required: + - secretName + - password + type: object + readTimeoutSeconds: + description: The read timeout in seconds when connecting to authorization server. If not set, the effective read timeout is 60 seconds. + type: integer + refreshToken: + description: Link to Kubernetes Secret containing the refresh token which can be used to obtain access token from the authorization server. + properties: + key: + description: The key under which the secret value is stored in the Kubernetes Secret. + type: string + secretName: + description: The name of the Kubernetes Secret containing the secret value. + type: string + required: + - key + - secretName + type: object + saslExtensions: + additionalProperties: + type: string + description: SASL extensions parameters. + type: object + scope: + description: OAuth scope to use when authenticating against the authorization server. Some authorization servers require this to be set. The possible values depend on how authorization server is configured. By default `scope` is not specified when doing the token endpoint request. + type: string + tlsTrustedCertificates: + description: Trusted certificates for TLS connection to the OAuth server. + items: + oneOf: + - properties: + certificate: {} + required: + - certificate + - properties: + pattern: {} + required: + - pattern + properties: + certificate: + description: The name of the file certificate in the secret. + type: string + pattern: + description: Pattern for the certificate files in the secret. Use the link:https://en.wikipedia.org/wiki/Glob_(programming)[_glob syntax_] for the pattern. All files in the secret that match the pattern are used. + type: string + secretName: + description: The name of the Secret containing the certificate. + type: string + required: + - secretName + type: object + type: array + tokenEndpointUri: + description: Authorization server token endpoint URI. + type: string + type: + description: Authentication type. Currently the supported types are `tls`, `scram-sha-256`, `scram-sha-512`, `plain`, and 'oauth'. `scram-sha-256` and `scram-sha-512` types use SASL SCRAM-SHA-256 and SASL SCRAM-SHA-512 Authentication, respectively. `plain` type uses SASL PLAIN Authentication. `oauth` type uses SASL OAUTHBEARER Authentication. The `tls` type uses TLS Client Authentication. The `tls` type is supported only over TLS connections. + enum: + - tls + - scram-sha-256 + - scram-sha-512 + - plain + - oauth + type: string + username: + description: Username used for the authentication. + type: string + required: + - type + type: object + bootstrapServers: + description: A list of host:port pairs for establishing the initial connection to the Kafka cluster. + type: string + clientRackInitImage: + description: The image of the init container used for initializing the `client.rack`. + type: string + consumer: + description: Kafka consumer related configuration. + properties: + config: + description: 'The Kafka consumer configuration used for consumer instances created by the bridge. Properties with the following prefixes cannot be set: ssl., bootstrap.servers, group.id, sasl., security. (with the exception of: ssl.endpoint.identification.algorithm, ssl.cipher.suites, ssl.protocol, ssl.enabled.protocols).' + type: object + x-kubernetes-preserve-unknown-fields: true + enabled: + description: Whether the HTTP consumer should be enabled or disabled. The default is enabled (`true`). + type: boolean + timeoutSeconds: + description: The timeout in seconds for deleting inactive consumers, default is -1 (disabled). + type: integer + type: object + enableMetrics: + description: Enable the metrics for the Kafka Bridge. Default is false. + type: boolean + http: + description: The HTTP related configuration. + properties: + cors: + description: CORS configuration for the HTTP Bridge. + properties: + allowedMethods: + description: List of allowed HTTP methods. + items: + type: string + type: array + allowedOrigins: + description: List of allowed origins. Java regular expressions can be used. + items: + type: string + type: array + required: + - allowedOrigins + - allowedMethods + type: object + port: + description: The port which is the server listening on. + minimum: 1023 + type: integer + type: object + image: + description: The container image used for Kafka Bridge pods. If no image name is explicitly specified, the image name corresponds to the image specified in the Cluster Operator configuration. If an image name is not defined in the Cluster Operator configuration, a default value is used. + type: string + jvmOptions: + description: '**Currently not supported** JVM Options for pods.' + properties: + -XX: + additionalProperties: + type: string + description: A map of -XX options to the JVM. + type: object + -Xms: + description: -Xms option to to the JVM. + pattern: ^[0-9]+[mMgG]?$ + type: string + -Xmx: + description: -Xmx option to to the JVM. + pattern: ^[0-9]+[mMgG]?$ + type: string + gcLoggingEnabled: + description: Specifies whether the Garbage Collection logging is enabled. The default is false. + type: boolean + javaSystemProperties: + description: A map of additional system properties which will be passed using the `-D` option to the JVM. + items: + properties: + name: + description: The system property name. + type: string + value: + description: The system property value. + type: string + type: object + type: array + type: object + livenessProbe: + description: Pod liveness checking. + properties: + failureThreshold: + description: Minimum consecutive failures for the probe to be considered failed after having succeeded. Defaults to 3. Minimum value is 1. + minimum: 1 + type: integer + initialDelaySeconds: + description: The initial delay before first the health is first checked. Default to 15 seconds. Minimum value is 0. + minimum: 0 + type: integer + periodSeconds: + description: How often (in seconds) to perform the probe. Default to 10 seconds. Minimum value is 1. + minimum: 1 + type: integer + successThreshold: + description: Minimum consecutive successes for the probe to be considered successful after having failed. Defaults to 1. Must be 1 for liveness. Minimum value is 1. + minimum: 1 + type: integer + timeoutSeconds: + description: The timeout for each attempted health check. Default to 5 seconds. Minimum value is 1. + minimum: 1 + type: integer + type: object + logging: + description: Logging configuration for Kafka Bridge. + properties: + loggers: + additionalProperties: + type: string + description: A Map from logger name to logger level. + type: object + type: + description: Logging type, must be either 'inline' or 'external'. + enum: + - inline + - external + type: string + valueFrom: + description: '`ConfigMap` entry where the logging configuration is stored. ' + properties: + configMapKeyRef: + description: Reference to the key in the ConfigMap containing the configuration. + properties: + key: + type: string + name: + type: string + optional: + type: boolean + type: object + type: object + required: + - type + type: object + producer: + description: Kafka producer related configuration. + properties: + config: + description: 'The Kafka producer configuration used for producer instances created by the bridge. Properties with the following prefixes cannot be set: ssl., bootstrap.servers, sasl., security. (with the exception of: ssl.endpoint.identification.algorithm, ssl.cipher.suites, ssl.protocol, ssl.enabled.protocols).' + type: object + x-kubernetes-preserve-unknown-fields: true + enabled: + description: Whether the HTTP producer should be enabled or disabled. The default is enabled (`true`). + type: boolean + type: object + rack: + description: Configuration of the node label which will be used as the client.rack consumer configuration. + properties: + topologyKey: + description: A key that matches labels assigned to the Kubernetes cluster nodes. The value of the label is used to set a broker's `broker.rack` config, and the `client.rack` config for Kafka Connect or MirrorMaker 2. + example: topology.kubernetes.io/zone + type: string + required: + - topologyKey + type: object + readinessProbe: + description: Pod readiness checking. + properties: + failureThreshold: + description: Minimum consecutive failures for the probe to be considered failed after having succeeded. Defaults to 3. Minimum value is 1. + minimum: 1 + type: integer + initialDelaySeconds: + description: The initial delay before first the health is first checked. Default to 15 seconds. Minimum value is 0. + minimum: 0 + type: integer + periodSeconds: + description: How often (in seconds) to perform the probe. Default to 10 seconds. Minimum value is 1. + minimum: 1 + type: integer + successThreshold: + description: Minimum consecutive successes for the probe to be considered successful after having failed. Defaults to 1. Must be 1 for liveness. Minimum value is 1. + minimum: 1 + type: integer + timeoutSeconds: + description: The timeout for each attempted health check. Default to 5 seconds. Minimum value is 1. + minimum: 1 + type: integer + type: object + replicas: + description: The number of pods in the `Deployment`. Defaults to `1`. + minimum: 0 + type: integer + resources: + description: CPU and memory resources to reserve. + properties: + claims: + items: + properties: + name: + type: string + type: object + type: array + limits: + additionalProperties: + anyOf: + - type: integer + - type: string + pattern: ^(\+|-)?(([0-9]+(\.[0-9]*)?)|(\.[0-9]+))(([KMGTPE]i)|[numkMGTPE]|([eE](\+|-)?(([0-9]+(\.[0-9]*)?)|(\.[0-9]+))))?$ + x-kubernetes-int-or-string: true + type: object + requests: + additionalProperties: + anyOf: + - type: integer + - type: string + pattern: ^(\+|-)?(([0-9]+(\.[0-9]*)?)|(\.[0-9]+))(([KMGTPE]i)|[numkMGTPE]|([eE](\+|-)?(([0-9]+(\.[0-9]*)?)|(\.[0-9]+))))?$ + x-kubernetes-int-or-string: true + type: object + type: object + template: + description: Template for Kafka Bridge resources. The template allows users to specify how a `Deployment` and `Pod` is generated. + properties: + apiService: + description: Template for Kafka Bridge API `Service`. + properties: + ipFamilies: + description: Specifies the IP Families used by the service. Available options are `IPv4` and `IPv6`. If unspecified, Kubernetes will choose the default value based on the `ipFamilyPolicy` setting. + items: + enum: + - IPv4 + - IPv6 + type: string + type: array + ipFamilyPolicy: + description: Specifies the IP Family Policy used by the service. Available options are `SingleStack`, `PreferDualStack` and `RequireDualStack`. `SingleStack` is for a single IP family. `PreferDualStack` is for two IP families on dual-stack configured clusters or a single IP family on single-stack clusters. `RequireDualStack` fails unless there are two IP families on dual-stack configured clusters. If unspecified, Kubernetes will choose the default value based on the service type. + enum: + - SingleStack + - PreferDualStack + - RequireDualStack + type: string + metadata: + description: Metadata applied to the resource. + properties: + annotations: + additionalProperties: + type: string + description: Annotations added to the Kubernetes resource. + type: object + labels: + additionalProperties: + type: string + description: Labels added to the Kubernetes resource. + type: object + type: object + type: object + bridgeContainer: + description: Template for the Kafka Bridge container. + properties: + env: + description: Environment variables which should be applied to the container. + items: + oneOf: + - properties: + value: {} + required: + - value + - properties: + valueFrom: {} + required: + - valueFrom + properties: + name: + description: The environment variable key. + type: string + value: + description: The environment variable value. + type: string + valueFrom: + description: Reference to the secret or config map property to which the environment variable is set. + oneOf: + - properties: + secretKeyRef: {} + required: + - secretKeyRef + - properties: + configMapKeyRef: {} + required: + - configMapKeyRef + properties: + configMapKeyRef: + description: Reference to a key in a config map. + properties: + key: + type: string + name: + type: string + optional: + type: boolean + type: object + secretKeyRef: + description: Reference to a key in a secret. + properties: + key: + type: string + name: + type: string + optional: + type: boolean + type: object + type: object + type: object + type: array + securityContext: + description: Security context for the container. + properties: + allowPrivilegeEscalation: + type: boolean + appArmorProfile: + properties: + localhostProfile: + type: string + type: + type: string + type: object + capabilities: + properties: + add: + items: + type: string + type: array + drop: + items: + type: string + type: array + type: object + privileged: + type: boolean + procMount: + type: string + readOnlyRootFilesystem: + type: boolean + runAsGroup: + type: integer + runAsNonRoot: + type: boolean + runAsUser: + type: integer + seLinuxOptions: + properties: + level: + type: string + role: + type: string + type: + type: string + user: + type: string + type: object + seccompProfile: + properties: + localhostProfile: + type: string + type: + type: string + type: object + windowsOptions: + properties: + gmsaCredentialSpec: + type: string + gmsaCredentialSpecName: + type: string + hostProcess: + type: boolean + runAsUserName: + type: string + type: object + type: object + volumeMounts: + description: Additional volume mounts which should be applied to the container. + items: + properties: + mountPath: + type: string + mountPropagation: + type: string + name: + type: string + readOnly: + type: boolean + recursiveReadOnly: + type: string + subPath: + type: string + subPathExpr: + type: string + type: object + type: array + type: object + clusterRoleBinding: + description: Template for the Kafka Bridge ClusterRoleBinding. + properties: + metadata: + description: Metadata applied to the resource. + properties: + annotations: + additionalProperties: + type: string + description: Annotations added to the Kubernetes resource. + type: object + labels: + additionalProperties: + type: string + description: Labels added to the Kubernetes resource. + type: object + type: object + type: object + deployment: + description: Template for Kafka Bridge `Deployment`. + properties: + deploymentStrategy: + description: Pod replacement strategy for deployment configuration changes. Valid values are `RollingUpdate` and `Recreate`. Defaults to `RollingUpdate`. + enum: + - RollingUpdate + - Recreate + type: string + metadata: + description: Metadata applied to the resource. + properties: + annotations: + additionalProperties: + type: string + description: Annotations added to the Kubernetes resource. + type: object + labels: + additionalProperties: + type: string + description: Labels added to the Kubernetes resource. + type: object + type: object + type: object + initContainer: + description: Template for the Kafka Bridge init container. + properties: + env: + description: Environment variables which should be applied to the container. + items: + oneOf: + - properties: + value: {} + required: + - value + - properties: + valueFrom: {} + required: + - valueFrom + properties: + name: + description: The environment variable key. + type: string + value: + description: The environment variable value. + type: string + valueFrom: + description: Reference to the secret or config map property to which the environment variable is set. + oneOf: + - properties: + secretKeyRef: {} + required: + - secretKeyRef + - properties: + configMapKeyRef: {} + required: + - configMapKeyRef + properties: + configMapKeyRef: + description: Reference to a key in a config map. + properties: + key: + type: string + name: + type: string + optional: + type: boolean + type: object + secretKeyRef: + description: Reference to a key in a secret. + properties: + key: + type: string + name: + type: string + optional: + type: boolean + type: object + type: object + type: object + type: array + securityContext: + description: Security context for the container. + properties: + allowPrivilegeEscalation: + type: boolean + appArmorProfile: + properties: + localhostProfile: + type: string + type: + type: string + type: object + capabilities: + properties: + add: + items: + type: string + type: array + drop: + items: + type: string + type: array + type: object + privileged: + type: boolean + procMount: + type: string + readOnlyRootFilesystem: + type: boolean + runAsGroup: + type: integer + runAsNonRoot: + type: boolean + runAsUser: + type: integer + seLinuxOptions: + properties: + level: + type: string + role: + type: string + type: + type: string + user: + type: string + type: object + seccompProfile: + properties: + localhostProfile: + type: string + type: + type: string + type: object + windowsOptions: + properties: + gmsaCredentialSpec: + type: string + gmsaCredentialSpecName: + type: string + hostProcess: + type: boolean + runAsUserName: + type: string + type: object + type: object + volumeMounts: + description: Additional volume mounts which should be applied to the container. + items: + properties: + mountPath: + type: string + mountPropagation: + type: string + name: + type: string + readOnly: + type: boolean + recursiveReadOnly: + type: string + subPath: + type: string + subPathExpr: + type: string + type: object + type: array + type: object + pod: + description: Template for Kafka Bridge `Pods`. + properties: + affinity: + description: The pod's affinity rules. + properties: + nodeAffinity: + properties: + preferredDuringSchedulingIgnoredDuringExecution: + items: + properties: + preference: + properties: + matchExpressions: + items: + properties: + key: + type: string + operator: + type: string + values: + items: + type: string + type: array + type: object + type: array + matchFields: + items: + properties: + key: + type: string + operator: + type: string + values: + items: + type: string + type: array + type: object + type: array + type: object + weight: + type: integer + type: object + type: array + requiredDuringSchedulingIgnoredDuringExecution: + properties: + nodeSelectorTerms: + items: + properties: + matchExpressions: + items: + properties: + key: + type: string + operator: + type: string + values: + items: + type: string + type: array + type: object + type: array + matchFields: + items: + properties: + key: + type: string + operator: + type: string + values: + items: + type: string + type: array + type: object + type: array + type: object + type: array + type: object + type: object + podAffinity: + properties: + preferredDuringSchedulingIgnoredDuringExecution: + items: + properties: + podAffinityTerm: + properties: + labelSelector: + properties: + matchExpressions: + items: + properties: + key: + type: string + operator: + type: string + values: + items: + type: string + type: array + type: object + type: array + matchLabels: + additionalProperties: + type: string + type: object + type: object + matchLabelKeys: + items: + type: string + type: array + mismatchLabelKeys: + items: + type: string + type: array + namespaceSelector: + properties: + matchExpressions: + items: + properties: + key: + type: string + operator: + type: string + values: + items: + type: string + type: array + type: object + type: array + matchLabels: + additionalProperties: + type: string + type: object + type: object + namespaces: + items: + type: string + type: array + topologyKey: + type: string + type: object + weight: + type: integer + type: object + type: array + requiredDuringSchedulingIgnoredDuringExecution: + items: + properties: + labelSelector: + properties: + matchExpressions: + items: + properties: + key: + type: string + operator: + type: string + values: + items: + type: string + type: array + type: object + type: array + matchLabels: + additionalProperties: + type: string + type: object + type: object + matchLabelKeys: + items: + type: string + type: array + mismatchLabelKeys: + items: + type: string + type: array + namespaceSelector: + properties: + matchExpressions: + items: + properties: + key: + type: string + operator: + type: string + values: + items: + type: string + type: array + type: object + type: array + matchLabels: + additionalProperties: + type: string + type: object + type: object + namespaces: + items: + type: string + type: array + topologyKey: + type: string + type: object + type: array + type: object + podAntiAffinity: + properties: + preferredDuringSchedulingIgnoredDuringExecution: + items: + properties: + podAffinityTerm: + properties: + labelSelector: + properties: + matchExpressions: + items: + properties: + key: + type: string + operator: + type: string + values: + items: + type: string + type: array + type: object + type: array + matchLabels: + additionalProperties: + type: string + type: object + type: object + matchLabelKeys: + items: + type: string + type: array + mismatchLabelKeys: + items: + type: string + type: array + namespaceSelector: + properties: + matchExpressions: + items: + properties: + key: + type: string + operator: + type: string + values: + items: + type: string + type: array + type: object + type: array + matchLabels: + additionalProperties: + type: string + type: object + type: object + namespaces: + items: + type: string + type: array + topologyKey: + type: string + type: object + weight: + type: integer + type: object + type: array + requiredDuringSchedulingIgnoredDuringExecution: + items: + properties: + labelSelector: + properties: + matchExpressions: + items: + properties: + key: + type: string + operator: + type: string + values: + items: + type: string + type: array + type: object + type: array + matchLabels: + additionalProperties: + type: string + type: object + type: object + matchLabelKeys: + items: + type: string + type: array + mismatchLabelKeys: + items: + type: string + type: array + namespaceSelector: + properties: + matchExpressions: + items: + properties: + key: + type: string + operator: + type: string + values: + items: + type: string + type: array + type: object + type: array + matchLabels: + additionalProperties: + type: string + type: object + type: object + namespaces: + items: + type: string + type: array + topologyKey: + type: string + type: object + type: array + type: object + type: object + enableServiceLinks: + description: Indicates whether information about services should be injected into Pod's environment variables. + type: boolean + hostAliases: + description: The pod's HostAliases. HostAliases is an optional list of hosts and IPs that will be injected into the Pod's hosts file if specified. + items: + properties: + hostnames: + items: + type: string + type: array + ip: + type: string + type: object + type: array + imagePullSecrets: + description: List of references to secrets in the same namespace to use for pulling any of the images used by this Pod. When the `STRIMZI_IMAGE_PULL_SECRETS` environment variable in Cluster Operator and the `imagePullSecrets` option are specified, only the `imagePullSecrets` variable is used and the `STRIMZI_IMAGE_PULL_SECRETS` variable is ignored. + items: + properties: + name: + type: string + type: object + type: array + metadata: + description: Metadata applied to the resource. + properties: + annotations: + additionalProperties: + type: string + description: Annotations added to the Kubernetes resource. + type: object + labels: + additionalProperties: + type: string + description: Labels added to the Kubernetes resource. + type: object + type: object + priorityClassName: + description: 'The name of the priority class used to assign priority to the pods. ' + type: string + schedulerName: + description: The name of the scheduler used to dispatch this `Pod`. If not specified, the default scheduler will be used. + type: string + securityContext: + description: Configures pod-level security attributes and common container settings. + properties: + appArmorProfile: + properties: + localhostProfile: + type: string + type: + type: string + type: object + fsGroup: + type: integer + fsGroupChangePolicy: + type: string + runAsGroup: + type: integer + runAsNonRoot: + type: boolean + runAsUser: + type: integer + seLinuxOptions: + properties: + level: + type: string + role: + type: string + type: + type: string + user: + type: string + type: object + seccompProfile: + properties: + localhostProfile: + type: string + type: + type: string + type: object + supplementalGroups: + items: + type: integer + type: array + sysctls: + items: + properties: + name: + type: string + value: + type: string + type: object + type: array + windowsOptions: + properties: + gmsaCredentialSpec: + type: string + gmsaCredentialSpecName: + type: string + hostProcess: + type: boolean + runAsUserName: + type: string + type: object + type: object + terminationGracePeriodSeconds: + description: The grace period is the duration in seconds after the processes running in the pod are sent a termination signal, and the time when the processes are forcibly halted with a kill signal. Set this value to longer than the expected cleanup time for your process. Value must be a non-negative integer. A zero value indicates delete immediately. You might need to increase the grace period for very large Kafka clusters, so that the Kafka brokers have enough time to transfer their work to another broker before they are terminated. Defaults to 30 seconds. + minimum: 0 + type: integer + tmpDirSizeLimit: + description: Defines the total amount of pod memory allocated for the temporary `EmptyDir` volume `/tmp`. Specify the allocation in memory units, for example, `100Mi` for 100 mebibytes. Default value is `5Mi`. The `/tmp` volume is backed by pod memory, not disk storage, so avoid setting a high value as it consumes pod memory resources. + pattern: ^([0-9.]+)([eEinumkKMGTP]*[-+]?[0-9]*)$ + type: string + tolerations: + description: The pod's tolerations. + items: + properties: + effect: + type: string + key: + type: string + operator: + type: string + tolerationSeconds: + type: integer + value: + type: string + type: object + type: array + topologySpreadConstraints: + description: The pod's topology spread constraints. + items: + properties: + labelSelector: + properties: + matchExpressions: + items: + properties: + key: + type: string + operator: + type: string + values: + items: + type: string + type: array + type: object + type: array + matchLabels: + additionalProperties: + type: string + type: object + type: object + matchLabelKeys: + items: + type: string + type: array + maxSkew: + type: integer + minDomains: + type: integer + nodeAffinityPolicy: + type: string + nodeTaintsPolicy: + type: string + topologyKey: + type: string + whenUnsatisfiable: + type: string + type: object + type: array + volumes: + description: Additional volumes that can be mounted to the pod. + items: + oneOf: + - properties: + configMap: {} + csi: {} + emptyDir: {} + persistentVolumeClaim: {} + secret: {} + properties: + configMap: + description: ConfigMap to use to populate the volume. + properties: + defaultMode: + type: integer + items: + items: + properties: + key: + type: string + mode: + type: integer + path: + type: string + type: object + type: array + name: + type: string + optional: + type: boolean + type: object + csi: + description: CSIVolumeSource object to use to populate the volume. + properties: + driver: + type: string + fsType: + type: string + nodePublishSecretRef: + properties: + name: + type: string + type: object + readOnly: + type: boolean + volumeAttributes: + additionalProperties: + type: string + type: object + type: object + emptyDir: + description: EmptyDir to use to populate the volume. + properties: + medium: + type: string + sizeLimit: + properties: + amount: + type: string + format: + type: string + type: object + type: object + name: + description: Name to use for the volume. Required. + type: string + persistentVolumeClaim: + description: PersistentVolumeClaim object to use to populate the volume. + properties: + claimName: + type: string + readOnly: + type: boolean + type: object + secret: + description: Secret to use populate the volume. + properties: + defaultMode: + type: integer + items: + items: + properties: + key: + type: string + mode: + type: integer + path: + type: string + type: object + type: array + optional: + type: boolean + secretName: + type: string + type: object + type: object + type: array + type: object + podDisruptionBudget: + description: Template for Kafka Bridge `PodDisruptionBudget`. + properties: + maxUnavailable: + description: Maximum number of unavailable pods to allow automatic Pod eviction. A Pod eviction is allowed when the `maxUnavailable` number of pods or fewer are unavailable after the eviction. Setting this value to 0 prevents all voluntary evictions, so the pods must be evicted manually. Defaults to 1. + minimum: 0 + type: integer + metadata: + description: Metadata to apply to the `PodDisruptionBudgetTemplate` resource. + properties: + annotations: + additionalProperties: + type: string + description: Annotations added to the Kubernetes resource. + type: object + labels: + additionalProperties: + type: string + description: Labels added to the Kubernetes resource. + type: object + type: object + type: object + serviceAccount: + description: Template for the Kafka Bridge service account. + properties: + metadata: + description: Metadata applied to the resource. + properties: + annotations: + additionalProperties: + type: string + description: Annotations added to the Kubernetes resource. + type: object + labels: + additionalProperties: + type: string + description: Labels added to the Kubernetes resource. + type: object + type: object + type: object + type: object + tls: + description: TLS configuration for connecting Kafka Bridge to the cluster. + properties: + trustedCertificates: + description: Trusted certificates for TLS connection. + items: + oneOf: + - properties: + certificate: {} + required: + - certificate + - properties: + pattern: {} + required: + - pattern + properties: + certificate: + description: The name of the file certificate in the secret. + type: string + pattern: + description: Pattern for the certificate files in the secret. Use the link:https://en.wikipedia.org/wiki/Glob_(programming)[_glob syntax_] for the pattern. All files in the secret that match the pattern are used. + type: string + secretName: + description: The name of the Secret containing the certificate. + type: string + required: + - secretName + type: object + type: array + type: object + tracing: + description: The configuration of tracing in Kafka Bridge. + properties: + type: + description: Type of the tracing used. Currently the only supported type is `opentelemetry` for OpenTelemetry tracing. As of Strimzi 0.37.0, `jaeger` type is not supported anymore and this option is ignored. + enum: + - jaeger + - opentelemetry + type: string + required: + - type + type: object + required: + - bootstrapServers + type: object + status: + description: The status of the Kafka Bridge. + properties: + conditions: + description: List of status conditions. + items: + properties: + lastTransitionTime: + description: Last time the condition of a type changed from one status to another. The required format is 'yyyy-MM-ddTHH:mm:ssZ', in the UTC time zone. + type: string + message: + description: Human-readable message indicating details about the condition's last transition. + type: string + reason: + description: The reason for the condition's last transition (a single word in CamelCase). + type: string + status: + description: The status of the condition, either True, False or Unknown. + type: string + type: + description: The unique identifier of a condition, used to distinguish between other conditions in the resource. + type: string + type: object + type: array + labelSelector: + description: Label selector for pods providing this resource. + type: string + observedGeneration: + description: The generation of the CRD that was last reconciled by the operator. + type: integer + replicas: + description: The current number of pods being used to provide this resource. + type: integer + url: + description: The URL at which external client applications can access the Kafka Bridge. + type: string + type: object + type: object + served: true + storage: true + subresources: + scale: + labelSelectorPath: .status.labelSelector + specReplicasPath: .spec.replicas + statusReplicasPath: .status.replicas + status: {} +--- +apiVersion: rbac.authorization.k8s.io/v1 +kind: RoleBinding +metadata: + labels: + app: strimzi + name: strimzi-cluster-operator-leader-election + namespace: kafka +roleRef: + apiGroup: rbac.authorization.k8s.io + kind: ClusterRole + name: strimzi-cluster-operator-leader-election +subjects: + - kind: ServiceAccount + name: strimzi-cluster-operator + namespace: kafka +--- +apiVersion: rbac.authorization.k8s.io/v1 +kind: ClusterRole +metadata: + labels: + app: strimzi + name: strimzi-entity-operator +rules: + - apiGroups: + - kafka.strimzi.io + resources: + - kafkatopics + verbs: + - get + - list + - watch + - create + - patch + - update + - delete + - apiGroups: + - kafka.strimzi.io + resources: + - kafkausers + verbs: + - get + - list + - watch + - create + - patch + - update + - apiGroups: + - kafka.strimzi.io + resources: + - kafkatopics/status + - kafkausers/status + verbs: + - get + - patch + - update + - apiGroups: + - "" + resources: + - secrets + verbs: + - get + - list + - watch + - create + - delete + - patch + - update +--- +apiVersion: rbac.authorization.k8s.io/v1 +kind: ClusterRole +metadata: + labels: + app: strimzi + name: strimzi-cluster-operator-namespaced +rules: + - apiGroups: + - rbac.authorization.k8s.io + resources: + - rolebindings + verbs: + - get + - list + - watch + - create + - delete + - patch + - update + - apiGroups: + - rbac.authorization.k8s.io + resources: + - roles + verbs: + - get + - list + - watch + - create + - delete + - patch + - update + - apiGroups: + - "" + resources: + - pods + - serviceaccounts + - configmaps + - services + - endpoints + - secrets + - persistentvolumeclaims + verbs: + - get + - list + - watch + - create + - delete + - patch + - update + - apiGroups: + - apps + resources: + - deployments + - statefulsets + - replicasets + verbs: + - get + - list + - watch + - create + - delete + - patch + - update + - apiGroups: + - apps + resources: + - deployments/scale + verbs: + - get + - patch + - update + - apiGroups: + - events.k8s.io + resources: + - events + verbs: + - create + - apiGroups: + - build.openshift.io + resources: + - buildconfigs + - buildconfigs/instantiate + - builds + verbs: + - get + - list + - watch + - create + - delete + - patch + - update + - apiGroups: + - networking.k8s.io + resources: + - networkpolicies + - ingresses + verbs: + - get + - list + - watch + - create + - delete + - patch + - update + - apiGroups: + - route.openshift.io + resources: + - routes + - routes/custom-host + verbs: + - get + - list + - watch + - create + - delete + - patch + - update + - apiGroups: + - image.openshift.io + resources: + - imagestreams + verbs: + - get + - apiGroups: + - policy + resources: + - poddisruptionbudgets + verbs: + - get + - list + - watch + - create + - delete + - patch + - update +--- +apiVersion: rbac.authorization.k8s.io/v1 +kind: ClusterRoleBinding +metadata: + labels: + app: strimzi + name: strimzi-cluster-operator-kafka-client-delegation +roleRef: + apiGroup: rbac.authorization.k8s.io + kind: ClusterRole + name: strimzi-kafka-client +subjects: + - kind: ServiceAccount + name: strimzi-cluster-operator + namespace: kafka +--- +apiVersion: rbac.authorization.k8s.io/v1 +kind: RoleBinding +metadata: + labels: + app: strimzi + name: strimzi-cluster-operator-watched + namespace: kafka +roleRef: + apiGroup: rbac.authorization.k8s.io + kind: ClusterRole + name: strimzi-cluster-operator-watched +subjects: + - kind: ServiceAccount + name: strimzi-cluster-operator + namespace: kafka +--- +apiVersion: v1 +kind: ServiceAccount +metadata: + labels: + app: strimzi + name: strimzi-cluster-operator + namespace: kafka +--- +apiVersion: rbac.authorization.k8s.io/v1 +kind: ClusterRole +metadata: + labels: + app: strimzi + name: strimzi-cluster-operator-global +rules: + - apiGroups: + - rbac.authorization.k8s.io + resources: + - clusterrolebindings + verbs: + - get + - list + - watch + - create + - delete + - patch + - update + - apiGroups: + - storage.k8s.io + resources: + - storageclasses + verbs: + - get + - apiGroups: + - "" + resources: + - nodes + verbs: + - list +--- +apiVersion: apiextensions.k8s.io/v1 +kind: CustomResourceDefinition +metadata: + labels: + app: strimzi + strimzi.io/crd-install: "true" + name: kafkaconnectors.kafka.strimzi.io +spec: + conversion: + strategy: None + group: kafka.strimzi.io + names: + categories: + - strimzi + kind: KafkaConnector + listKind: KafkaConnectorList + plural: kafkaconnectors + shortNames: + - kctr + singular: kafkaconnector + scope: Namespaced + versions: + - additionalPrinterColumns: + - description: The name of the Kafka Connect cluster this connector belongs to + jsonPath: .metadata.labels.strimzi\.io/cluster + name: Cluster + type: string + - description: The class used by this connector + jsonPath: .spec.class + name: Connector class + type: string + - description: Maximum number of tasks + jsonPath: .spec.tasksMax + name: Max Tasks + type: integer + - description: The state of the custom resource + jsonPath: .status.conditions[?(@.type=="Ready")].status + name: Ready + type: string + name: v1beta2 + schema: + openAPIV3Schema: + properties: + apiVersion: + description: 'APIVersion defines the versioned schema of this representation of an object. Servers should convert recognized schemas to the latest internal value, and may reject unrecognized values. More info: https://git.k8s.io/community/contributors/devel/sig-architecture/api-conventions.md#resources' + type: string + kind: + description: 'Kind is a string value representing the REST resource this object represents. Servers may infer this from the endpoint the client submits requests to. Cannot be updated. In CamelCase. More info: https://git.k8s.io/community/contributors/devel/sig-architecture/api-conventions.md#types-kinds' + type: string + metadata: + type: object + spec: + description: The specification of the Kafka Connector. + properties: + alterOffsets: + description: Configuration for altering offsets. + properties: + fromConfigMap: + description: Reference to the ConfigMap where the new offsets are stored. + properties: + name: + type: string + type: object + required: + - fromConfigMap + type: object + autoRestart: + description: Automatic restart of connector and tasks configuration. + properties: + enabled: + description: Whether automatic restart for failed connectors and tasks should be enabled or disabled. + type: boolean + maxRestarts: + description: The maximum number of connector restarts that the operator will try. If the connector remains in a failed state after reaching this limit, it must be restarted manually by the user. Defaults to an unlimited number of restarts. + type: integer + type: object + class: + description: The Class for the Kafka Connector. + type: string + config: + description: 'The Kafka Connector configuration. The following properties cannot be set: name, connector.class, tasks.max.' + type: object + x-kubernetes-preserve-unknown-fields: true + listOffsets: + description: Configuration for listing offsets. + properties: + toConfigMap: + description: Reference to the ConfigMap where the list of offsets will be written to. + properties: + name: + type: string + type: object + required: + - toConfigMap + type: object + pause: + description: Whether the connector should be paused. Defaults to false. + type: boolean + state: + description: The state the connector should be in. Defaults to running. + enum: + - paused + - stopped + - running + type: string + tasksMax: + description: The maximum number of tasks for the Kafka Connector. + minimum: 1 + type: integer + type: object + status: + description: The status of the Kafka Connector. + properties: + autoRestart: + description: The auto restart status. + properties: + connectorName: + description: The name of the connector being restarted. + type: string + count: + description: The number of times the connector or task is restarted. + type: integer + lastRestartTimestamp: + description: The last time the automatic restart was attempted. The required format is 'yyyy-MM-ddTHH:mm:ssZ' in the UTC time zone. + type: string + type: object + conditions: + description: List of status conditions. + items: + properties: + lastTransitionTime: + description: Last time the condition of a type changed from one status to another. The required format is 'yyyy-MM-ddTHH:mm:ssZ', in the UTC time zone. + type: string + message: + description: Human-readable message indicating details about the condition's last transition. + type: string + reason: + description: The reason for the condition's last transition (a single word in CamelCase). + type: string + status: + description: The status of the condition, either True, False or Unknown. + type: string + type: + description: The unique identifier of a condition, used to distinguish between other conditions in the resource. + type: string + type: object + type: array + connectorStatus: + description: The connector status, as reported by the Kafka Connect REST API. + type: object + x-kubernetes-preserve-unknown-fields: true + observedGeneration: + description: The generation of the CRD that was last reconciled by the operator. + type: integer + tasksMax: + description: The maximum number of tasks for the Kafka Connector. + type: integer + topics: + description: The list of topics used by the Kafka Connector. + items: + type: string + type: array + type: object + type: object + served: true + storage: true + subresources: + scale: + specReplicasPath: .spec.tasksMax + statusReplicasPath: .status.tasksMax + status: {} +--- +apiVersion: rbac.authorization.k8s.io/v1 +kind: RoleBinding +metadata: + labels: + app: strimzi + name: strimzi-cluster-operator + namespace: kafka +roleRef: + apiGroup: rbac.authorization.k8s.io + kind: ClusterRole + name: strimzi-cluster-operator-namespaced +subjects: + - kind: ServiceAccount + name: strimzi-cluster-operator + namespace: kafka +--- +apiVersion: rbac.authorization.k8s.io/v1 +kind: ClusterRoleBinding +metadata: + labels: + app: strimzi + name: strimzi-cluster-operator +roleRef: + apiGroup: rbac.authorization.k8s.io + kind: ClusterRole + name: strimzi-cluster-operator-global +subjects: + - kind: ServiceAccount + name: strimzi-cluster-operator + namespace: kafka +--- +apiVersion: rbac.authorization.k8s.io/v1 +kind: RoleBinding +metadata: + labels: + app: strimzi + name: strimzi-cluster-operator-entity-operator-delegation + namespace: kafka +roleRef: + apiGroup: rbac.authorization.k8s.io + kind: ClusterRole + name: strimzi-entity-operator +subjects: + - kind: ServiceAccount + name: strimzi-cluster-operator + namespace: kafka +--- +apiVersion: rbac.authorization.k8s.io/v1 +kind: ClusterRole +metadata: + labels: + app: strimzi + name: strimzi-kafka-client +rules: + - apiGroups: + - "" + resources: + - nodes + verbs: + - get +--- +apiVersion: rbac.authorization.k8s.io/v1 +kind: ClusterRole +metadata: + labels: + app: strimzi + name: strimzi-cluster-operator-leader-election +rules: + - apiGroups: + - coordination.k8s.io + resources: + - leases + verbs: + - create + - apiGroups: + - coordination.k8s.io + resourceNames: + - strimzi-cluster-operator + resources: + - leases + verbs: + - get + - list + - watch + - delete + - patch + - update +--- +apiVersion: rbac.authorization.k8s.io/v1 +kind: ClusterRole +metadata: + labels: + app: strimzi + name: strimzi-kafka-broker +rules: + - apiGroups: + - "" + resources: + - nodes + verbs: + - get +--- +apiVersion: apps/v1 +kind: Deployment +metadata: + labels: + app: strimzi + name: strimzi-cluster-operator + namespace: kafka +spec: + replicas: 1 + selector: + matchLabels: + name: strimzi-cluster-operator + strimzi.io/kind: cluster-operator + template: + metadata: + labels: + name: strimzi-cluster-operator + strimzi.io/kind: cluster-operator + spec: + containers: + - args: + - /opt/strimzi/bin/cluster_operator_run.sh + env: + - name: STRIMZI_NAMESPACE + valueFrom: + fieldRef: + fieldPath: metadata.namespace + - name: STRIMZI_FULL_RECONCILIATION_INTERVAL_MS + value: "120000" + - name: STRIMZI_OPERATION_TIMEOUT_MS + value: "300000" + - name: STRIMZI_DEFAULT_KAFKA_EXPORTER_IMAGE + value: quay.io/strimzi/kafka:0.45.0-kafka-3.9.0 + - name: STRIMZI_DEFAULT_CRUISE_CONTROL_IMAGE + value: quay.io/strimzi/kafka:0.45.0-kafka-3.9.0 + - name: STRIMZI_KAFKA_IMAGES + value: | + 3.8.0=quay.io/strimzi/kafka:0.45.0-kafka-3.8.0 + 3.8.1=quay.io/strimzi/kafka:0.45.0-kafka-3.8.1 + 3.9.0=quay.io/strimzi/kafka:0.45.0-kafka-3.9.0 + - name: STRIMZI_KAFKA_CONNECT_IMAGES + value: | + 3.8.0=quay.io/strimzi/kafka:0.45.0-kafka-3.8.0 + 3.8.1=quay.io/strimzi/kafka:0.45.0-kafka-3.8.1 + 3.9.0=quay.io/strimzi/kafka:0.45.0-kafka-3.9.0 + - name: STRIMZI_KAFKA_MIRROR_MAKER_IMAGES + value: | + 3.8.0=quay.io/strimzi/kafka:0.45.0-kafka-3.8.0 + 3.8.1=quay.io/strimzi/kafka:0.45.0-kafka-3.8.1 + 3.9.0=quay.io/strimzi/kafka:0.45.0-kafka-3.9.0 + - name: STRIMZI_KAFKA_MIRROR_MAKER_2_IMAGES + value: | + 3.8.0=quay.io/strimzi/kafka:0.45.0-kafka-3.8.0 + 3.8.1=quay.io/strimzi/kafka:0.45.0-kafka-3.8.1 + 3.9.0=quay.io/strimzi/kafka:0.45.0-kafka-3.9.0 + - name: STRIMZI_DEFAULT_TOPIC_OPERATOR_IMAGE + value: quay.io/strimzi/operator:0.45.0 + - name: STRIMZI_DEFAULT_USER_OPERATOR_IMAGE + value: quay.io/strimzi/operator:0.45.0 + - name: STRIMZI_DEFAULT_KAFKA_INIT_IMAGE + value: quay.io/strimzi/operator:0.45.0 + - name: STRIMZI_DEFAULT_KAFKA_BRIDGE_IMAGE + value: quay.io/strimzi/kafka-bridge:0.31.1 + - name: STRIMZI_DEFAULT_KANIKO_EXECUTOR_IMAGE + value: quay.io/strimzi/kaniko-executor:0.45.0 + - name: STRIMZI_DEFAULT_MAVEN_BUILDER + value: quay.io/strimzi/maven-builder:0.45.0 + - name: STRIMZI_OPERATOR_NAMESPACE + valueFrom: + fieldRef: + fieldPath: metadata.namespace + - name: STRIMZI_FEATURE_GATES + value: "" + - name: STRIMZI_LEADER_ELECTION_ENABLED + value: "true" + - name: STRIMZI_LEADER_ELECTION_LEASE_NAME + value: strimzi-cluster-operator + - name: STRIMZI_LEADER_ELECTION_LEASE_NAMESPACE + valueFrom: + fieldRef: + fieldPath: metadata.namespace + - name: STRIMZI_LEADER_ELECTION_IDENTITY + valueFrom: + fieldRef: + fieldPath: metadata.name + image: quay.io/strimzi/operator:0.45.0 + livenessProbe: + httpGet: + path: /healthy + port: http + initialDelaySeconds: 10 + periodSeconds: 30 + name: strimzi-cluster-operator + ports: + - containerPort: 8080 + name: http + readinessProbe: + httpGet: + path: /ready + port: http + initialDelaySeconds: 10 + periodSeconds: 30 + resources: + limits: + cpu: 1000m + memory: 384Mi + requests: + cpu: 200m + memory: 384Mi + volumeMounts: + - mountPath: /tmp + name: strimzi-tmp + - mountPath: /opt/strimzi/custom-config/ + name: co-config-volume + serviceAccountName: strimzi-cluster-operator + volumes: + - emptyDir: + medium: Memory + sizeLimit: 1Mi + name: strimzi-tmp + - configMap: + name: strimzi-cluster-operator + name: co-config-volume +--- +apiVersion: rbac.authorization.k8s.io/v1 +kind: ClusterRole +metadata: + labels: + app: strimzi + name: strimzi-cluster-operator-watched +rules: + - apiGroups: + - "" + resources: + - pods + verbs: + - watch + - list + - apiGroups: + - kafka.strimzi.io + resources: + - kafkas + - kafkanodepools + - kafkaconnects + - kafkaconnectors + - kafkamirrormakers + - kafkabridges + - kafkamirrormaker2s + - kafkarebalances + verbs: + - get + - list + - watch + - create + - patch + - update + - apiGroups: + - kafka.strimzi.io + resources: + - kafkas/status + - kafkanodepools/status + - kafkaconnects/status + - kafkaconnectors/status + - kafkamirrormakers/status + - kafkabridges/status + - kafkamirrormaker2s/status + - kafkarebalances/status + verbs: + - get + - patch + - update + - apiGroups: + - core.strimzi.io + resources: + - strimzipodsets + verbs: + - get + - list + - watch + - create + - delete + - patch + - update + - apiGroups: + - core.strimzi.io + resources: + - strimzipodsets/status + verbs: + - get + - patch + - update + - apiGroups: + - kafka.strimzi.io + resources: + - kafkarebalances + verbs: + - delete +--- +apiVersion: v1 +data: + log4j2.properties: | + name = COConfig + monitorInterval = 30 + + appender.console.type = Console + appender.console.name = STDOUT + appender.console.layout.type = PatternLayout + appender.console.layout.pattern = %d{yyyy-MM-dd HH:mm:ss} %-5p %c{1}:%L - %m%n + + rootLogger.level = ${env:STRIMZI_LOG_LEVEL:-INFO} + rootLogger.appenderRefs = stdout + rootLogger.appenderRef.console.ref = STDOUT + + # Kafka AdminClient logging is a bit noisy at INFO level + logger.kafka.name = org.apache.kafka + logger.kafka.level = WARN + + # Zookeeper is very verbose even on INFO level -> We set it to WARN by default + logger.zookeepertrustmanager.name = org.apache.zookeeper + logger.zookeepertrustmanager.level = WARN + + # Keeps separate level for Netty logging -> to not be changed by the root logger + logger.netty.name = io.netty + logger.netty.level = INFO +kind: ConfigMap +metadata: + labels: + app: strimzi + name: strimzi-cluster-operator + namespace: kafka +--- +apiVersion: apiextensions.k8s.io/v1 +kind: CustomResourceDefinition +metadata: + labels: + app: strimzi + strimzi.io/crd-install: "true" + name: kafkaconnects.kafka.strimzi.io +spec: + conversion: + strategy: None + group: kafka.strimzi.io + names: + categories: + - strimzi + kind: KafkaConnect + listKind: KafkaConnectList + plural: kafkaconnects + shortNames: + - kc + singular: kafkaconnect + scope: Namespaced + versions: + - additionalPrinterColumns: + - description: The desired number of Kafka Connect replicas + jsonPath: .spec.replicas + name: Desired replicas + type: integer + - description: The state of the custom resource + jsonPath: .status.conditions[?(@.type=="Ready")].status + name: Ready + type: string + name: v1beta2 + schema: + openAPIV3Schema: + properties: + apiVersion: + description: 'APIVersion defines the versioned schema of this representation of an object. Servers should convert recognized schemas to the latest internal value, and may reject unrecognized values. More info: https://git.k8s.io/community/contributors/devel/sig-architecture/api-conventions.md#resources' + type: string + kind: + description: 'Kind is a string value representing the REST resource this object represents. Servers may infer this from the endpoint the client submits requests to. Cannot be updated. In CamelCase. More info: https://git.k8s.io/community/contributors/devel/sig-architecture/api-conventions.md#types-kinds' + type: string + metadata: + type: object + spec: + description: The specification of the Kafka Connect cluster. + properties: + authentication: + description: Authentication configuration for Kafka Connect. + properties: + accessToken: + description: Link to Kubernetes Secret containing the access token which was obtained from the authorization server. + properties: + key: + description: The key under which the secret value is stored in the Kubernetes Secret. + type: string + secretName: + description: The name of the Kubernetes Secret containing the secret value. + type: string + required: + - key + - secretName + type: object + accessTokenIsJwt: + description: Configure whether access token should be treated as JWT. This should be set to `false` if the authorization server returns opaque tokens. Defaults to `true`. + type: boolean + accessTokenLocation: + description: Path to the token file containing an access token to be used for authentication. + type: string + audience: + description: OAuth audience to use when authenticating against the authorization server. Some authorization servers require the audience to be explicitly set. The possible values depend on how the authorization server is configured. By default, `audience` is not specified when performing the token endpoint request. + type: string + certificateAndKey: + description: Reference to the `Secret` which holds the certificate and private key pair. + properties: + certificate: + description: The name of the file certificate in the Secret. + type: string + key: + description: The name of the private key in the Secret. + type: string + secretName: + description: The name of the Secret containing the certificate. + type: string + required: + - secretName + - certificate + - key + type: object + clientAssertion: + description: Link to Kubernetes secret containing the client assertion which was manually configured for the client. + properties: + key: + description: The key under which the secret value is stored in the Kubernetes Secret. + type: string + secretName: + description: The name of the Kubernetes Secret containing the secret value. + type: string + required: + - key + - secretName + type: object + clientAssertionLocation: + description: Path to the file containing the client assertion to be used for authentication. + type: string + clientAssertionType: + description: The client assertion type. If not set, and either `clientAssertion` or `clientAssertionLocation` is configured, this value defaults to `urn:ietf:params:oauth:client-assertion-type:jwt-bearer`. + type: string + clientId: + description: OAuth Client ID which the Kafka client can use to authenticate against the OAuth server and use the token endpoint URI. + type: string + clientSecret: + description: Link to Kubernetes Secret containing the OAuth client secret which the Kafka client can use to authenticate against the OAuth server and use the token endpoint URI. + properties: + key: + description: The key under which the secret value is stored in the Kubernetes Secret. + type: string + secretName: + description: The name of the Kubernetes Secret containing the secret value. + type: string + required: + - key + - secretName + type: object + connectTimeoutSeconds: + description: The connect timeout in seconds when connecting to authorization server. If not set, the effective connect timeout is 60 seconds. + type: integer + disableTlsHostnameVerification: + description: Enable or disable TLS hostname verification. Default value is `false`. + type: boolean + enableMetrics: + description: Enable or disable OAuth metrics. Default value is `false`. + type: boolean + httpRetries: + description: The maximum number of retries to attempt if an initial HTTP request fails. If not set, the default is to not attempt any retries. + type: integer + httpRetryPauseMs: + description: The pause to take before retrying a failed HTTP request. If not set, the default is to not pause at all but to immediately repeat a request. + type: integer + includeAcceptHeader: + description: Whether the Accept header should be set in requests to the authorization servers. The default value is `true`. + type: boolean + maxTokenExpirySeconds: + description: Set or limit time-to-live of the access tokens to the specified number of seconds. This should be set if the authorization server returns opaque tokens. + type: integer + passwordSecret: + description: Reference to the `Secret` which holds the password. + properties: + password: + description: The name of the key in the Secret under which the password is stored. + type: string + secretName: + description: The name of the Secret containing the password. + type: string + required: + - secretName + - password + type: object + readTimeoutSeconds: + description: The read timeout in seconds when connecting to authorization server. If not set, the effective read timeout is 60 seconds. + type: integer + refreshToken: + description: Link to Kubernetes Secret containing the refresh token which can be used to obtain access token from the authorization server. + properties: + key: + description: The key under which the secret value is stored in the Kubernetes Secret. + type: string + secretName: + description: The name of the Kubernetes Secret containing the secret value. + type: string + required: + - key + - secretName + type: object + saslExtensions: + additionalProperties: + type: string + description: SASL extensions parameters. + type: object + scope: + description: OAuth scope to use when authenticating against the authorization server. Some authorization servers require this to be set. The possible values depend on how authorization server is configured. By default `scope` is not specified when doing the token endpoint request. + type: string + tlsTrustedCertificates: + description: Trusted certificates for TLS connection to the OAuth server. + items: + oneOf: + - properties: + certificate: {} + required: + - certificate + - properties: + pattern: {} + required: + - pattern + properties: + certificate: + description: The name of the file certificate in the secret. + type: string + pattern: + description: Pattern for the certificate files in the secret. Use the link:https://en.wikipedia.org/wiki/Glob_(programming)[_glob syntax_] for the pattern. All files in the secret that match the pattern are used. + type: string + secretName: + description: The name of the Secret containing the certificate. + type: string + required: + - secretName + type: object + type: array + tokenEndpointUri: + description: Authorization server token endpoint URI. + type: string + type: + description: Authentication type. Currently the supported types are `tls`, `scram-sha-256`, `scram-sha-512`, `plain`, and 'oauth'. `scram-sha-256` and `scram-sha-512` types use SASL SCRAM-SHA-256 and SASL SCRAM-SHA-512 Authentication, respectively. `plain` type uses SASL PLAIN Authentication. `oauth` type uses SASL OAUTHBEARER Authentication. The `tls` type uses TLS Client Authentication. The `tls` type is supported only over TLS connections. + enum: + - tls + - scram-sha-256 + - scram-sha-512 + - plain + - oauth + type: string + username: + description: Username used for the authentication. + type: string + required: + - type + type: object + bootstrapServers: + description: Bootstrap servers to connect to. This should be given as a comma separated list of __:__ pairs. + type: string + build: + description: Configures how the Connect container image should be built. Optional. + properties: + output: + description: Configures where should the newly built image be stored. Required. + properties: + additionalKanikoOptions: + description: 'Configures additional options which will be passed to the Kaniko executor when building the new Connect image. Allowed options are: --customPlatform, --custom-platform, --insecure, --insecure-pull, --insecure-registry, --log-format, --log-timestamp, --registry-mirror, --reproducible, --single-snapshot, --skip-tls-verify, --skip-tls-verify-pull, --skip-tls-verify-registry, --verbosity, --snapshotMode, --use-new-run, --registry-certificate, --registry-client-cert. These options will be used only on Kubernetes where the Kaniko executor is used. They will be ignored on OpenShift. The options are described in the link:https://github.com/GoogleContainerTools/kaniko[Kaniko GitHub repository^]. Changing this field does not trigger new build of the Kafka Connect image.' + items: + type: string + type: array + image: + description: The name of the image which will be built. Required. + type: string + pushSecret: + description: Container Registry Secret with the credentials for pushing the newly built image. + type: string + type: + description: Output type. Must be either `docker` for pushing the newly build image to Docker compatible registry or `imagestream` for pushing the image to OpenShift ImageStream. Required. + enum: + - docker + - imagestream + type: string + required: + - image + - type + type: object + plugins: + description: List of connector plugins which should be added to the Kafka Connect. Required. + items: + properties: + artifacts: + description: List of artifacts which belong to this connector plugin. Required. + items: + properties: + artifact: + description: Maven artifact id. Applicable to the `maven` artifact type only. + type: string + fileName: + description: Name under which the artifact will be stored. + type: string + group: + description: Maven group id. Applicable to the `maven` artifact type only. + type: string + insecure: + description: By default, connections using TLS are verified to check they are secure. The server certificate used must be valid, trusted, and contain the server name. By setting this option to `true`, all TLS verification is disabled and the artifact will be downloaded, even when the server is considered insecure. + type: boolean + repository: + description: Maven repository to download the artifact from. Applicable to the `maven` artifact type only. + type: string + sha512sum: + description: 'SHA512 checksum of the artifact. Optional. If specified, the checksum will be verified while building the new container. If not specified, the downloaded artifact will not be verified. Not applicable to the `maven` artifact type. ' + type: string + type: + description: Artifact type. Currently, the supported artifact types are `tgz`, `jar`, `zip`, `other` and `maven`. + enum: + - jar + - tgz + - zip + - maven + - other + type: string + url: + description: URL of the artifact which will be downloaded. Strimzi does not do any security scanning of the downloaded artifacts. For security reasons, you should first verify the artifacts manually and configure the checksum verification to make sure the same artifact is used in the automated build. Required for `jar`, `zip`, `tgz` and `other` artifacts. Not applicable to the `maven` artifact type. + pattern: ^(https?|ftp)://[-a-zA-Z0-9+&@#/%?=~_|!:,.;]*[-a-zA-Z0-9+&@#/%=~_|]$ + type: string + version: + description: Maven version number. Applicable to the `maven` artifact type only. + type: string + required: + - type + type: object + type: array + name: + description: 'The unique name of the connector plugin. Will be used to generate the path where the connector artifacts will be stored. The name has to be unique within the KafkaConnect resource. The name has to follow the following pattern: `^[a-z][-_a-z0-9]*[a-z]$`. Required.' + pattern: ^[a-z0-9][-_a-z0-9]*[a-z0-9]$ + type: string + required: + - name + - artifacts + type: object + type: array + resources: + description: CPU and memory resources to reserve for the build. + properties: + claims: + items: + properties: + name: + type: string + type: object + type: array + limits: + additionalProperties: + anyOf: + - type: integer + - type: string + pattern: ^(\+|-)?(([0-9]+(\.[0-9]*)?)|(\.[0-9]+))(([KMGTPE]i)|[numkMGTPE]|([eE](\+|-)?(([0-9]+(\.[0-9]*)?)|(\.[0-9]+))))?$ + x-kubernetes-int-or-string: true + type: object + requests: + additionalProperties: + anyOf: + - type: integer + - type: string + pattern: ^(\+|-)?(([0-9]+(\.[0-9]*)?)|(\.[0-9]+))(([KMGTPE]i)|[numkMGTPE]|([eE](\+|-)?(([0-9]+(\.[0-9]*)?)|(\.[0-9]+))))?$ + x-kubernetes-int-or-string: true + type: object + type: object + required: + - output + - plugins + type: object + clientRackInitImage: + description: The image of the init container used for initializing the `client.rack`. + type: string + config: + description: 'The Kafka Connect configuration. Properties with the following prefixes cannot be set: ssl., sasl., security., listeners, plugin.path, rest., bootstrap.servers, consumer.interceptor.classes, producer.interceptor.classes (with the exception of: ssl.endpoint.identification.algorithm, ssl.cipher.suites, ssl.protocol, ssl.enabled.protocols).' + type: object + x-kubernetes-preserve-unknown-fields: true + externalConfiguration: + description: Pass data from Secrets or ConfigMaps to the Kafka Connect pods and use them to configure connectors. + properties: + env: + description: Makes data from a Secret or ConfigMap available in the Kafka Connect pods as environment variables. + items: + properties: + name: + description: Name of the environment variable which will be passed to the Kafka Connect pods. The name of the environment variable cannot start with `KAFKA_` or `STRIMZI_`. + type: string + valueFrom: + description: Value of the environment variable which will be passed to the Kafka Connect pods. It can be passed either as a reference to Secret or ConfigMap field. The field has to specify exactly one Secret or ConfigMap. + oneOf: + - properties: + secretKeyRef: {} + required: + - secretKeyRef + - properties: + configMapKeyRef: {} + required: + - configMapKeyRef + properties: + configMapKeyRef: + description: Reference to a key in a ConfigMap. + properties: + key: + type: string + name: + type: string + optional: + type: boolean + type: object + secretKeyRef: + description: Reference to a key in a Secret. + properties: + key: + type: string + name: + type: string + optional: + type: boolean + type: object + type: object + required: + - name + - valueFrom + type: object + type: array + volumes: + description: Makes data from a Secret or ConfigMap available in the Kafka Connect pods as volumes. + items: + oneOf: + - properties: + secret: {} + required: + - secret + - properties: + configMap: {} + required: + - configMap + properties: + configMap: + description: Reference to a key in a ConfigMap. Exactly one Secret or ConfigMap has to be specified. + properties: + defaultMode: + type: integer + items: + items: + properties: + key: + type: string + mode: + type: integer + path: + type: string + type: object + type: array + name: + type: string + optional: + type: boolean + type: object + name: + description: Name of the volume which will be added to the Kafka Connect pods. + type: string + secret: + description: Reference to a key in a Secret. Exactly one Secret or ConfigMap has to be specified. + properties: + defaultMode: + type: integer + items: + items: + properties: + key: + type: string + mode: + type: integer + path: + type: string + type: object + type: array + optional: + type: boolean + secretName: + type: string + type: object + required: + - name + type: object + type: array + type: object + image: + description: The container image used for Kafka Connect pods. If no image name is explicitly specified, it is determined based on the `spec.version` configuration. The image names are specifically mapped to corresponding versions in the Cluster Operator configuration. + type: string + jmxOptions: + description: JMX Options. + properties: + authentication: + description: Authentication configuration for connecting to the JMX port. + properties: + type: + description: Authentication type. Currently the only supported types are `password`.`password` type creates a username and protected port with no TLS. + enum: + - password + type: string + required: + - type + type: object + type: object + jvmOptions: + description: JVM Options for pods. + properties: + -XX: + additionalProperties: + type: string + description: A map of -XX options to the JVM. + type: object + -Xms: + description: -Xms option to to the JVM. + pattern: ^[0-9]+[mMgG]?$ + type: string + -Xmx: + description: -Xmx option to to the JVM. + pattern: ^[0-9]+[mMgG]?$ + type: string + gcLoggingEnabled: + description: Specifies whether the Garbage Collection logging is enabled. The default is false. + type: boolean + javaSystemProperties: + description: A map of additional system properties which will be passed using the `-D` option to the JVM. + items: + properties: + name: + description: The system property name. + type: string + value: + description: The system property value. + type: string + type: object + type: array + type: object + livenessProbe: + description: Pod liveness checking. + properties: + failureThreshold: + description: Minimum consecutive failures for the probe to be considered failed after having succeeded. Defaults to 3. Minimum value is 1. + minimum: 1 + type: integer + initialDelaySeconds: + description: The initial delay before first the health is first checked. Default to 15 seconds. Minimum value is 0. + minimum: 0 + type: integer + periodSeconds: + description: How often (in seconds) to perform the probe. Default to 10 seconds. Minimum value is 1. + minimum: 1 + type: integer + successThreshold: + description: Minimum consecutive successes for the probe to be considered successful after having failed. Defaults to 1. Must be 1 for liveness. Minimum value is 1. + minimum: 1 + type: integer + timeoutSeconds: + description: The timeout for each attempted health check. Default to 5 seconds. Minimum value is 1. + minimum: 1 + type: integer + type: object + logging: + description: Logging configuration for Kafka Connect. + properties: + loggers: + additionalProperties: + type: string + description: A Map from logger name to logger level. + type: object + type: + description: Logging type, must be either 'inline' or 'external'. + enum: + - inline + - external + type: string + valueFrom: + description: '`ConfigMap` entry where the logging configuration is stored. ' + properties: + configMapKeyRef: + description: Reference to the key in the ConfigMap containing the configuration. + properties: + key: + type: string + name: + type: string + optional: + type: boolean + type: object + type: object + required: + - type + type: object + metricsConfig: + description: Metrics configuration. + properties: + type: + description: Metrics type. Only 'jmxPrometheusExporter' supported currently. + enum: + - jmxPrometheusExporter + type: string + valueFrom: + description: 'ConfigMap entry where the Prometheus JMX Exporter configuration is stored. ' + properties: + configMapKeyRef: + description: Reference to the key in the ConfigMap containing the configuration. + properties: + key: + type: string + name: + type: string + optional: + type: boolean + type: object + type: object + required: + - type + - valueFrom + type: object + rack: + description: Configuration of the node label which will be used as the `client.rack` consumer configuration. + properties: + topologyKey: + description: A key that matches labels assigned to the Kubernetes cluster nodes. The value of the label is used to set a broker's `broker.rack` config, and the `client.rack` config for Kafka Connect or MirrorMaker 2. + example: topology.kubernetes.io/zone + type: string + required: + - topologyKey + type: object + readinessProbe: + description: Pod readiness checking. + properties: + failureThreshold: + description: Minimum consecutive failures for the probe to be considered failed after having succeeded. Defaults to 3. Minimum value is 1. + minimum: 1 + type: integer + initialDelaySeconds: + description: The initial delay before first the health is first checked. Default to 15 seconds. Minimum value is 0. + minimum: 0 + type: integer + periodSeconds: + description: How often (in seconds) to perform the probe. Default to 10 seconds. Minimum value is 1. + minimum: 1 + type: integer + successThreshold: + description: Minimum consecutive successes for the probe to be considered successful after having failed. Defaults to 1. Must be 1 for liveness. Minimum value is 1. + minimum: 1 + type: integer + timeoutSeconds: + description: The timeout for each attempted health check. Default to 5 seconds. Minimum value is 1. + minimum: 1 + type: integer + type: object + replicas: + description: The number of pods in the Kafka Connect group. Defaults to `3`. + type: integer + resources: + description: The maximum limits for CPU and memory resources and the requested initial resources. + properties: + claims: + items: + properties: + name: + type: string + type: object + type: array + limits: + additionalProperties: + anyOf: + - type: integer + - type: string + pattern: ^(\+|-)?(([0-9]+(\.[0-9]*)?)|(\.[0-9]+))(([KMGTPE]i)|[numkMGTPE]|([eE](\+|-)?(([0-9]+(\.[0-9]*)?)|(\.[0-9]+))))?$ + x-kubernetes-int-or-string: true + type: object + requests: + additionalProperties: + anyOf: + - type: integer + - type: string + pattern: ^(\+|-)?(([0-9]+(\.[0-9]*)?)|(\.[0-9]+))(([KMGTPE]i)|[numkMGTPE]|([eE](\+|-)?(([0-9]+(\.[0-9]*)?)|(\.[0-9]+))))?$ + x-kubernetes-int-or-string: true + type: object + type: object + template: + description: Template for Kafka Connect and Kafka MirrorMaker 2 resources. The template allows users to specify how the `Pods`, `Service`, and other services are generated. + properties: + apiService: + description: Template for Kafka Connect API `Service`. + properties: + ipFamilies: + description: Specifies the IP Families used by the service. Available options are `IPv4` and `IPv6`. If unspecified, Kubernetes will choose the default value based on the `ipFamilyPolicy` setting. + items: + enum: + - IPv4 + - IPv6 + type: string + type: array + ipFamilyPolicy: + description: Specifies the IP Family Policy used by the service. Available options are `SingleStack`, `PreferDualStack` and `RequireDualStack`. `SingleStack` is for a single IP family. `PreferDualStack` is for two IP families on dual-stack configured clusters or a single IP family on single-stack clusters. `RequireDualStack` fails unless there are two IP families on dual-stack configured clusters. If unspecified, Kubernetes will choose the default value based on the service type. + enum: + - SingleStack + - PreferDualStack + - RequireDualStack + type: string + metadata: + description: Metadata applied to the resource. + properties: + annotations: + additionalProperties: + type: string + description: Annotations added to the Kubernetes resource. + type: object + labels: + additionalProperties: + type: string + description: Labels added to the Kubernetes resource. + type: object + type: object + type: object + buildConfig: + description: Template for the Kafka Connect BuildConfig used to build new container images. The BuildConfig is used only on OpenShift. + properties: + metadata: + description: Metadata to apply to the `PodDisruptionBudgetTemplate` resource. + properties: + annotations: + additionalProperties: + type: string + description: Annotations added to the Kubernetes resource. + type: object + labels: + additionalProperties: + type: string + description: Labels added to the Kubernetes resource. + type: object + type: object + pullSecret: + description: Container Registry Secret with the credentials for pulling the base image. + type: string + type: object + buildContainer: + description: Template for the Kafka Connect Build container. The build container is used only on Kubernetes. + properties: + env: + description: Environment variables which should be applied to the container. + items: + oneOf: + - properties: + value: {} + required: + - value + - properties: + valueFrom: {} + required: + - valueFrom + properties: + name: + description: The environment variable key. + type: string + value: + description: The environment variable value. + type: string + valueFrom: + description: Reference to the secret or config map property to which the environment variable is set. + oneOf: + - properties: + secretKeyRef: {} + required: + - secretKeyRef + - properties: + configMapKeyRef: {} + required: + - configMapKeyRef + properties: + configMapKeyRef: + description: Reference to a key in a config map. + properties: + key: + type: string + name: + type: string + optional: + type: boolean + type: object + secretKeyRef: + description: Reference to a key in a secret. + properties: + key: + type: string + name: + type: string + optional: + type: boolean + type: object + type: object + type: object + type: array + securityContext: + description: Security context for the container. + properties: + allowPrivilegeEscalation: + type: boolean + appArmorProfile: + properties: + localhostProfile: + type: string + type: + type: string + type: object + capabilities: + properties: + add: + items: + type: string + type: array + drop: + items: + type: string + type: array + type: object + privileged: + type: boolean + procMount: + type: string + readOnlyRootFilesystem: + type: boolean + runAsGroup: + type: integer + runAsNonRoot: + type: boolean + runAsUser: + type: integer + seLinuxOptions: + properties: + level: + type: string + role: + type: string + type: + type: string + user: + type: string + type: object + seccompProfile: + properties: + localhostProfile: + type: string + type: + type: string + type: object + windowsOptions: + properties: + gmsaCredentialSpec: + type: string + gmsaCredentialSpecName: + type: string + hostProcess: + type: boolean + runAsUserName: + type: string + type: object + type: object + volumeMounts: + description: Additional volume mounts which should be applied to the container. + items: + properties: + mountPath: + type: string + mountPropagation: + type: string + name: + type: string + readOnly: + type: boolean + recursiveReadOnly: + type: string + subPath: + type: string + subPathExpr: + type: string + type: object + type: array + type: object + buildPod: + description: Template for Kafka Connect Build `Pods`. The build pod is used only on Kubernetes. + properties: + affinity: + description: The pod's affinity rules. + properties: + nodeAffinity: + properties: + preferredDuringSchedulingIgnoredDuringExecution: + items: + properties: + preference: + properties: + matchExpressions: + items: + properties: + key: + type: string + operator: + type: string + values: + items: + type: string + type: array + type: object + type: array + matchFields: + items: + properties: + key: + type: string + operator: + type: string + values: + items: + type: string + type: array + type: object + type: array + type: object + weight: + type: integer + type: object + type: array + requiredDuringSchedulingIgnoredDuringExecution: + properties: + nodeSelectorTerms: + items: + properties: + matchExpressions: + items: + properties: + key: + type: string + operator: + type: string + values: + items: + type: string + type: array + type: object + type: array + matchFields: + items: + properties: + key: + type: string + operator: + type: string + values: + items: + type: string + type: array + type: object + type: array + type: object + type: array + type: object + type: object + podAffinity: + properties: + preferredDuringSchedulingIgnoredDuringExecution: + items: + properties: + podAffinityTerm: + properties: + labelSelector: + properties: + matchExpressions: + items: + properties: + key: + type: string + operator: + type: string + values: + items: + type: string + type: array + type: object + type: array + matchLabels: + additionalProperties: + type: string + type: object + type: object + matchLabelKeys: + items: + type: string + type: array + mismatchLabelKeys: + items: + type: string + type: array + namespaceSelector: + properties: + matchExpressions: + items: + properties: + key: + type: string + operator: + type: string + values: + items: + type: string + type: array + type: object + type: array + matchLabels: + additionalProperties: + type: string + type: object + type: object + namespaces: + items: + type: string + type: array + topologyKey: + type: string + type: object + weight: + type: integer + type: object + type: array + requiredDuringSchedulingIgnoredDuringExecution: + items: + properties: + labelSelector: + properties: + matchExpressions: + items: + properties: + key: + type: string + operator: + type: string + values: + items: + type: string + type: array + type: object + type: array + matchLabels: + additionalProperties: + type: string + type: object + type: object + matchLabelKeys: + items: + type: string + type: array + mismatchLabelKeys: + items: + type: string + type: array + namespaceSelector: + properties: + matchExpressions: + items: + properties: + key: + type: string + operator: + type: string + values: + items: + type: string + type: array + type: object + type: array + matchLabels: + additionalProperties: + type: string + type: object + type: object + namespaces: + items: + type: string + type: array + topologyKey: + type: string + type: object + type: array + type: object + podAntiAffinity: + properties: + preferredDuringSchedulingIgnoredDuringExecution: + items: + properties: + podAffinityTerm: + properties: + labelSelector: + properties: + matchExpressions: + items: + properties: + key: + type: string + operator: + type: string + values: + items: + type: string + type: array + type: object + type: array + matchLabels: + additionalProperties: + type: string + type: object + type: object + matchLabelKeys: + items: + type: string + type: array + mismatchLabelKeys: + items: + type: string + type: array + namespaceSelector: + properties: + matchExpressions: + items: + properties: + key: + type: string + operator: + type: string + values: + items: + type: string + type: array + type: object + type: array + matchLabels: + additionalProperties: + type: string + type: object + type: object + namespaces: + items: + type: string + type: array + topologyKey: + type: string + type: object + weight: + type: integer + type: object + type: array + requiredDuringSchedulingIgnoredDuringExecution: + items: + properties: + labelSelector: + properties: + matchExpressions: + items: + properties: + key: + type: string + operator: + type: string + values: + items: + type: string + type: array + type: object + type: array + matchLabels: + additionalProperties: + type: string + type: object + type: object + matchLabelKeys: + items: + type: string + type: array + mismatchLabelKeys: + items: + type: string + type: array + namespaceSelector: + properties: + matchExpressions: + items: + properties: + key: + type: string + operator: + type: string + values: + items: + type: string + type: array + type: object + type: array + matchLabels: + additionalProperties: + type: string + type: object + type: object + namespaces: + items: + type: string + type: array + topologyKey: + type: string + type: object + type: array + type: object + type: object + enableServiceLinks: + description: Indicates whether information about services should be injected into Pod's environment variables. + type: boolean + hostAliases: + description: The pod's HostAliases. HostAliases is an optional list of hosts and IPs that will be injected into the Pod's hosts file if specified. + items: + properties: + hostnames: + items: + type: string + type: array + ip: + type: string + type: object + type: array + imagePullSecrets: + description: List of references to secrets in the same namespace to use for pulling any of the images used by this Pod. When the `STRIMZI_IMAGE_PULL_SECRETS` environment variable in Cluster Operator and the `imagePullSecrets` option are specified, only the `imagePullSecrets` variable is used and the `STRIMZI_IMAGE_PULL_SECRETS` variable is ignored. + items: + properties: + name: + type: string + type: object + type: array + metadata: + description: Metadata applied to the resource. + properties: + annotations: + additionalProperties: + type: string + description: Annotations added to the Kubernetes resource. + type: object + labels: + additionalProperties: + type: string + description: Labels added to the Kubernetes resource. + type: object + type: object + priorityClassName: + description: 'The name of the priority class used to assign priority to the pods. ' + type: string + schedulerName: + description: The name of the scheduler used to dispatch this `Pod`. If not specified, the default scheduler will be used. + type: string + securityContext: + description: Configures pod-level security attributes and common container settings. + properties: + appArmorProfile: + properties: + localhostProfile: + type: string + type: + type: string + type: object + fsGroup: + type: integer + fsGroupChangePolicy: + type: string + runAsGroup: + type: integer + runAsNonRoot: + type: boolean + runAsUser: + type: integer + seLinuxOptions: + properties: + level: + type: string + role: + type: string + type: + type: string + user: + type: string + type: object + seccompProfile: + properties: + localhostProfile: + type: string + type: + type: string + type: object + supplementalGroups: + items: + type: integer + type: array + sysctls: + items: + properties: + name: + type: string + value: + type: string + type: object + type: array + windowsOptions: + properties: + gmsaCredentialSpec: + type: string + gmsaCredentialSpecName: + type: string + hostProcess: + type: boolean + runAsUserName: + type: string + type: object + type: object + terminationGracePeriodSeconds: + description: The grace period is the duration in seconds after the processes running in the pod are sent a termination signal, and the time when the processes are forcibly halted with a kill signal. Set this value to longer than the expected cleanup time for your process. Value must be a non-negative integer. A zero value indicates delete immediately. You might need to increase the grace period for very large Kafka clusters, so that the Kafka brokers have enough time to transfer their work to another broker before they are terminated. Defaults to 30 seconds. + minimum: 0 + type: integer + tmpDirSizeLimit: + description: Defines the total amount of pod memory allocated for the temporary `EmptyDir` volume `/tmp`. Specify the allocation in memory units, for example, `100Mi` for 100 mebibytes. Default value is `5Mi`. The `/tmp` volume is backed by pod memory, not disk storage, so avoid setting a high value as it consumes pod memory resources. + pattern: ^([0-9.]+)([eEinumkKMGTP]*[-+]?[0-9]*)$ + type: string + tolerations: + description: The pod's tolerations. + items: + properties: + effect: + type: string + key: + type: string + operator: + type: string + tolerationSeconds: + type: integer + value: + type: string + type: object + type: array + topologySpreadConstraints: + description: The pod's topology spread constraints. + items: + properties: + labelSelector: + properties: + matchExpressions: + items: + properties: + key: + type: string + operator: + type: string + values: + items: + type: string + type: array + type: object + type: array + matchLabels: + additionalProperties: + type: string + type: object + type: object + matchLabelKeys: + items: + type: string + type: array + maxSkew: + type: integer + minDomains: + type: integer + nodeAffinityPolicy: + type: string + nodeTaintsPolicy: + type: string + topologyKey: + type: string + whenUnsatisfiable: + type: string + type: object + type: array + volumes: + description: Additional volumes that can be mounted to the pod. + items: + oneOf: + - properties: + configMap: {} + csi: {} + emptyDir: {} + persistentVolumeClaim: {} + secret: {} + properties: + configMap: + description: ConfigMap to use to populate the volume. + properties: + defaultMode: + type: integer + items: + items: + properties: + key: + type: string + mode: + type: integer + path: + type: string + type: object + type: array + name: + type: string + optional: + type: boolean + type: object + csi: + description: CSIVolumeSource object to use to populate the volume. + properties: + driver: + type: string + fsType: + type: string + nodePublishSecretRef: + properties: + name: + type: string + type: object + readOnly: + type: boolean + volumeAttributes: + additionalProperties: + type: string + type: object + type: object + emptyDir: + description: EmptyDir to use to populate the volume. + properties: + medium: + type: string + sizeLimit: + properties: + amount: + type: string + format: + type: string + type: object + type: object + name: + description: Name to use for the volume. Required. + type: string + persistentVolumeClaim: + description: PersistentVolumeClaim object to use to populate the volume. + properties: + claimName: + type: string + readOnly: + type: boolean + type: object + secret: + description: Secret to use populate the volume. + properties: + defaultMode: + type: integer + items: + items: + properties: + key: + type: string + mode: + type: integer + path: + type: string + type: object + type: array + optional: + type: boolean + secretName: + type: string + type: object + type: object + type: array + type: object + buildServiceAccount: + description: Template for the Kafka Connect Build service account. + properties: + metadata: + description: Metadata applied to the resource. + properties: + annotations: + additionalProperties: + type: string + description: Annotations added to the Kubernetes resource. + type: object + labels: + additionalProperties: + type: string + description: Labels added to the Kubernetes resource. + type: object + type: object + type: object + clusterRoleBinding: + description: Template for the Kafka Connect ClusterRoleBinding. + properties: + metadata: + description: Metadata applied to the resource. + properties: + annotations: + additionalProperties: + type: string + description: Annotations added to the Kubernetes resource. + type: object + labels: + additionalProperties: + type: string + description: Labels added to the Kubernetes resource. + type: object + type: object + type: object + connectContainer: + description: Template for the Kafka Connect container. + properties: + env: + description: Environment variables which should be applied to the container. + items: + oneOf: + - properties: + value: {} + required: + - value + - properties: + valueFrom: {} + required: + - valueFrom + properties: + name: + description: The environment variable key. + type: string + value: + description: The environment variable value. + type: string + valueFrom: + description: Reference to the secret or config map property to which the environment variable is set. + oneOf: + - properties: + secretKeyRef: {} + required: + - secretKeyRef + - properties: + configMapKeyRef: {} + required: + - configMapKeyRef + properties: + configMapKeyRef: + description: Reference to a key in a config map. + properties: + key: + type: string + name: + type: string + optional: + type: boolean + type: object + secretKeyRef: + description: Reference to a key in a secret. + properties: + key: + type: string + name: + type: string + optional: + type: boolean + type: object + type: object + type: object + type: array + securityContext: + description: Security context for the container. + properties: + allowPrivilegeEscalation: + type: boolean + appArmorProfile: + properties: + localhostProfile: + type: string + type: + type: string + type: object + capabilities: + properties: + add: + items: + type: string + type: array + drop: + items: + type: string + type: array + type: object + privileged: + type: boolean + procMount: + type: string + readOnlyRootFilesystem: + type: boolean + runAsGroup: + type: integer + runAsNonRoot: + type: boolean + runAsUser: + type: integer + seLinuxOptions: + properties: + level: + type: string + role: + type: string + type: + type: string + user: + type: string + type: object + seccompProfile: + properties: + localhostProfile: + type: string + type: + type: string + type: object + windowsOptions: + properties: + gmsaCredentialSpec: + type: string + gmsaCredentialSpecName: + type: string + hostProcess: + type: boolean + runAsUserName: + type: string + type: object + type: object + volumeMounts: + description: Additional volume mounts which should be applied to the container. + items: + properties: + mountPath: + type: string + mountPropagation: + type: string + name: + type: string + readOnly: + type: boolean + recursiveReadOnly: + type: string + subPath: + type: string + subPathExpr: + type: string + type: object + type: array + type: object + deployment: + description: Template for Kafka Connect `Deployment`. + properties: + deploymentStrategy: + description: Pod replacement strategy for deployment configuration changes. Valid values are `RollingUpdate` and `Recreate`. Defaults to `RollingUpdate`. + enum: + - RollingUpdate + - Recreate + type: string + metadata: + description: Metadata applied to the resource. + properties: + annotations: + additionalProperties: + type: string + description: Annotations added to the Kubernetes resource. + type: object + labels: + additionalProperties: + type: string + description: Labels added to the Kubernetes resource. + type: object + type: object + type: object + headlessService: + description: Template for Kafka Connect headless `Service`. + properties: + ipFamilies: + description: Specifies the IP Families used by the service. Available options are `IPv4` and `IPv6`. If unspecified, Kubernetes will choose the default value based on the `ipFamilyPolicy` setting. + items: + enum: + - IPv4 + - IPv6 + type: string + type: array + ipFamilyPolicy: + description: Specifies the IP Family Policy used by the service. Available options are `SingleStack`, `PreferDualStack` and `RequireDualStack`. `SingleStack` is for a single IP family. `PreferDualStack` is for two IP families on dual-stack configured clusters or a single IP family on single-stack clusters. `RequireDualStack` fails unless there are two IP families on dual-stack configured clusters. If unspecified, Kubernetes will choose the default value based on the service type. + enum: + - SingleStack + - PreferDualStack + - RequireDualStack + type: string + metadata: + description: Metadata applied to the resource. + properties: + annotations: + additionalProperties: + type: string + description: Annotations added to the Kubernetes resource. + type: object + labels: + additionalProperties: + type: string + description: Labels added to the Kubernetes resource. + type: object + type: object + type: object + initContainer: + description: Template for the Kafka init container. + properties: + env: + description: Environment variables which should be applied to the container. + items: + oneOf: + - properties: + value: {} + required: + - value + - properties: + valueFrom: {} + required: + - valueFrom + properties: + name: + description: The environment variable key. + type: string + value: + description: The environment variable value. + type: string + valueFrom: + description: Reference to the secret or config map property to which the environment variable is set. + oneOf: + - properties: + secretKeyRef: {} + required: + - secretKeyRef + - properties: + configMapKeyRef: {} + required: + - configMapKeyRef + properties: + configMapKeyRef: + description: Reference to a key in a config map. + properties: + key: + type: string + name: + type: string + optional: + type: boolean + type: object + secretKeyRef: + description: Reference to a key in a secret. + properties: + key: + type: string + name: + type: string + optional: + type: boolean + type: object + type: object + type: object + type: array + securityContext: + description: Security context for the container. + properties: + allowPrivilegeEscalation: + type: boolean + appArmorProfile: + properties: + localhostProfile: + type: string + type: + type: string + type: object + capabilities: + properties: + add: + items: + type: string + type: array + drop: + items: + type: string + type: array + type: object + privileged: + type: boolean + procMount: + type: string + readOnlyRootFilesystem: + type: boolean + runAsGroup: + type: integer + runAsNonRoot: + type: boolean + runAsUser: + type: integer + seLinuxOptions: + properties: + level: + type: string + role: + type: string + type: + type: string + user: + type: string + type: object + seccompProfile: + properties: + localhostProfile: + type: string + type: + type: string + type: object + windowsOptions: + properties: + gmsaCredentialSpec: + type: string + gmsaCredentialSpecName: + type: string + hostProcess: + type: boolean + runAsUserName: + type: string + type: object + type: object + volumeMounts: + description: Additional volume mounts which should be applied to the container. + items: + properties: + mountPath: + type: string + mountPropagation: + type: string + name: + type: string + readOnly: + type: boolean + recursiveReadOnly: + type: string + subPath: + type: string + subPathExpr: + type: string + type: object + type: array + type: object + jmxSecret: + description: Template for Secret of the Kafka Connect Cluster JMX authentication. + properties: + metadata: + description: Metadata applied to the resource. + properties: + annotations: + additionalProperties: + type: string + description: Annotations added to the Kubernetes resource. + type: object + labels: + additionalProperties: + type: string + description: Labels added to the Kubernetes resource. + type: object + type: object + type: object + pod: + description: Template for Kafka Connect `Pods`. + properties: + affinity: + description: The pod's affinity rules. + properties: + nodeAffinity: + properties: + preferredDuringSchedulingIgnoredDuringExecution: + items: + properties: + preference: + properties: + matchExpressions: + items: + properties: + key: + type: string + operator: + type: string + values: + items: + type: string + type: array + type: object + type: array + matchFields: + items: + properties: + key: + type: string + operator: + type: string + values: + items: + type: string + type: array + type: object + type: array + type: object + weight: + type: integer + type: object + type: array + requiredDuringSchedulingIgnoredDuringExecution: + properties: + nodeSelectorTerms: + items: + properties: + matchExpressions: + items: + properties: + key: + type: string + operator: + type: string + values: + items: + type: string + type: array + type: object + type: array + matchFields: + items: + properties: + key: + type: string + operator: + type: string + values: + items: + type: string + type: array + type: object + type: array + type: object + type: array + type: object + type: object + podAffinity: + properties: + preferredDuringSchedulingIgnoredDuringExecution: + items: + properties: + podAffinityTerm: + properties: + labelSelector: + properties: + matchExpressions: + items: + properties: + key: + type: string + operator: + type: string + values: + items: + type: string + type: array + type: object + type: array + matchLabels: + additionalProperties: + type: string + type: object + type: object + matchLabelKeys: + items: + type: string + type: array + mismatchLabelKeys: + items: + type: string + type: array + namespaceSelector: + properties: + matchExpressions: + items: + properties: + key: + type: string + operator: + type: string + values: + items: + type: string + type: array + type: object + type: array + matchLabels: + additionalProperties: + type: string + type: object + type: object + namespaces: + items: + type: string + type: array + topologyKey: + type: string + type: object + weight: + type: integer + type: object + type: array + requiredDuringSchedulingIgnoredDuringExecution: + items: + properties: + labelSelector: + properties: + matchExpressions: + items: + properties: + key: + type: string + operator: + type: string + values: + items: + type: string + type: array + type: object + type: array + matchLabels: + additionalProperties: + type: string + type: object + type: object + matchLabelKeys: + items: + type: string + type: array + mismatchLabelKeys: + items: + type: string + type: array + namespaceSelector: + properties: + matchExpressions: + items: + properties: + key: + type: string + operator: + type: string + values: + items: + type: string + type: array + type: object + type: array + matchLabels: + additionalProperties: + type: string + type: object + type: object + namespaces: + items: + type: string + type: array + topologyKey: + type: string + type: object + type: array + type: object + podAntiAffinity: + properties: + preferredDuringSchedulingIgnoredDuringExecution: + items: + properties: + podAffinityTerm: + properties: + labelSelector: + properties: + matchExpressions: + items: + properties: + key: + type: string + operator: + type: string + values: + items: + type: string + type: array + type: object + type: array + matchLabels: + additionalProperties: + type: string + type: object + type: object + matchLabelKeys: + items: + type: string + type: array + mismatchLabelKeys: + items: + type: string + type: array + namespaceSelector: + properties: + matchExpressions: + items: + properties: + key: + type: string + operator: + type: string + values: + items: + type: string + type: array + type: object + type: array + matchLabels: + additionalProperties: + type: string + type: object + type: object + namespaces: + items: + type: string + type: array + topologyKey: + type: string + type: object + weight: + type: integer + type: object + type: array + requiredDuringSchedulingIgnoredDuringExecution: + items: + properties: + labelSelector: + properties: + matchExpressions: + items: + properties: + key: + type: string + operator: + type: string + values: + items: + type: string + type: array + type: object + type: array + matchLabels: + additionalProperties: + type: string + type: object + type: object + matchLabelKeys: + items: + type: string + type: array + mismatchLabelKeys: + items: + type: string + type: array + namespaceSelector: + properties: + matchExpressions: + items: + properties: + key: + type: string + operator: + type: string + values: + items: + type: string + type: array + type: object + type: array + matchLabels: + additionalProperties: + type: string + type: object + type: object + namespaces: + items: + type: string + type: array + topologyKey: + type: string + type: object + type: array + type: object + type: object + enableServiceLinks: + description: Indicates whether information about services should be injected into Pod's environment variables. + type: boolean + hostAliases: + description: The pod's HostAliases. HostAliases is an optional list of hosts and IPs that will be injected into the Pod's hosts file if specified. + items: + properties: + hostnames: + items: + type: string + type: array + ip: + type: string + type: object + type: array + imagePullSecrets: + description: List of references to secrets in the same namespace to use for pulling any of the images used by this Pod. When the `STRIMZI_IMAGE_PULL_SECRETS` environment variable in Cluster Operator and the `imagePullSecrets` option are specified, only the `imagePullSecrets` variable is used and the `STRIMZI_IMAGE_PULL_SECRETS` variable is ignored. + items: + properties: + name: + type: string + type: object + type: array + metadata: + description: Metadata applied to the resource. + properties: + annotations: + additionalProperties: + type: string + description: Annotations added to the Kubernetes resource. + type: object + labels: + additionalProperties: + type: string + description: Labels added to the Kubernetes resource. + type: object + type: object + priorityClassName: + description: 'The name of the priority class used to assign priority to the pods. ' + type: string + schedulerName: + description: The name of the scheduler used to dispatch this `Pod`. If not specified, the default scheduler will be used. + type: string + securityContext: + description: Configures pod-level security attributes and common container settings. + properties: + appArmorProfile: + properties: + localhostProfile: + type: string + type: + type: string + type: object + fsGroup: + type: integer + fsGroupChangePolicy: + type: string + runAsGroup: + type: integer + runAsNonRoot: + type: boolean + runAsUser: + type: integer + seLinuxOptions: + properties: + level: + type: string + role: + type: string + type: + type: string + user: + type: string + type: object + seccompProfile: + properties: + localhostProfile: + type: string + type: + type: string + type: object + supplementalGroups: + items: + type: integer + type: array + sysctls: + items: + properties: + name: + type: string + value: + type: string + type: object + type: array + windowsOptions: + properties: + gmsaCredentialSpec: + type: string + gmsaCredentialSpecName: + type: string + hostProcess: + type: boolean + runAsUserName: + type: string + type: object + type: object + terminationGracePeriodSeconds: + description: The grace period is the duration in seconds after the processes running in the pod are sent a termination signal, and the time when the processes are forcibly halted with a kill signal. Set this value to longer than the expected cleanup time for your process. Value must be a non-negative integer. A zero value indicates delete immediately. You might need to increase the grace period for very large Kafka clusters, so that the Kafka brokers have enough time to transfer their work to another broker before they are terminated. Defaults to 30 seconds. + minimum: 0 + type: integer + tmpDirSizeLimit: + description: Defines the total amount of pod memory allocated for the temporary `EmptyDir` volume `/tmp`. Specify the allocation in memory units, for example, `100Mi` for 100 mebibytes. Default value is `5Mi`. The `/tmp` volume is backed by pod memory, not disk storage, so avoid setting a high value as it consumes pod memory resources. + pattern: ^([0-9.]+)([eEinumkKMGTP]*[-+]?[0-9]*)$ + type: string + tolerations: + description: The pod's tolerations. + items: + properties: + effect: + type: string + key: + type: string + operator: + type: string + tolerationSeconds: + type: integer + value: + type: string + type: object + type: array + topologySpreadConstraints: + description: The pod's topology spread constraints. + items: + properties: + labelSelector: + properties: + matchExpressions: + items: + properties: + key: + type: string + operator: + type: string + values: + items: + type: string + type: array + type: object + type: array + matchLabels: + additionalProperties: + type: string + type: object + type: object + matchLabelKeys: + items: + type: string + type: array + maxSkew: + type: integer + minDomains: + type: integer + nodeAffinityPolicy: + type: string + nodeTaintsPolicy: + type: string + topologyKey: + type: string + whenUnsatisfiable: + type: string + type: object + type: array + volumes: + description: Additional volumes that can be mounted to the pod. + items: + oneOf: + - properties: + configMap: {} + csi: {} + emptyDir: {} + persistentVolumeClaim: {} + secret: {} + properties: + configMap: + description: ConfigMap to use to populate the volume. + properties: + defaultMode: + type: integer + items: + items: + properties: + key: + type: string + mode: + type: integer + path: + type: string + type: object + type: array + name: + type: string + optional: + type: boolean + type: object + csi: + description: CSIVolumeSource object to use to populate the volume. + properties: + driver: + type: string + fsType: + type: string + nodePublishSecretRef: + properties: + name: + type: string + type: object + readOnly: + type: boolean + volumeAttributes: + additionalProperties: + type: string + type: object + type: object + emptyDir: + description: EmptyDir to use to populate the volume. + properties: + medium: + type: string + sizeLimit: + properties: + amount: + type: string + format: + type: string + type: object + type: object + name: + description: Name to use for the volume. Required. + type: string + persistentVolumeClaim: + description: PersistentVolumeClaim object to use to populate the volume. + properties: + claimName: + type: string + readOnly: + type: boolean + type: object + secret: + description: Secret to use populate the volume. + properties: + defaultMode: + type: integer + items: + items: + properties: + key: + type: string + mode: + type: integer + path: + type: string + type: object + type: array + optional: + type: boolean + secretName: + type: string + type: object + type: object + type: array + type: object + podDisruptionBudget: + description: Template for Kafka Connect `PodDisruptionBudget`. + properties: + maxUnavailable: + description: Maximum number of unavailable pods to allow automatic Pod eviction. A Pod eviction is allowed when the `maxUnavailable` number of pods or fewer are unavailable after the eviction. Setting this value to 0 prevents all voluntary evictions, so the pods must be evicted manually. Defaults to 1. + minimum: 0 + type: integer + metadata: + description: Metadata to apply to the `PodDisruptionBudgetTemplate` resource. + properties: + annotations: + additionalProperties: + type: string + description: Annotations added to the Kubernetes resource. + type: object + labels: + additionalProperties: + type: string + description: Labels added to the Kubernetes resource. + type: object + type: object + type: object + podSet: + description: Template for Kafka Connect `StrimziPodSet` resource. + properties: + metadata: + description: Metadata applied to the resource. + properties: + annotations: + additionalProperties: + type: string + description: Annotations added to the Kubernetes resource. + type: object + labels: + additionalProperties: + type: string + description: Labels added to the Kubernetes resource. + type: object + type: object + type: object + serviceAccount: + description: Template for the Kafka Connect service account. + properties: + metadata: + description: Metadata applied to the resource. + properties: + annotations: + additionalProperties: + type: string + description: Annotations added to the Kubernetes resource. + type: object + labels: + additionalProperties: + type: string + description: Labels added to the Kubernetes resource. + type: object + type: object + type: object + type: object + tls: + description: TLS configuration. + properties: + trustedCertificates: + description: Trusted certificates for TLS connection. + items: + oneOf: + - properties: + certificate: {} + required: + - certificate + - properties: + pattern: {} + required: + - pattern + properties: + certificate: + description: The name of the file certificate in the secret. + type: string + pattern: + description: Pattern for the certificate files in the secret. Use the link:https://en.wikipedia.org/wiki/Glob_(programming)[_glob syntax_] for the pattern. All files in the secret that match the pattern are used. + type: string + secretName: + description: The name of the Secret containing the certificate. + type: string + required: + - secretName + type: object + type: array + type: object + tracing: + description: The configuration of tracing in Kafka Connect. + properties: + type: + description: Type of the tracing used. Currently the only supported type is `opentelemetry` for OpenTelemetry tracing. As of Strimzi 0.37.0, `jaeger` type is not supported anymore and this option is ignored. + enum: + - jaeger + - opentelemetry + type: string + required: + - type + type: object + version: + description: The Kafka Connect version. Defaults to the latest version. Consult the user documentation to understand the process required to upgrade or downgrade the version. + type: string + required: + - bootstrapServers + type: object + status: + description: The status of the Kafka Connect cluster. + properties: + conditions: + description: List of status conditions. + items: + properties: + lastTransitionTime: + description: Last time the condition of a type changed from one status to another. The required format is 'yyyy-MM-ddTHH:mm:ssZ', in the UTC time zone. + type: string + message: + description: Human-readable message indicating details about the condition's last transition. + type: string + reason: + description: The reason for the condition's last transition (a single word in CamelCase). + type: string + status: + description: The status of the condition, either True, False or Unknown. + type: string + type: + description: The unique identifier of a condition, used to distinguish between other conditions in the resource. + type: string + type: object + type: array + connectorPlugins: + description: The list of connector plugins available in this Kafka Connect deployment. + items: + properties: + class: + description: The class of the connector plugin. + type: string + type: + description: The type of the connector plugin. The available types are `sink` and `source`. + type: string + version: + description: The version of the connector plugin. + type: string + type: object + type: array + labelSelector: + description: Label selector for pods providing this resource. + type: string + observedGeneration: + description: The generation of the CRD that was last reconciled by the operator. + type: integer + replicas: + description: The current number of pods being used to provide this resource. + type: integer + url: + description: The URL of the REST API endpoint for managing and monitoring Kafka Connect connectors. + type: string + type: object + type: object + served: true + storage: true + subresources: + scale: + labelSelectorPath: .status.labelSelector + specReplicasPath: .spec.replicas + statusReplicasPath: .status.replicas + status: {} +--- +apiVersion: rbac.authorization.k8s.io/v1 +kind: ClusterRoleBinding +metadata: + labels: + app: strimzi + name: strimzi-cluster-operator-kafka-broker-delegation +roleRef: + apiGroup: rbac.authorization.k8s.io + kind: ClusterRole + name: strimzi-kafka-broker +subjects: + - kind: ServiceAccount + name: strimzi-cluster-operator + namespace: kafka +--- +apiVersion: apiextensions.k8s.io/v1 +kind: CustomResourceDefinition +metadata: + labels: + app: strimzi + strimzi.io/crd-install: "true" + name: kafkarebalances.kafka.strimzi.io +spec: + conversion: + strategy: None + group: kafka.strimzi.io + names: + categories: + - strimzi + kind: KafkaRebalance + listKind: KafkaRebalanceList + plural: kafkarebalances + shortNames: + - kr + singular: kafkarebalance + scope: Namespaced + versions: + - additionalPrinterColumns: + - description: The name of the Kafka cluster this resource rebalances + jsonPath: .metadata.labels.strimzi\.io/cluster + name: Cluster + type: string + - description: If this rebalance resource is a template + jsonPath: .metadata.annotations.strimzi\.io/rebalance-template + name: Template + type: string + - description: Status of the current rebalancing operation + jsonPath: .status.conditions[*].type + name: Status + type: string + name: v1beta2 + schema: + openAPIV3Schema: + properties: + apiVersion: + description: 'APIVersion defines the versioned schema of this representation of an object. Servers should convert recognized schemas to the latest internal value, and may reject unrecognized values. More info: https://git.k8s.io/community/contributors/devel/sig-architecture/api-conventions.md#resources' + type: string + kind: + description: 'Kind is a string value representing the REST resource this object represents. Servers may infer this from the endpoint the client submits requests to. Cannot be updated. In CamelCase. More info: https://git.k8s.io/community/contributors/devel/sig-architecture/api-conventions.md#types-kinds' + type: string + metadata: + type: object + spec: + description: The specification of the Kafka rebalance. + properties: + brokers: + description: The list of newly added brokers in case of scaling up or the ones to be removed in case of scaling down to use for rebalancing. This list can be used only with rebalancing mode `add-brokers` and `removed-brokers`. It is ignored with `full` mode. + items: + type: integer + type: array + concurrentIntraBrokerPartitionMovements: + description: The upper bound of ongoing partition replica movements between disks within each broker. Default is 2. + minimum: 0 + type: integer + concurrentLeaderMovements: + description: The upper bound of ongoing partition leadership movements. Default is 1000. + minimum: 0 + type: integer + concurrentPartitionMovementsPerBroker: + description: The upper bound of ongoing partition replica movements going into/out of each broker. Default is 5. + minimum: 0 + type: integer + excludedTopics: + description: A regular expression where any matching topics will be excluded from the calculation of optimization proposals. This expression will be parsed by the java.util.regex.Pattern class; for more information on the supported format consult the documentation for that class. + type: string + goals: + description: A list of goals, ordered by decreasing priority, to use for generating and executing the rebalance proposal. The supported goals are available at https://github.com/linkedin/cruise-control#goals. If an empty goals list is provided, the goals declared in the default.goals Cruise Control configuration parameter are used. + items: + type: string + type: array + mode: + description: "Mode to run the rebalancing. The supported modes are `full`, `add-brokers`, `remove-brokers`.\nIf not specified, the `full` mode is used by default. \n\n* `full` mode runs the rebalancing across all the brokers in the cluster.\n* `add-brokers` mode can be used after scaling up the cluster to move some replicas to the newly added brokers.\n* `remove-brokers` mode can be used before scaling down the cluster to move replicas out of the brokers to be removed.\n* `remove-disks` mode can be used to move data across the volumes within the same broker\n." + enum: + - full + - add-brokers + - remove-brokers + - remove-disks + type: string + moveReplicasOffVolumes: + description: List of brokers and their corresponding volumes from which replicas need to be moved. + items: + properties: + brokerId: + description: ID of the broker that contains the disk from which you want to move the partition replicas. + type: integer + volumeIds: + description: IDs of the disks from which the partition replicas need to be moved. + items: + type: integer + minItems: 1 + type: array + type: object + minItems: 1 + type: array + rebalanceDisk: + description: Enables intra-broker disk balancing, which balances disk space utilization between disks on the same broker. Only applies to Kafka deployments that use JBOD storage with multiple disks. When enabled, inter-broker balancing is disabled. Default is false. + type: boolean + replicaMovementStrategies: + description: A list of strategy class names used to determine the execution order for the replica movements in the generated optimization proposal. By default BaseReplicaMovementStrategy is used, which will execute the replica movements in the order that they were generated. + items: + type: string + type: array + replicationThrottle: + description: The upper bound, in bytes per second, on the bandwidth used to move replicas. There is no limit by default. + minimum: 0 + type: integer + skipHardGoalCheck: + description: Whether to allow the hard goals specified in the Kafka CR to be skipped in optimization proposal generation. This can be useful when some of those hard goals are preventing a balance solution being found. Default is false. + type: boolean + type: object + status: + description: The status of the Kafka rebalance. + properties: + conditions: + description: List of status conditions. + items: + properties: + lastTransitionTime: + description: Last time the condition of a type changed from one status to another. The required format is 'yyyy-MM-ddTHH:mm:ssZ', in the UTC time zone. + type: string + message: + description: Human-readable message indicating details about the condition's last transition. + type: string + reason: + description: The reason for the condition's last transition (a single word in CamelCase). + type: string + status: + description: The status of the condition, either True, False or Unknown. + type: string + type: + description: The unique identifier of a condition, used to distinguish between other conditions in the resource. + type: string + type: object + type: array + observedGeneration: + description: The generation of the CRD that was last reconciled by the operator. + type: integer + optimizationResult: + description: A JSON object describing the optimization result. + type: object + x-kubernetes-preserve-unknown-fields: true + sessionId: + description: The session identifier for requests to Cruise Control pertaining to this KafkaRebalance resource. This is used by the Kafka Rebalance operator to track the status of ongoing rebalancing operations. + type: string + type: object + type: object + served: true + storage: true + subresources: + status: {} +--- +apiVersion: apiextensions.k8s.io/v1 +kind: CustomResourceDefinition +metadata: + labels: + app: strimzi + strimzi.io/crd-install: "true" + name: kafkatopics.kafka.strimzi.io +spec: + conversion: + strategy: None + group: kafka.strimzi.io + names: + categories: + - strimzi + kind: KafkaTopic + listKind: KafkaTopicList + plural: kafkatopics + shortNames: + - kt + singular: kafkatopic + scope: Namespaced + versions: + - additionalPrinterColumns: + - description: The name of the Kafka cluster this topic belongs to + jsonPath: .metadata.labels.strimzi\.io/cluster + name: Cluster + type: string + - description: The desired number of partitions in the topic + jsonPath: .spec.partitions + name: Partitions + type: integer + - description: The desired number of replicas of each partition + jsonPath: .spec.replicas + name: Replication factor + type: integer + - description: The state of the custom resource + jsonPath: .status.conditions[?(@.type=="Ready")].status + name: Ready + type: string + name: v1beta2 + schema: + openAPIV3Schema: + properties: + apiVersion: + description: 'APIVersion defines the versioned schema of this representation of an object. Servers should convert recognized schemas to the latest internal value, and may reject unrecognized values. More info: https://git.k8s.io/community/contributors/devel/sig-architecture/api-conventions.md#resources' + type: string + kind: + description: 'Kind is a string value representing the REST resource this object represents. Servers may infer this from the endpoint the client submits requests to. Cannot be updated. In CamelCase. More info: https://git.k8s.io/community/contributors/devel/sig-architecture/api-conventions.md#types-kinds' + type: string + metadata: + type: object + spec: + description: The specification of the topic. + properties: + config: + description: The topic configuration. + type: object + x-kubernetes-preserve-unknown-fields: true + partitions: + description: The number of partitions the topic should have. This cannot be decreased after topic creation. It can be increased after topic creation, but it is important to understand the consequences that has, especially for topics with semantic partitioning. When absent this will default to the broker configuration for `num.partitions`. + minimum: 1 + type: integer + replicas: + description: The number of replicas the topic should have. When absent this will default to the broker configuration for `default.replication.factor`. + maximum: 32767 + minimum: 1 + type: integer + topicName: + description: The name of the topic. When absent this will default to the metadata.name of the topic. It is recommended to not set this unless the topic name is not a valid Kubernetes resource name. + type: string + type: object + status: + description: The status of the topic. + properties: + conditions: + description: List of status conditions. + items: + properties: + lastTransitionTime: + description: Last time the condition of a type changed from one status to another. The required format is 'yyyy-MM-ddTHH:mm:ssZ', in the UTC time zone. + type: string + message: + description: Human-readable message indicating details about the condition's last transition. + type: string + reason: + description: The reason for the condition's last transition (a single word in CamelCase). + type: string + status: + description: The status of the condition, either True, False or Unknown. + type: string + type: + description: The unique identifier of a condition, used to distinguish between other conditions in the resource. + type: string + type: object + type: array + observedGeneration: + description: The generation of the CRD that was last reconciled by the operator. + type: integer + replicasChange: + description: Replication factor change status. + properties: + message: + description: Message for the user related to the replicas change request. This may contain transient error messages that would disappear on periodic reconciliations. + type: string + sessionId: + description: The session identifier for replicas change requests pertaining to this KafkaTopic resource. This is used by the Topic Operator to track the status of `ongoing` replicas change operations. + type: string + state: + description: Current state of the replicas change operation. This can be `pending`, when the change has been requested, or `ongoing`, when the change has been successfully submitted to Cruise Control. + enum: + - pending + - ongoing + type: string + targetReplicas: + description: The target replicas value requested by the user. This may be different from .spec.replicas when a change is ongoing. + type: integer + type: object + topicId: + description: The topic's id. For a KafkaTopic with the ready condition, this will change only if the topic gets deleted and recreated with the same name. + type: string + topicName: + description: Topic name. + type: string + type: object + type: object + served: true + storage: true + subresources: + status: {} + - additionalPrinterColumns: + - description: The name of the Kafka cluster this topic belongs to + jsonPath: .metadata.labels.strimzi\.io/cluster + name: Cluster + type: string + - description: The desired number of partitions in the topic + jsonPath: .spec.partitions + name: Partitions + type: integer + - description: The desired number of replicas of each partition + jsonPath: .spec.replicas + name: Replication factor + type: integer + - description: The state of the custom resource + jsonPath: .status.conditions[?(@.type=="Ready")].status + name: Ready + type: string + name: v1beta1 + schema: + openAPIV3Schema: + properties: + apiVersion: + description: 'APIVersion defines the versioned schema of this representation of an object. Servers should convert recognized schemas to the latest internal value, and may reject unrecognized values. More info: https://git.k8s.io/community/contributors/devel/sig-architecture/api-conventions.md#resources' + type: string + kind: + description: 'Kind is a string value representing the REST resource this object represents. Servers may infer this from the endpoint the client submits requests to. Cannot be updated. In CamelCase. More info: https://git.k8s.io/community/contributors/devel/sig-architecture/api-conventions.md#types-kinds' + type: string + metadata: + type: object + spec: + description: The specification of the topic. + properties: + config: + description: The topic configuration. + type: object + x-kubernetes-preserve-unknown-fields: true + partitions: + description: The number of partitions the topic should have. This cannot be decreased after topic creation. It can be increased after topic creation, but it is important to understand the consequences that has, especially for topics with semantic partitioning. When absent this will default to the broker configuration for `num.partitions`. + minimum: 1 + type: integer + replicas: + description: The number of replicas the topic should have. When absent this will default to the broker configuration for `default.replication.factor`. + maximum: 32767 + minimum: 1 + type: integer + topicName: + description: The name of the topic. When absent this will default to the metadata.name of the topic. It is recommended to not set this unless the topic name is not a valid Kubernetes resource name. + type: string + type: object + status: + description: The status of the topic. + properties: + conditions: + description: List of status conditions. + items: + properties: + lastTransitionTime: + description: Last time the condition of a type changed from one status to another. The required format is 'yyyy-MM-ddTHH:mm:ssZ', in the UTC time zone. + type: string + message: + description: Human-readable message indicating details about the condition's last transition. + type: string + reason: + description: The reason for the condition's last transition (a single word in CamelCase). + type: string + status: + description: The status of the condition, either True, False or Unknown. + type: string + type: + description: The unique identifier of a condition, used to distinguish between other conditions in the resource. + type: string + type: object + type: array + observedGeneration: + description: The generation of the CRD that was last reconciled by the operator. + type: integer + replicasChange: + description: Replication factor change status. + properties: + message: + description: Message for the user related to the replicas change request. This may contain transient error messages that would disappear on periodic reconciliations. + type: string + sessionId: + description: The session identifier for replicas change requests pertaining to this KafkaTopic resource. This is used by the Topic Operator to track the status of `ongoing` replicas change operations. + type: string + state: + description: Current state of the replicas change operation. This can be `pending`, when the change has been requested, or `ongoing`, when the change has been successfully submitted to Cruise Control. + enum: + - pending + - ongoing + type: string + targetReplicas: + description: The target replicas value requested by the user. This may be different from .spec.replicas when a change is ongoing. + type: integer + type: object + topicId: + description: The topic's id. For a KafkaTopic with the ready condition, this will change only if the topic gets deleted and recreated with the same name. + type: string + topicName: + description: Topic name. + type: string + type: object + type: object + served: true + storage: false + subresources: + status: {} + - additionalPrinterColumns: + - description: The name of the Kafka cluster this topic belongs to + jsonPath: .metadata.labels.strimzi\.io/cluster + name: Cluster + type: string + - description: The desired number of partitions in the topic + jsonPath: .spec.partitions + name: Partitions + type: integer + - description: The desired number of replicas of each partition + jsonPath: .spec.replicas + name: Replication factor + type: integer + - description: The state of the custom resource + jsonPath: .status.conditions[?(@.type=="Ready")].status + name: Ready + type: string + name: v1alpha1 + schema: + openAPIV3Schema: + properties: + apiVersion: + description: 'APIVersion defines the versioned schema of this representation of an object. Servers should convert recognized schemas to the latest internal value, and may reject unrecognized values. More info: https://git.k8s.io/community/contributors/devel/sig-architecture/api-conventions.md#resources' + type: string + kind: + description: 'Kind is a string value representing the REST resource this object represents. Servers may infer this from the endpoint the client submits requests to. Cannot be updated. In CamelCase. More info: https://git.k8s.io/community/contributors/devel/sig-architecture/api-conventions.md#types-kinds' + type: string + metadata: + type: object + spec: + description: The specification of the topic. + properties: + config: + description: The topic configuration. + type: object + x-kubernetes-preserve-unknown-fields: true + partitions: + description: The number of partitions the topic should have. This cannot be decreased after topic creation. It can be increased after topic creation, but it is important to understand the consequences that has, especially for topics with semantic partitioning. When absent this will default to the broker configuration for `num.partitions`. + minimum: 1 + type: integer + replicas: + description: The number of replicas the topic should have. When absent this will default to the broker configuration for `default.replication.factor`. + maximum: 32767 + minimum: 1 + type: integer + topicName: + description: The name of the topic. When absent this will default to the metadata.name of the topic. It is recommended to not set this unless the topic name is not a valid Kubernetes resource name. + type: string + type: object + status: + description: The status of the topic. + properties: + conditions: + description: List of status conditions. + items: + properties: + lastTransitionTime: + description: Last time the condition of a type changed from one status to another. The required format is 'yyyy-MM-ddTHH:mm:ssZ', in the UTC time zone. + type: string + message: + description: Human-readable message indicating details about the condition's last transition. + type: string + reason: + description: The reason for the condition's last transition (a single word in CamelCase). + type: string + status: + description: The status of the condition, either True, False or Unknown. + type: string + type: + description: The unique identifier of a condition, used to distinguish between other conditions in the resource. + type: string + type: object + type: array + observedGeneration: + description: The generation of the CRD that was last reconciled by the operator. + type: integer + replicasChange: + description: Replication factor change status. + properties: + message: + description: Message for the user related to the replicas change request. This may contain transient error messages that would disappear on periodic reconciliations. + type: string + sessionId: + description: The session identifier for replicas change requests pertaining to this KafkaTopic resource. This is used by the Topic Operator to track the status of `ongoing` replicas change operations. + type: string + state: + description: Current state of the replicas change operation. This can be `pending`, when the change has been requested, or `ongoing`, when the change has been successfully submitted to Cruise Control. + enum: + - pending + - ongoing + type: string + targetReplicas: + description: The target replicas value requested by the user. This may be different from .spec.replicas when a change is ongoing. + type: integer + type: object + topicId: + description: The topic's id. For a KafkaTopic with the ready condition, this will change only if the topic gets deleted and recreated with the same name. + type: string + topicName: + description: Topic name. + type: string + type: object + type: object + served: true + storage: false + subresources: + status: {} +--- +apiVersion: apiextensions.k8s.io/v1 +kind: CustomResourceDefinition +metadata: + labels: + app: strimzi + strimzi.io/crd-install: "true" + name: kafkausers.kafka.strimzi.io +spec: + conversion: + strategy: None + group: kafka.strimzi.io + names: + categories: + - strimzi + kind: KafkaUser + listKind: KafkaUserList + plural: kafkausers + shortNames: + - ku + singular: kafkauser + scope: Namespaced + versions: + - additionalPrinterColumns: + - description: The name of the Kafka cluster this user belongs to + jsonPath: .metadata.labels.strimzi\.io/cluster + name: Cluster + type: string + - description: How the user is authenticated + jsonPath: .spec.authentication.type + name: Authentication + type: string + - description: How the user is authorised + jsonPath: .spec.authorization.type + name: Authorization + type: string + - description: The state of the custom resource + jsonPath: .status.conditions[?(@.type=="Ready")].status + name: Ready + type: string + name: v1beta2 + schema: + openAPIV3Schema: + properties: + apiVersion: + description: 'APIVersion defines the versioned schema of this representation of an object. Servers should convert recognized schemas to the latest internal value, and may reject unrecognized values. More info: https://git.k8s.io/community/contributors/devel/sig-architecture/api-conventions.md#resources' + type: string + kind: + description: 'Kind is a string value representing the REST resource this object represents. Servers may infer this from the endpoint the client submits requests to. Cannot be updated. In CamelCase. More info: https://git.k8s.io/community/contributors/devel/sig-architecture/api-conventions.md#types-kinds' + type: string + metadata: + type: object + spec: + description: The specification of the user. + properties: + authentication: + description: "Authentication mechanism enabled for this Kafka user. The supported authentication mechanisms are `scram-sha-512`, `tls`, and `tls-external`. \n\n* `scram-sha-512` generates a secret with SASL SCRAM-SHA-512 credentials.\n* `tls` generates a secret with user certificate for mutual TLS authentication.\n* `tls-external` does not generate a user certificate. But prepares the user for using mutual TLS authentication using a user certificate generated outside the User Operator.\n ACLs and quotas set for this user are configured in the `CN=` format.\n\nAuthentication is optional. If authentication is not configured, no credentials are generated. ACLs and quotas set for the user are configured in the `` format suitable for SASL authentication." + properties: + password: + description: Specify the password for the user. If not set, a new password is generated by the User Operator. + properties: + valueFrom: + description: Secret from which the password should be read. + properties: + secretKeyRef: + description: Selects a key of a Secret in the resource's namespace. + properties: + key: + type: string + name: + type: string + optional: + type: boolean + type: object + type: object + required: + - valueFrom + type: object + type: + description: Authentication type. + enum: + - tls + - tls-external + - scram-sha-512 + type: string + required: + - type + type: object + authorization: + description: Authorization rules for this Kafka user. + properties: + acls: + description: List of ACL rules which should be applied to this user. + items: + properties: + host: + description: The host from which the action described in the ACL rule is allowed or denied. If not set, it defaults to `*`, allowing or denying the action from any host. + type: string + operation: + description: 'Operation which will be allowed or denied. Supported operations are: Read, Write, Create, Delete, Alter, Describe, ClusterAction, AlterConfigs, DescribeConfigs, IdempotentWrite and All.' + enum: + - Read + - Write + - Create + - Delete + - Alter + - Describe + - ClusterAction + - AlterConfigs + - DescribeConfigs + - IdempotentWrite + - All + type: string + operations: + description: 'List of operations to allow or deny. Supported operations are: Read, Write, Create, Delete, Alter, Describe, ClusterAction, AlterConfigs, DescribeConfigs, IdempotentWrite and All. Only certain operations work with the specified resource.' + items: + enum: + - Read + - Write + - Create + - Delete + - Alter + - Describe + - ClusterAction + - AlterConfigs + - DescribeConfigs + - IdempotentWrite + - All + type: string + type: array + resource: + description: Indicates the resource for which given ACL rule applies. + properties: + name: + description: Name of resource for which given ACL rule applies. Can be combined with `patternType` field to use prefix pattern. + type: string + patternType: + description: Describes the pattern used in the resource field. The supported types are `literal` and `prefix`. With `literal` pattern type, the resource field will be used as a definition of a full name. With `prefix` pattern type, the resource name will be used only as a prefix. Default value is `literal`. + enum: + - literal + - prefix + type: string + type: + description: Resource type. The available resource types are `topic`, `group`, `cluster`, and `transactionalId`. + enum: + - topic + - group + - cluster + - transactionalId + type: string + required: + - type + type: object + type: + description: The type of the rule. Currently the only supported type is `allow`. ACL rules with type `allow` are used to allow user to execute the specified operations. Default value is `allow`. + enum: + - allow + - deny + type: string + required: + - resource + type: object + type: array + type: + description: Authorization type. Currently the only supported type is `simple`. `simple` authorization type uses the Kafka Admin API for managing the ACL rules. + enum: + - simple + type: string + required: + - acls + - type + type: object + quotas: + description: Quotas on requests to control the broker resources used by clients. Network bandwidth and request rate quotas can be enforced.Kafka documentation for Kafka User quotas can be found at http://kafka.apache.org/documentation/#design_quotas. + properties: + consumerByteRate: + description: A quota on the maximum bytes per-second that each client group can fetch from a broker before the clients in the group are throttled. Defined on a per-broker basis. + minimum: 0 + type: integer + controllerMutationRate: + description: A quota on the rate at which mutations are accepted for the create topics request, the create partitions request and the delete topics request. The rate is accumulated by the number of partitions created or deleted. + minimum: 0 + type: number + producerByteRate: + description: A quota on the maximum bytes per-second that each client group can publish to a broker before the clients in the group are throttled. Defined on a per-broker basis. + minimum: 0 + type: integer + requestPercentage: + description: A quota on the maximum CPU utilization of each client group as a percentage of network and I/O threads. + minimum: 0 + type: integer + type: object + template: + description: Template to specify how Kafka User `Secrets` are generated. + properties: + secret: + description: Template for KafkaUser resources. The template allows users to specify how the `Secret` with password or TLS certificates is generated. + properties: + metadata: + description: Metadata applied to the resource. + properties: + annotations: + additionalProperties: + type: string + description: Annotations added to the Kubernetes resource. + type: object + labels: + additionalProperties: + type: string + description: Labels added to the Kubernetes resource. + type: object + type: object + type: object + type: object + type: object + status: + description: The status of the Kafka User. + properties: + conditions: + description: List of status conditions. + items: + properties: + lastTransitionTime: + description: Last time the condition of a type changed from one status to another. The required format is 'yyyy-MM-ddTHH:mm:ssZ', in the UTC time zone. + type: string + message: + description: Human-readable message indicating details about the condition's last transition. + type: string + reason: + description: The reason for the condition's last transition (a single word in CamelCase). + type: string + status: + description: The status of the condition, either True, False or Unknown. + type: string + type: + description: The unique identifier of a condition, used to distinguish between other conditions in the resource. + type: string + type: object + type: array + observedGeneration: + description: The generation of the CRD that was last reconciled by the operator. + type: integer + secret: + description: The name of `Secret` where the credentials are stored. + type: string + username: + description: Username. + type: string + type: object + type: object + served: true + storage: true + subresources: + status: {} + - additionalPrinterColumns: + - description: The name of the Kafka cluster this user belongs to + jsonPath: .metadata.labels.strimzi\.io/cluster + name: Cluster + type: string + - description: How the user is authenticated + jsonPath: .spec.authentication.type + name: Authentication + type: string + - description: How the user is authorised + jsonPath: .spec.authorization.type + name: Authorization + type: string + - description: The state of the custom resource + jsonPath: .status.conditions[?(@.type=="Ready")].status + name: Ready + type: string + name: v1beta1 + schema: + openAPIV3Schema: + properties: + apiVersion: + description: 'APIVersion defines the versioned schema of this representation of an object. Servers should convert recognized schemas to the latest internal value, and may reject unrecognized values. More info: https://git.k8s.io/community/contributors/devel/sig-architecture/api-conventions.md#resources' + type: string + kind: + description: 'Kind is a string value representing the REST resource this object represents. Servers may infer this from the endpoint the client submits requests to. Cannot be updated. In CamelCase. More info: https://git.k8s.io/community/contributors/devel/sig-architecture/api-conventions.md#types-kinds' + type: string + metadata: + type: object + spec: + description: The specification of the user. + properties: + authentication: + description: "Authentication mechanism enabled for this Kafka user. The supported authentication mechanisms are `scram-sha-512`, `tls`, and `tls-external`. \n\n* `scram-sha-512` generates a secret with SASL SCRAM-SHA-512 credentials.\n* `tls` generates a secret with user certificate for mutual TLS authentication.\n* `tls-external` does not generate a user certificate. But prepares the user for using mutual TLS authentication using a user certificate generated outside the User Operator.\n ACLs and quotas set for this user are configured in the `CN=` format.\n\nAuthentication is optional. If authentication is not configured, no credentials are generated. ACLs and quotas set for the user are configured in the `` format suitable for SASL authentication." + properties: + password: + description: Specify the password for the user. If not set, a new password is generated by the User Operator. + properties: + valueFrom: + description: Secret from which the password should be read. + properties: + secretKeyRef: + description: Selects a key of a Secret in the resource's namespace. + properties: + key: + type: string + name: + type: string + optional: + type: boolean + type: object + type: object + required: + - valueFrom + type: object + type: + description: Authentication type. + enum: + - tls + - tls-external + - scram-sha-512 + type: string + required: + - type + type: object + authorization: + description: Authorization rules for this Kafka user. + properties: + acls: + description: List of ACL rules which should be applied to this user. + items: + properties: + host: + description: The host from which the action described in the ACL rule is allowed or denied. If not set, it defaults to `*`, allowing or denying the action from any host. + type: string + operation: + description: 'Operation which will be allowed or denied. Supported operations are: Read, Write, Create, Delete, Alter, Describe, ClusterAction, AlterConfigs, DescribeConfigs, IdempotentWrite and All.' + enum: + - Read + - Write + - Create + - Delete + - Alter + - Describe + - ClusterAction + - AlterConfigs + - DescribeConfigs + - IdempotentWrite + - All + type: string + operations: + description: 'List of operations to allow or deny. Supported operations are: Read, Write, Create, Delete, Alter, Describe, ClusterAction, AlterConfigs, DescribeConfigs, IdempotentWrite and All. Only certain operations work with the specified resource.' + items: + enum: + - Read + - Write + - Create + - Delete + - Alter + - Describe + - ClusterAction + - AlterConfigs + - DescribeConfigs + - IdempotentWrite + - All + type: string + type: array + resource: + description: Indicates the resource for which given ACL rule applies. + properties: + name: + description: Name of resource for which given ACL rule applies. Can be combined with `patternType` field to use prefix pattern. + type: string + patternType: + description: Describes the pattern used in the resource field. The supported types are `literal` and `prefix`. With `literal` pattern type, the resource field will be used as a definition of a full name. With `prefix` pattern type, the resource name will be used only as a prefix. Default value is `literal`. + enum: + - literal + - prefix + type: string + type: + description: Resource type. The available resource types are `topic`, `group`, `cluster`, and `transactionalId`. + enum: + - topic + - group + - cluster + - transactionalId + type: string + required: + - type + type: object + type: + description: The type of the rule. Currently the only supported type is `allow`. ACL rules with type `allow` are used to allow user to execute the specified operations. Default value is `allow`. + enum: + - allow + - deny + type: string + required: + - resource + type: object + type: array + type: + description: Authorization type. Currently the only supported type is `simple`. `simple` authorization type uses the Kafka Admin API for managing the ACL rules. + enum: + - simple + type: string + required: + - acls + - type + type: object + quotas: + description: Quotas on requests to control the broker resources used by clients. Network bandwidth and request rate quotas can be enforced.Kafka documentation for Kafka User quotas can be found at http://kafka.apache.org/documentation/#design_quotas. + properties: + consumerByteRate: + description: A quota on the maximum bytes per-second that each client group can fetch from a broker before the clients in the group are throttled. Defined on a per-broker basis. + minimum: 0 + type: integer + controllerMutationRate: + description: A quota on the rate at which mutations are accepted for the create topics request, the create partitions request and the delete topics request. The rate is accumulated by the number of partitions created or deleted. + minimum: 0 + type: number + producerByteRate: + description: A quota on the maximum bytes per-second that each client group can publish to a broker before the clients in the group are throttled. Defined on a per-broker basis. + minimum: 0 + type: integer + requestPercentage: + description: A quota on the maximum CPU utilization of each client group as a percentage of network and I/O threads. + minimum: 0 + type: integer + type: object + template: + description: Template to specify how Kafka User `Secrets` are generated. + properties: + secret: + description: Template for KafkaUser resources. The template allows users to specify how the `Secret` with password or TLS certificates is generated. + properties: + metadata: + description: Metadata applied to the resource. + properties: + annotations: + additionalProperties: + type: string + description: Annotations added to the Kubernetes resource. + type: object + labels: + additionalProperties: + type: string + description: Labels added to the Kubernetes resource. + type: object + type: object + type: object + type: object + type: object + status: + description: The status of the Kafka User. + properties: + conditions: + description: List of status conditions. + items: + properties: + lastTransitionTime: + description: Last time the condition of a type changed from one status to another. The required format is 'yyyy-MM-ddTHH:mm:ssZ', in the UTC time zone. + type: string + message: + description: Human-readable message indicating details about the condition's last transition. + type: string + reason: + description: The reason for the condition's last transition (a single word in CamelCase). + type: string + status: + description: The status of the condition, either True, False or Unknown. + type: string + type: + description: The unique identifier of a condition, used to distinguish between other conditions in the resource. + type: string + type: object + type: array + observedGeneration: + description: The generation of the CRD that was last reconciled by the operator. + type: integer + secret: + description: The name of `Secret` where the credentials are stored. + type: string + username: + description: Username. + type: string + type: object + type: object + served: true + storage: false + subresources: + status: {} + - additionalPrinterColumns: + - description: The name of the Kafka cluster this user belongs to + jsonPath: .metadata.labels.strimzi\.io/cluster + name: Cluster + type: string + - description: How the user is authenticated + jsonPath: .spec.authentication.type + name: Authentication + type: string + - description: How the user is authorised + jsonPath: .spec.authorization.type + name: Authorization + type: string + - description: The state of the custom resource + jsonPath: .status.conditions[?(@.type=="Ready")].status + name: Ready + type: string + name: v1alpha1 + schema: + openAPIV3Schema: + properties: + apiVersion: + description: 'APIVersion defines the versioned schema of this representation of an object. Servers should convert recognized schemas to the latest internal value, and may reject unrecognized values. More info: https://git.k8s.io/community/contributors/devel/sig-architecture/api-conventions.md#resources' + type: string + kind: + description: 'Kind is a string value representing the REST resource this object represents. Servers may infer this from the endpoint the client submits requests to. Cannot be updated. In CamelCase. More info: https://git.k8s.io/community/contributors/devel/sig-architecture/api-conventions.md#types-kinds' + type: string + metadata: + type: object + spec: + description: The specification of the user. + properties: + authentication: + description: "Authentication mechanism enabled for this Kafka user. The supported authentication mechanisms are `scram-sha-512`, `tls`, and `tls-external`. \n\n* `scram-sha-512` generates a secret with SASL SCRAM-SHA-512 credentials.\n* `tls` generates a secret with user certificate for mutual TLS authentication.\n* `tls-external` does not generate a user certificate. But prepares the user for using mutual TLS authentication using a user certificate generated outside the User Operator.\n ACLs and quotas set for this user are configured in the `CN=` format.\n\nAuthentication is optional. If authentication is not configured, no credentials are generated. ACLs and quotas set for the user are configured in the `` format suitable for SASL authentication." + properties: + password: + description: Specify the password for the user. If not set, a new password is generated by the User Operator. + properties: + valueFrom: + description: Secret from which the password should be read. + properties: + secretKeyRef: + description: Selects a key of a Secret in the resource's namespace. + properties: + key: + type: string + name: + type: string + optional: + type: boolean + type: object + type: object + required: + - valueFrom + type: object + type: + description: Authentication type. + enum: + - tls + - tls-external + - scram-sha-512 + type: string + required: + - type + type: object + authorization: + description: Authorization rules for this Kafka user. + properties: + acls: + description: List of ACL rules which should be applied to this user. + items: + properties: + host: + description: The host from which the action described in the ACL rule is allowed or denied. If not set, it defaults to `*`, allowing or denying the action from any host. + type: string + operation: + description: 'Operation which will be allowed or denied. Supported operations are: Read, Write, Create, Delete, Alter, Describe, ClusterAction, AlterConfigs, DescribeConfigs, IdempotentWrite and All.' + enum: + - Read + - Write + - Create + - Delete + - Alter + - Describe + - ClusterAction + - AlterConfigs + - DescribeConfigs + - IdempotentWrite + - All + type: string + operations: + description: 'List of operations to allow or deny. Supported operations are: Read, Write, Create, Delete, Alter, Describe, ClusterAction, AlterConfigs, DescribeConfigs, IdempotentWrite and All. Only certain operations work with the specified resource.' + items: + enum: + - Read + - Write + - Create + - Delete + - Alter + - Describe + - ClusterAction + - AlterConfigs + - DescribeConfigs + - IdempotentWrite + - All + type: string + type: array + resource: + description: Indicates the resource for which given ACL rule applies. + properties: + name: + description: Name of resource for which given ACL rule applies. Can be combined with `patternType` field to use prefix pattern. + type: string + patternType: + description: Describes the pattern used in the resource field. The supported types are `literal` and `prefix`. With `literal` pattern type, the resource field will be used as a definition of a full name. With `prefix` pattern type, the resource name will be used only as a prefix. Default value is `literal`. + enum: + - literal + - prefix + type: string + type: + description: Resource type. The available resource types are `topic`, `group`, `cluster`, and `transactionalId`. + enum: + - topic + - group + - cluster + - transactionalId + type: string + required: + - type + type: object + type: + description: The type of the rule. Currently the only supported type is `allow`. ACL rules with type `allow` are used to allow user to execute the specified operations. Default value is `allow`. + enum: + - allow + - deny + type: string + required: + - resource + type: object + type: array + type: + description: Authorization type. Currently the only supported type is `simple`. `simple` authorization type uses the Kafka Admin API for managing the ACL rules. + enum: + - simple + type: string + required: + - acls + - type + type: object + quotas: + description: Quotas on requests to control the broker resources used by clients. Network bandwidth and request rate quotas can be enforced.Kafka documentation for Kafka User quotas can be found at http://kafka.apache.org/documentation/#design_quotas. + properties: + consumerByteRate: + description: A quota on the maximum bytes per-second that each client group can fetch from a broker before the clients in the group are throttled. Defined on a per-broker basis. + minimum: 0 + type: integer + controllerMutationRate: + description: A quota on the rate at which mutations are accepted for the create topics request, the create partitions request and the delete topics request. The rate is accumulated by the number of partitions created or deleted. + minimum: 0 + type: number + producerByteRate: + description: A quota on the maximum bytes per-second that each client group can publish to a broker before the clients in the group are throttled. Defined on a per-broker basis. + minimum: 0 + type: integer + requestPercentage: + description: A quota on the maximum CPU utilization of each client group as a percentage of network and I/O threads. + minimum: 0 + type: integer + type: object + template: + description: Template to specify how Kafka User `Secrets` are generated. + properties: + secret: + description: Template for KafkaUser resources. The template allows users to specify how the `Secret` with password or TLS certificates is generated. + properties: + metadata: + description: Metadata applied to the resource. + properties: + annotations: + additionalProperties: + type: string + description: Annotations added to the Kubernetes resource. + type: object + labels: + additionalProperties: + type: string + description: Labels added to the Kubernetes resource. + type: object + type: object + type: object + type: object + type: object + status: + description: The status of the Kafka User. + properties: + conditions: + description: List of status conditions. + items: + properties: + lastTransitionTime: + description: Last time the condition of a type changed from one status to another. The required format is 'yyyy-MM-ddTHH:mm:ssZ', in the UTC time zone. + type: string + message: + description: Human-readable message indicating details about the condition's last transition. + type: string + reason: + description: The reason for the condition's last transition (a single word in CamelCase). + type: string + status: + description: The status of the condition, either True, False or Unknown. + type: string + type: + description: The unique identifier of a condition, used to distinguish between other conditions in the resource. + type: string + type: object + type: array + observedGeneration: + description: The generation of the CRD that was last reconciled by the operator. + type: integer + secret: + description: The name of `Secret` where the credentials are stored. + type: string + username: + description: Username. + type: string + type: object + type: object + served: true + storage: false + subresources: + status: {} From 97f134de695a3ab6055ec55933c111b31459f3b9 Mon Sep 17 00:00:00 2001 From: Adam Czajkowski Date: Fri, 14 Feb 2025 12:04:26 +0100 Subject: [PATCH 52/76] Copy OpenTelemetry collector deployment from https://github.com/GoogleCloudPlatform/otlp-k8s-ingest/tree/3b383a083250487e5ca27e58abf0d81cb25a914c/k8s/base --- .../kubernetes/otlp/0_namespace.yaml | 18 ++ .../kubernetes/otlp/1_configmap.yaml | 179 ++++++++++++++++++ .../performance/kubernetes/otlp/2_rbac.yaml | 57 ++++++ .../kubernetes/otlp/3_service.yaml | 35 ++++ .../kubernetes/otlp/4_deployment.yaml | 74 ++++++++ .../performance/kubernetes/otlp/5_hpa.yaml | 39 ++++ .../kubernetes/otlp/kustomization.yml | 23 +++ 7 files changed, 425 insertions(+) create mode 100644 google-cloud-bigtable-kafka-connect-sink/performance/kubernetes/otlp/0_namespace.yaml create mode 100644 google-cloud-bigtable-kafka-connect-sink/performance/kubernetes/otlp/1_configmap.yaml create mode 100644 google-cloud-bigtable-kafka-connect-sink/performance/kubernetes/otlp/2_rbac.yaml create mode 100644 google-cloud-bigtable-kafka-connect-sink/performance/kubernetes/otlp/3_service.yaml create mode 100644 google-cloud-bigtable-kafka-connect-sink/performance/kubernetes/otlp/4_deployment.yaml create mode 100644 google-cloud-bigtable-kafka-connect-sink/performance/kubernetes/otlp/5_hpa.yaml create mode 100644 google-cloud-bigtable-kafka-connect-sink/performance/kubernetes/otlp/kustomization.yml diff --git a/google-cloud-bigtable-kafka-connect-sink/performance/kubernetes/otlp/0_namespace.yaml b/google-cloud-bigtable-kafka-connect-sink/performance/kubernetes/otlp/0_namespace.yaml new file mode 100644 index 0000000000..43f7f78b29 --- /dev/null +++ b/google-cloud-bigtable-kafka-connect-sink/performance/kubernetes/otlp/0_namespace.yaml @@ -0,0 +1,18 @@ +# Copyright 2024 Google LLC +# +# Licensed under the Apache License, Version 2.0 (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.apache.org/licenses/LICENSE-2.0 +# +# Unless required by applicable law or agreed to in writing, software +# distributed under the License is distributed on an "AS IS" BASIS, +# WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. +# See the License for the specific language governing permissions and +# limitations under the License. + +apiVersion: v1 +kind: Namespace +metadata: + name: opentelemetry diff --git a/google-cloud-bigtable-kafka-connect-sink/performance/kubernetes/otlp/1_configmap.yaml b/google-cloud-bigtable-kafka-connect-sink/performance/kubernetes/otlp/1_configmap.yaml new file mode 100644 index 0000000000..b8168a90c1 --- /dev/null +++ b/google-cloud-bigtable-kafka-connect-sink/performance/kubernetes/otlp/1_configmap.yaml @@ -0,0 +1,179 @@ +apiVersion: v1 +data: + collector.yaml: | + # Copyright 2024 Google LLC + # + # Licensed under the Apache License, Version 2.0 (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.apache.org/licenses/LICENSE-2.0 + # + # Unless required by applicable law or agreed to in writing, software + # distributed under the License is distributed on an "AS IS" BASIS, + # WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + # See the License for the specific language governing permissions and + # limitations under the License. + + exporters: + googlecloud: + log: + default_log_name: opentelemetry-collector + user_agent: Google-Cloud-OTLP manifests:0.1.0 otel/opentelemetry-collector-contrib:0.118.0 + googlemanagedprometheus: + user_agent: Google-Cloud-OTLP manifests:0.1.0 otel/opentelemetry-collector-contrib:0.118.0 + + extensions: + health_check: + endpoint: ${env:MY_POD_IP}:13133 + processors: + filter/self-metrics: + metrics: + include: + match_type: strict + metric_names: + - otelcol_process_uptime + - otelcol_process_memory_rss + - otelcol_grpc_io_client_completed_rpcs + - otelcol_googlecloudmonitoring_point_count + batch: + send_batch_max_size: 200 + send_batch_size: 200 + timeout: 5s + + k8sattributes: + extract: + metadata: + - k8s.namespace.name + - k8s.deployment.name + - k8s.statefulset.name + - k8s.daemonset.name + - k8s.cronjob.name + - k8s.job.name + - k8s.node.name + - k8s.pod.name + - k8s.pod.uid + - k8s.pod.start_time + passthrough: false + pod_association: + - sources: + - from: resource_attribute + name: k8s.pod.ip + - sources: + - from: resource_attribute + name: k8s.pod.uid + - sources: + - from: connection + memory_limiter: + check_interval: 1s + limit_percentage: 65 + spike_limit_percentage: 20 + + metricstransform/self-metrics: + transforms: + - action: update + include: otelcol_process_uptime + operations: + - action: add_label + new_label: version + new_value: Google-Cloud-OTLP manifests:0.1.0 otel/opentelemetry-collector-contrib:0.118.0 + + resourcedetection: + detectors: [gcp] + timeout: 10s + + transform/collision: + metric_statements: + - context: datapoint + statements: + - set(attributes["exported_location"], attributes["location"]) + - delete_key(attributes, "location") + - set(attributes["exported_cluster"], attributes["cluster"]) + - delete_key(attributes, "cluster") + - set(attributes["exported_namespace"], attributes["namespace"]) + - delete_key(attributes, "namespace") + - set(attributes["exported_job"], attributes["job"]) + - delete_key(attributes, "job") + - set(attributes["exported_instance"], attributes["instance"]) + - delete_key(attributes, "instance") + - set(attributes["exported_project_id"], attributes["project_id"]) + - delete_key(attributes, "project_id") + + receivers: + otlp: + protocols: + grpc: + endpoint: ${env:MY_POD_IP}:4317 + http: + cors: + allowed_origins: + - http://* + - https://* + endpoint: ${env:MY_POD_IP}:4318 + otlp/self-metrics: + protocols: + grpc: + endpoint: ${env:MY_POD_IP}:14317 + + service: + extensions: + - health_check + pipelines: + logs: + exporters: + - googlecloud + processors: + - k8sattributes + - resourcedetection + - memory_limiter + - batch + receivers: + - otlp + metrics/otlp: + exporters: + - googlemanagedprometheus + processors: + - k8sattributes + - memory_limiter + - resourcedetection + - transform/collision + - batch + receivers: + - otlp + metrics/self-metrics: + exporters: + - googlemanagedprometheus + processors: + - filter/self-metrics + - metricstransform/self-metrics + - k8sattributes + - memory_limiter + - resourcedetection + - batch + receivers: + - otlp/self-metrics + traces: + exporters: + - googlecloud + processors: + - k8sattributes + - memory_limiter + - resourcedetection + - batch + receivers: + - otlp + telemetry: + logs: + encoding: json + metrics: + readers: + - periodic: + exporter: + otlp: + protocol: grpc/protobuf + endpoint: ${env:MY_POD_IP}:14317 +kind: ConfigMap +metadata: + creationTimestamp: null + name: collector-config + namespace: opentelemetry diff --git a/google-cloud-bigtable-kafka-connect-sink/performance/kubernetes/otlp/2_rbac.yaml b/google-cloud-bigtable-kafka-connect-sink/performance/kubernetes/otlp/2_rbac.yaml new file mode 100644 index 0000000000..6fae230d73 --- /dev/null +++ b/google-cloud-bigtable-kafka-connect-sink/performance/kubernetes/otlp/2_rbac.yaml @@ -0,0 +1,57 @@ +# Copyright 2024 Google LLC +# +# Licensed under the Apache License, Version 2.0 (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.apache.org/licenses/LICENSE-2.0 +# +# Unless required by applicable law or agreed to in writing, software +# distributed under the License is distributed on an "AS IS" BASIS, +# WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. +# See the License for the specific language governing permissions and +# limitations under the License. + +apiVersion: v1 +kind: ServiceAccount +metadata: + name: opentelemetry-collector + namespace: opentelemetry + labels: + app.kubernetes.io/name: opentelemetry-collector + app.kubernetes.io/version: "0.118.0" +--- +apiVersion: rbac.authorization.k8s.io/v1 +kind: ClusterRole +metadata: + name: opentelemetry-collector + namespace: opentelemetry + labels: + app.kubernetes.io/name: opentelemetry-collector + app.kubernetes.io/version: "0.118.0" +rules: + - apiGroups: [""] + resources: ["pods", "namespaces", "nodes"] + verbs: ["get", "watch", "list"] + - apiGroups: ["apps"] + resources: ["replicasets"] + verbs: ["get", "list", "watch"] + - apiGroups: ["extensions"] + resources: ["replicasets"] + verbs: ["get", "list", "watch"] +--- +apiVersion: rbac.authorization.k8s.io/v1 +kind: ClusterRoleBinding +metadata: + name: opentelemetry-collector + labels: + app.kubernetes.io/name: opentelemetry-collector + app.kubernetes.io/version: "0.118.0" +roleRef: + apiGroup: rbac.authorization.k8s.io + kind: ClusterRole + name: opentelemetry-collector +subjects: +- kind: ServiceAccount + name: opentelemetry-collector + namespace: opentelemetry diff --git a/google-cloud-bigtable-kafka-connect-sink/performance/kubernetes/otlp/3_service.yaml b/google-cloud-bigtable-kafka-connect-sink/performance/kubernetes/otlp/3_service.yaml new file mode 100644 index 0000000000..db5b668be8 --- /dev/null +++ b/google-cloud-bigtable-kafka-connect-sink/performance/kubernetes/otlp/3_service.yaml @@ -0,0 +1,35 @@ +# Copyright 2024 Google LLC +# +# Licensed under the Apache License, Version 2.0 (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.apache.org/licenses/LICENSE-2.0 +# +# Unless required by applicable law or agreed to in writing, software +# distributed under the License is distributed on an "AS IS" BASIS, +# WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. +# See the License for the specific language governing permissions and +# limitations under the License. + +apiVersion: v1 +kind: Service +metadata: + name: opentelemetry-collector + namespace: opentelemetry + labels: + app: opentelemetry-collector +spec: + type: ClusterIP + selector: + app: opentelemetry-collector + internalTrafficPolicy: Cluster + ports: + - name: otel-grpc + protocol: TCP + port: 4317 + targetPort: 4317 + - name: otlp-http + port: 4318 + targetPort: 4318 + protocol: TCP diff --git a/google-cloud-bigtable-kafka-connect-sink/performance/kubernetes/otlp/4_deployment.yaml b/google-cloud-bigtable-kafka-connect-sink/performance/kubernetes/otlp/4_deployment.yaml new file mode 100644 index 0000000000..dbec84a661 --- /dev/null +++ b/google-cloud-bigtable-kafka-connect-sink/performance/kubernetes/otlp/4_deployment.yaml @@ -0,0 +1,74 @@ +# Copyright 2024 Google LLC +# +# Licensed under the Apache License, Version 2.0 (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.apache.org/licenses/LICENSE-2.0 +# +# Unless required by applicable law or agreed to in writing, software +# distributed under the License is distributed on an "AS IS" BASIS, +# WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. +# See the License for the specific language governing permissions and +# limitations under the License. + +apiVersion: apps/v1 +kind: Deployment +metadata: + name: opentelemetry-collector + namespace: opentelemetry + labels: + app: opentelemetry-collector +spec: + replicas: 2 + selector: + matchLabels: + app: opentelemetry-collector + template: + metadata: + labels: + app: opentelemetry-collector + spec: + serviceAccountName: opentelemetry-collector + securityContext: + {} + containers: + - name: opentelemetry-collector + imagePullPolicy: Always + image: otel/opentelemetry-collector-contrib:0.118.0 + args: + - "--config=/conf/collector.yaml" + - "--feature-gates=exporter.googlemanagedprometheus.intToDouble" + ports: + - name: otlp-grpc + containerPort: 4317 + protocol: TCP + - name: otlp-http + containerPort: 4318 + protocol: TCP + env: + - name: MY_POD_IP + valueFrom: + fieldRef: + apiVersion: v1 + fieldPath: status.podIP + resources: + requests: + memory: "200Mi" + cpu: "200m" + limits: + memory: "400Mi" + volumeMounts: + - name: collector-config + mountPath: /conf + readinessProbe: + httpGet: + path: / + port: 13133 + volumes: + - name: collector-config + configMap: + name: collector-config + items: + - key: collector.yaml + path: collector.yaml diff --git a/google-cloud-bigtable-kafka-connect-sink/performance/kubernetes/otlp/5_hpa.yaml b/google-cloud-bigtable-kafka-connect-sink/performance/kubernetes/otlp/5_hpa.yaml new file mode 100644 index 0000000000..2aede1fd71 --- /dev/null +++ b/google-cloud-bigtable-kafka-connect-sink/performance/kubernetes/otlp/5_hpa.yaml @@ -0,0 +1,39 @@ +# Copyright 2024 Google LLC +# +# Licensed under the Apache License, Version 2.0 (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.apache.org/licenses/LICENSE-2.0 +# +# Unless required by applicable law or agreed to in writing, software +# distributed under the License is distributed on an "AS IS" BASIS, +# WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. +# See the License for the specific language governing permissions and +# limitations under the License. + +apiVersion: autoscaling/v2 +kind: HorizontalPodAutoscaler +metadata: + name: opentelemetry-collector + namespace: opentelemetry +spec: + scaleTargetRef: + apiVersion: apps/v1 + kind: Deployment + name: opentelemetry-collector + minReplicas: 1 + maxReplicas: 10 + metrics: + - type: Resource + resource: + name: memory + target: + type: Utilization + averageUtilization: 80 + - type: Resource + resource: + name: cpu + target: + type: Utilization + averageUtilization: 80 diff --git a/google-cloud-bigtable-kafka-connect-sink/performance/kubernetes/otlp/kustomization.yml b/google-cloud-bigtable-kafka-connect-sink/performance/kubernetes/otlp/kustomization.yml new file mode 100644 index 0000000000..e607c0575c --- /dev/null +++ b/google-cloud-bigtable-kafka-connect-sink/performance/kubernetes/otlp/kustomization.yml @@ -0,0 +1,23 @@ +# Copyright 2024 Google LLC +# +# Licensed under the Apache License, Version 2.0 (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.apache.org/licenses/LICENSE-2.0 +# +# Unless required by applicable law or agreed to in writing, software +# distributed under the License is distributed on an "AS IS" BASIS, +# WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. +# See the License for the specific language governing permissions and +# limitations under the License. + +apiVersion: kustomize.config.k8s.io/v1beta1 +kind: Kustomization +resources: + - 0_namespace.yaml + - 1_configmap.yaml + - 2_rbac.yaml + - 3_service.yaml + - 4_deployment.yaml + - 5_hpa.yaml From c0701369bd44c4da1fad7be0b863457af59431ad Mon Sep 17 00:00:00 2001 From: Adam Czajkowski Date: Tue, 25 Feb 2025 13:41:32 +0100 Subject: [PATCH 53/76] Copy Strimzi Kafka Connect metrics config from https://github.com/strimzi/strimzi-kafka-operator/blob/release-0.45.x/examples/metrics/kafka-connect-metrics.yaml --- .../kafka-connect-metrics-config.yaml | 183 ++++++++++++++++++ 1 file changed, 183 insertions(+) create mode 100644 google-cloud-bigtable-kafka-connect-sink/performance/kubernetes/kafka-connect-metrics-config.yaml diff --git a/google-cloud-bigtable-kafka-connect-sink/performance/kubernetes/kafka-connect-metrics-config.yaml b/google-cloud-bigtable-kafka-connect-sink/performance/kubernetes/kafka-connect-metrics-config.yaml new file mode 100644 index 0000000000..ebe1967faa --- /dev/null +++ b/google-cloud-bigtable-kafka-connect-sink/performance/kubernetes/kafka-connect-metrics-config.yaml @@ -0,0 +1,183 @@ +kind: ConfigMap +apiVersion: v1 +metadata: + name: connect-metrics + labels: + app: strimzi +data: + metrics-config.yml: | + # Inspired by kafka-connect rules + # https://github.com/prometheus/jmx_exporter/blob/master/example_configs/kafka-connect.yml + # See https://github.com/prometheus/jmx_exporter for more info about JMX Prometheus Exporter metrics + lowercaseOutputName: true + lowercaseOutputLabelNames: true + rules: + #kafka.connect:type=app-info,client-id="{clientid}" + #kafka.consumer:type=app-info,client-id="{clientid}" + #kafka.producer:type=app-info,client-id="{clientid}" + - pattern: 'kafka.(.+)<>start-time-ms' + name: kafka_$1_start_time_seconds + labels: + clientId: "$2" + help: "Kafka $1 JMX metric start time seconds" + type: GAUGE + valueFactor: 0.001 + - pattern: 'kafka.(.+)<>(commit-id|version): (.+)' + name: kafka_$1_$3_info + value: 1 + labels: + clientId: "$2" + $3: "$4" + help: "Kafka $1 JMX metric info version and commit-id" + type: UNTYPED + + #kafka.consumer:type=consumer-fetch-manager-metrics,client-id="{clientid}",topic="{topic}"", partition="{partition}" + - pattern: kafka.consumer<>(.+-total) + name: kafka_consumer_fetch_manager_$4 + labels: + clientId: "$1" + topic: "$2" + partition: "$3" + help: "Kafka Consumer JMX metric type consumer-fetch-manager-metrics" + type: COUNTER + - pattern: kafka.consumer<>(compression-rate|.+-avg|.+-replica|.+-lag|.+-lead) + name: kafka_consumer_fetch_manager_$4 + labels: + clientId: "$1" + topic: "$2" + partition: "$3" + help: "Kafka Consumer JMX metric type consumer-fetch-manager-metrics" + type: GAUGE + + #kafka.producer:type=producer-topic-metrics,client-id="{clientid}",topic="{topic}" + - pattern: kafka.producer<>(.+-total) + name: kafka_producer_topic_$3 + labels: + clientId: "$1" + topic: "$2" + help: "Kafka Producer JMX metric type producer-topic-metrics" + type: COUNTER + - pattern: kafka.producer<>(compression-rate|.+-avg|.+rate) + name: kafka_producer_topic_$3 + labels: + clientId: "$1" + topic: "$2" + help: "Kafka Producer JMX metric type producer-topic-metrics" + type: GAUGE + + #kafka.connect:type=connect-node-metrics,client-id="{clientid}",node-id="{nodeid}" + #kafka.consumer:type=consumer-node-metrics,client-id=consumer-1,node-id="{nodeid}" + - pattern: kafka.(.+)<>(.+-total) + name: kafka_$2_$5 + labels: + clientId: "$3" + nodeId: "$4" + help: "Kafka $1 JMX metric type $2" + type: COUNTER + - pattern: kafka.(.+)<>(.+-avg|.+-rate) + name: kafka_$2_$5 + labels: + clientId: "$3" + nodeId: "$4" + help: "Kafka $1 JMX metric type $2" + type: GAUGE + + #kafka.connect:type=kafka-metrics-count,client-id="{clientid}" + #kafka.consumer:type=consumer-fetch-manager-metrics,client-id="{clientid}" + #kafka.consumer:type=consumer-coordinator-metrics,client-id="{clientid}" + #kafka.consumer:type=consumer-metrics,client-id="{clientid}" + - pattern: kafka.(.+)<>(.+-total) + name: kafka_$2_$4 + labels: + clientId: "$3" + help: "Kafka $1 JMX metric type $2" + type: COUNTER + - pattern: kafka.(.+)<>(.+-avg|.+-bytes|.+-count|.+-ratio|.+-age|.+-flight|.+-threads|.+-connectors|.+-tasks|.+-ago) + name: kafka_$2_$4 + labels: + clientId: "$3" + help: "Kafka $1 JMX metric type $2" + type: GAUGE + + #kafka.connect:type=connector-metrics,connector="{connector}" + - pattern: 'kafka.connect<>(connector-class|connector-type|connector-version|status): (.+)' + name: kafka_connect_connector_$2 + value: 1 + labels: + connector: "$1" + $2: "$3" + help: "Kafka Connect $2 JMX metric type connector" + type: GAUGE + + #kafka.connect:type=connector-task-metrics,connector="{connector}",task="{task}<> status" + - pattern: 'kafka.connect<>status: ([a-z-]+)' + name: kafka_connect_connector_task_status + value: 1 + labels: + connector: "$1" + task: "$2" + status: "$3" + help: "Kafka Connect JMX Connector task status" + type: GAUGE + + #kafka.connect:type=task-error-metrics,connector="{connector}",task="{task}" + #kafka.connect:type=source-task-metrics,connector="{connector}",task="{task}" + #kafka.connect:type=sink-task-metrics,connector="{connector}",task="{task}" + #kafka.connect:type=connector-task-metrics,connector="{connector}",task="{task}" + - pattern: kafka.connect<>(.+-total) + name: kafka_connect_$1_$4 + labels: + connector: "$2" + task: "$3" + help: "Kafka Connect JMX metric type $1" + type: COUNTER + - pattern: kafka.connect<>(.+-count|.+-ms|.+-ratio|.+-seq-no|.+-rate|.+-max|.+-avg|.+-failures|.+-requests|.+-timestamp|.+-logged|.+-errors|.+-retries|.+-skipped) + name: kafka_connect_$1_$4 + labels: + connector: "$2" + task: "$3" + help: "Kafka Connect JMX metric type $1" + type: GAUGE + + #kafka.connect:type=connect-worker-metrics,connector="{connector}" + - pattern: kafka.connect<>([a-z-]+) + name: kafka_connect_worker_$2 + labels: + connector: "$1" + help: "Kafka Connect JMX metric $1" + type: GAUGE + + #kafka.connect:type=connect-worker-metrics + - pattern: kafka.connect<>([a-z-]+-total) + name: kafka_connect_worker_$1 + help: "Kafka Connect JMX metric worker" + type: COUNTER + - pattern: kafka.connect<>([a-z-]+) + name: kafka_connect_worker_$1 + help: "Kafka Connect JMX metric worker" + type: GAUGE + + #kafka.connect:type=connect-worker-rebalance-metrics,leader-name|connect-protocol + - pattern: 'kafka.connect<>(leader-name|connect-protocol): (.+)' + name: kafka_connect_worker_rebalance_$1 + value: 1 + labels: + $1: "$2" + help: "Kafka Connect $2 JMX metric type worker rebalance" + type: UNTYPED + + #kafka.connect:type=connect-worker-rebalance-metrics + - pattern: kafka.connect<>([a-z-]+-total) + name: kafka_connect_worker_rebalance_$1 + help: "Kafka Connect JMX metric rebalance information" + type: COUNTER + - pattern: kafka.connect<>([a-z-]+) + name: kafka_connect_worker_rebalance_$1 + help: "Kafka Connect JMX metric rebalance information" + type: GAUGE + + #kafka.connect:type=connect-coordinator-metrics + - pattern: kafka.connect<>(assigned-connectors|assigned-tasks) + name: kafka_connect_coordinator_$1 + help: "Kafka Connect JMX metric assignment information" + type: GAUGE From 6a931a55d0bcff39e42266a50f7cb18161d53afe Mon Sep 17 00:00:00 2001 From: Adam Czajkowski Date: Tue, 25 Feb 2025 13:46:11 +0100 Subject: [PATCH 54/76] Bump Google libraries --- google-cloud-bigtable-kafka-connect-sink/pom.xml | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/google-cloud-bigtable-kafka-connect-sink/pom.xml b/google-cloud-bigtable-kafka-connect-sink/pom.xml index 5658d7b436..888d99c5ba 100644 --- a/google-cloud-bigtable-kafka-connect-sink/pom.xml +++ b/google-cloud-bigtable-kafka-connect-sink/pom.xml @@ -11,7 +11,7 @@ Google Bigtable sink connector for Apache Kafka Connect - 26.31.0 + 26.54.0 3.6.1 2.12 2.6.1 From 3e0f4df82f255a7f212c485efd431054e1480319 Mon Sep 17 00:00:00 2001 From: Adam Czajkowski Date: Tue, 25 Feb 2025 13:53:23 +0100 Subject: [PATCH 55/76] Performance tests --- .../performance/MessageTracer.java | 81 ++ .../performance/README.md | 276 +++++ .../performance/dashboard.json | 1095 +++++++++++++++++ .../performance/enable_tracing.patch | 69 ++ .../kafka-connect-managed-prometheus.yaml | 11 + .../kubernetes/otlp/1_configmap.yaml | 18 + .../kubernetes/otlp/4_deployment.yaml | 4 +- .../performance/kubernetes/otlp/5_hpa.yaml | 2 +- .../performance/terraform/.gitignore | 6 + .../performance/terraform/.terraform.lock.hcl | 42 + .../performance/terraform/Dockerfile | 17 + .../performance/terraform/bigtable.tf | 50 + .../terraform/container_registry.tf | 9 + .../performance/terraform/gcp_apis.tf | 14 + .../performance/terraform/generate_load.sh | 23 + .../performance/terraform/gke.tf | 52 + .../performance/terraform/kafka.tf | 32 + .../performance/terraform/kubernetes.tf | 362 ++++++ .../performance/terraform/main.rs | 96 ++ .../performance/terraform/main.tf | 17 + .../performance/terraform/perf_test.sh | 136 ++ .../performance/terraform/providers.tf | 19 + .../performance/terraform/service_account.tf | 33 + .../performance/terraform/terraform.tf | 12 + 24 files changed, 2473 insertions(+), 3 deletions(-) create mode 100644 google-cloud-bigtable-kafka-connect-sink/performance/MessageTracer.java create mode 100644 google-cloud-bigtable-kafka-connect-sink/performance/README.md create mode 100644 google-cloud-bigtable-kafka-connect-sink/performance/dashboard.json create mode 100644 google-cloud-bigtable-kafka-connect-sink/performance/enable_tracing.patch create mode 100644 google-cloud-bigtable-kafka-connect-sink/performance/kubernetes/kafka-connect-managed-prometheus.yaml create mode 100644 google-cloud-bigtable-kafka-connect-sink/performance/terraform/.gitignore create mode 100644 google-cloud-bigtable-kafka-connect-sink/performance/terraform/.terraform.lock.hcl create mode 100644 google-cloud-bigtable-kafka-connect-sink/performance/terraform/Dockerfile create mode 100644 google-cloud-bigtable-kafka-connect-sink/performance/terraform/bigtable.tf create mode 100644 google-cloud-bigtable-kafka-connect-sink/performance/terraform/container_registry.tf create mode 100644 google-cloud-bigtable-kafka-connect-sink/performance/terraform/gcp_apis.tf create mode 100755 google-cloud-bigtable-kafka-connect-sink/performance/terraform/generate_load.sh create mode 100644 google-cloud-bigtable-kafka-connect-sink/performance/terraform/gke.tf create mode 100644 google-cloud-bigtable-kafka-connect-sink/performance/terraform/kafka.tf create mode 100644 google-cloud-bigtable-kafka-connect-sink/performance/terraform/kubernetes.tf create mode 100644 google-cloud-bigtable-kafka-connect-sink/performance/terraform/main.rs create mode 100644 google-cloud-bigtable-kafka-connect-sink/performance/terraform/main.tf create mode 100755 google-cloud-bigtable-kafka-connect-sink/performance/terraform/perf_test.sh create mode 100644 google-cloud-bigtable-kafka-connect-sink/performance/terraform/providers.tf create mode 100644 google-cloud-bigtable-kafka-connect-sink/performance/terraform/service_account.tf create mode 100644 google-cloud-bigtable-kafka-connect-sink/performance/terraform/terraform.tf diff --git a/google-cloud-bigtable-kafka-connect-sink/performance/MessageTracer.java b/google-cloud-bigtable-kafka-connect-sink/performance/MessageTracer.java new file mode 100644 index 0000000000..fc464c81a1 --- /dev/null +++ b/google-cloud-bigtable-kafka-connect-sink/performance/MessageTracer.java @@ -0,0 +1,81 @@ +/* + * Copyright 2025 Google LLC + * + * Licensed under the Apache License, Version 2.0 (the "License"); + * you may not use this file except in compliance with the License. + * You may obtain a copy of the License at + * + * https://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ +package com.google.cloud.kafka.connect.bigtable.tracing; + +import com.google.cloud.kafka.connect.bigtable.BigtableSinkConnector; +import io.opentelemetry.api.GlobalOpenTelemetry; +import io.opentelemetry.api.OpenTelemetry; +import io.opentelemetry.api.trace.Span; +import io.opentelemetry.api.trace.SpanBuilder; +import io.opentelemetry.api.trace.Tracer; +import io.opentelemetry.api.trace.propagation.W3CTraceContextPropagator; +import io.opentelemetry.context.Context; +import io.opentelemetry.context.propagation.TextMapGetter; +import java.util.Collections; +import java.util.Optional; +import org.apache.kafka.connect.header.Header; +import org.apache.kafka.connect.sink.SinkRecord; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +public class MessageTracer { + private static final String KAFKA_HEADER_NAME = "traceparent"; + private static final Logger logger = LoggerFactory.getLogger(MessageTracer.class); + private static final OpenTelemetry otel = GlobalOpenTelemetry.get(); + + private static Context extractParentContext(SinkRecord record) { + Object traceparentHeaderValue = + Optional.ofNullable(record) + .map(SinkRecord::headers) + .map(hs -> hs.lastWithName(KAFKA_HEADER_NAME)) + .map(Header::value) + .orElse(null); + String traceparent; + if (traceparentHeaderValue instanceof String) { + traceparent = (String) traceparentHeaderValue; + } else { + logger.warn( + "Parent not found for '{}' header value '{}'", KAFKA_HEADER_NAME, traceparentHeaderValue); + return null; + } + // https://github.com/open-telemetry/opentelemetry-java-instrumentation/discussions/4546#discussioncomment-1572327 + return W3CTraceContextPropagator.getInstance() + .extract( + Context.root(), + traceparent, + new TextMapGetter<>() { + @Override + public Iterable keys(String carrier) { + return Collections.singleton(KAFKA_HEADER_NAME); + } + + @Override + public String get(String carrier, String key) { + return key.equals(KAFKA_HEADER_NAME) ? carrier : null; + } + }); + } + + public static Span getRecordSpan(SinkRecord record, String spanName) { + Tracer tracer = otel.getTracer(BigtableSinkConnector.class.getName()); + SpanBuilder spanBuilder = tracer.spanBuilder(spanName); + Context parentContext = extractParentContext(record); + if (parentContext != null) { + spanBuilder.setParent(parentContext); + } + return spanBuilder.startSpan(); + } +} diff --git a/google-cloud-bigtable-kafka-connect-sink/performance/README.md b/google-cloud-bigtable-kafka-connect-sink/performance/README.md new file mode 100644 index 0000000000..91266d50be --- /dev/null +++ b/google-cloud-bigtable-kafka-connect-sink/performance/README.md @@ -0,0 +1,276 @@ +# Performance tests of Cloud Bigtable Sink Connector for Kafka Connect + +## Measured values +We want to measure these latencies under a constant load: +- from the moment the Kafka producer starts writing to the moment the data lands in Cloud Bigtable, +- from the moment the message is in Kafka to the moment the data lands in Cloud Bigtable, +- from the moment our sink receives the data (via `put()` method call) to the moment the data lands in Cloud Bigtable. + +Of course, results are not valid if any error occurs. + +## Infrastructure +The performance test infrastructure is all set up within GCP. + +The high level overview of it is presented in the following diagram: +```mermaid +flowchart + subgraph Kubernetes + KafkaConnect + OpentelemetryCollector + ManagedPrometheus + LoadGenerator + end + ManagedKafka + CloudBigtable + Stackdriver + + LoadGenerator -->|Traces+Metrics| OpentelemetryCollector + KafkaConnect -->|Traces| OpentelemetryCollector --> Stackdriver + KafkaConnect -->|Metrics| ManagedPrometheus --> Stackdriver + LoadGenerator -->|Data| ManagedKafka -->|Data| KafkaConnect -->|Data| CloudBigtable +``` + +### Implementation details +- Strimzi Kafka operator is used to configure Kafka Connect. +- `terraform` code describing used infrastructure is available in directory [terraform](terraform/). +- Some Kubernetes manifests used are available in directory [kubernetes](kubernetes/). + +## Infrastructure configuration +### Location +The tests were conducted in `europe-central2-a` (Warsaw, Poland) zone. + +### Versions +- Kubernetes: 1.31.5 +- `terraform`: 1.10.5 + - `hashicorp/google` provider: 6.19.0 + - `hashicorp/kubernetes` provider: 2.35.1 +- Strimzi Kafka Operator: 0.45.0 + - Kafka Connect: 3.8.1 + +### Cloud Bigtable +Single-node cluster with SSD storage. + +See [bigtable.tf](terraform/bigtable.tf) for details. + +### Managed Kafka +A cluster with 3 vCPU and 3 GB of RAM. + +See [kafka.tf](terraform/kafka.tf) for details. + +### Kubernetes +A cluster consisting of 4 `n1-standard-4` machines (4 vCPU, 15 GB of RAM) + +#### Kafka Connect +A three-node cluster with each machine reserving 2.66 vCPU and 6 GB of RAM. + +Its metrics are exported using [JMX Prometheus exporter](https://github.com/prometheus/jmx_exporter). + +See [kubernetes.tf](terraform/kubernetes.tf) for details. + +#### Load generator +A single Kubernetes Pod reserving 2 vCPU and 8 GB of RAM and running a [simple script](terraform/generate_load.sh) consisting of a three-part shell pipeline: +- [a simple script generating data of desired shape](terraform/main.rs), +- [`pv`](https://www.ivarch.com/programs/pv.shtml) used to rate limit the generator script, +- upstream-provided wrapper for Kafka producer, `kafka-console-producer.sh`. + +#### Opentelemetry Collector +An autoscaled service consisting of 6..10 Pods each limited to 0.2 vCPU and 0.8GB of RAM. + +#### Managed Prometheus +An autoscaled service using default configuration. + +## Components' configuration +### Kafka clients (Kafka Connect and the load generator) +Kafka clients used by the load generator and Kafka Connect are configured to optimize latency as suggested in [Confluent's "Optimizing Your Apache Kafka® Deployment" whitepaper](https://www.confluent.io/resources/white-paper/optimizing-your-apache-kafka-deployment/). + +Producer settings: +- `linger.ms`: `0` +- `compression.type`: `none` +- `acks`: `1` +Consumer settings: +- `fetch.min.bytes`: `1` + +### Kafka Connect +Replication factor of Kafka topics used for internal state storage: the same as in Kafka. + +Key converter: `org.apache.kafka.connect.json.JsonConverter` with schemas disabled. + +Value converter; `org.apache.kafka.connect.json.JsonConverter` with schemas enabled. + +It is also configured to use TLS when accessing Kafka API. + +### Kafka topic +The topic is configured to use 60 partitions with replication to all 3 Kafka nodes. + +### Cloud Bigtable Sink Connector +The full configuration of the sink connector is available in [kubernetes.tf](terraform/kubernetes.tf). +The interesting parts of these settings are: +```json +{ + "auto.create.column.families": "false", + "auto.create.tables": "false", + "error.mode": "FAIL", + "insert.mode": "upsert", + "max.batch.size": "1000", + "retry.timeout.ms": "90000", + "tasks.max": "60", + "value.null.mode": "write" +} +``` + +### Load generator +[generate_load.sh](terraform/generate_load.sh) and [main.rs](terraform/main.rs) are configured using environment variables: +- `KAFKA_CONNECT_BOOTSTRAP_SERVERS`, `KAFKA_CONNECT_TLS`, `KAFKA_CONNECT_SASL_USERNAME`, `KAFKA_CONNECT_SASL_PASSWORD_FILE`, `KAFKA_CONNECT_SASL_MECHANISM` - the same meaning as in Strimzi's Kafka Connect +- `THROUGHPUT` - number of messages to be sent per second +- `TIMEOUT` - duration of load generation (default unit: seconds) +- `TOPIC` - topic to write to +The script generates data in two shapes: +- a String key and a String value - when `COLUMN_FAMILIES` is equal to `0`, +- a String key and a Struct mapping to `COLUMN_FAMILIES` column families each containing `COLUMNS_PER_FAMILY` columns. +The values in each cell have `FIELD_VALUE_SIZE` bytes each. +Note that we use JSON with in-band schema, so messages contain schema bytes. +This overhead is not insignificant, for example: +```bash +# Just a String value +$ FIELD_VALUE_SIZE=100 COLUMN_FAMILIES=0 COLUMNS_PER_FAMILY=1 ./gen 2>/dev/null | cut -d"|" -f2 | head -1 | wc --bytes +158 +``` + +The default values are: +- `THROUGHPUT` - `3000` +- `TIMEOUT` - `600` +- `COLUMN_FAMILIES` - `0` +- `COLUMNS_PER_FAMILY` - `1` +- `FIELD_VALUE_SIZE` - `100` + +## Tracing setup +### Tracing overview +All the code (the load generator, Kafka Connect, and our sink) is instrumented using Opentelemetry. + +In case of the load generator and Kafka Connect, the instrumentation is automatic, because Kafka clients (both reader and producer) are supported by [the Opentelemetry agent](https://opentelemetry.io/docs/zero-code/java/agent/), which we run alongside these two components. + +The sink is instrumented by calling appropriate functions in the code. +Interestingly, it also uses the Opentelemetry agent indirectly - our code uses `GlobalOpenTelemetry` which is injected by the agent. + +### Tracing data path +All the traces pass through Opentelemetry collector, which is responsible for sending it into Google Stackdriver. + +### Tracing code +Tracing code is not present in the release code. +The code of tracing implementation is available in [`MessageTracer.java`](MessageTracer.java). +To enable tracing, put it into `src/main/java/com/google/cloud/kafka/connect/bigtable/tracing` and apply [`enable_tracing.patch` git patch](enable_tracing.patch). + +## Test steps +### High level overview +A single test run consists of just running a load generator for a stretch of time and collecting metrics and traces from all the components. + +### Required programs +- `terraform` (with GCP credentials configured) +- `kubectl` +- `jq` +- `bash` + coreutils +- `gcloud` (logged into the desired project) + +### Preparation +- Adjust variables such as project name and region in [`main.tf`](terraform/main.tf) +- Adjust the test config by modifying variables in [`kubernetes.tf`](terraform/kubernetes.tf). + +### Running tests +[`perf_test.sh`](terraform/perf_test.sh) is the script that executes all the needed steps: + +Always run it from the directory it resides in. + +#### `perf_test.sh prepare` +Needs to only be executed once at the beginning. + +It prepares all the environment: sets up the infrastructure, builds the docker container containing our connector and load generator, creates helper Kubernetes resources (Strimzi Kafka Operator, Opentelemetry collector, Managed Prometheus). + +It is idempontent. + +#### `perf_test.sh run` +Creates Cloud Bigtable table and Kafka topic and then executes a performance test using current configuration from .tf files. + +#### `perf_test.sh cleanup` +Cleans up the environment for the next `perf_test.sh run`: removes existing Kafka Connect and the connector and deletes Kafka topic and Cloud Bigtable table. + +#### `perf_test.sh rerun` +A shortcut for `perf_test.sh cleanup && perf_test.sh run`. + +#### `perf_test.sh destroy` +Removes all the infrastructure set up for the testing. + +Note that it requires manual confirmation. + +## Metrics +Import the following JSON into https://console.cloud.google.com/monitoring/dashboards: +[dashboard.json](dashboard.json). + +You might need to replace some strings such as project name analogically to [main.tf modifications](#preparation). + +## Trace analysis +You can browse traces at https://console.cloud.google.com/traces/explorer. + +Despite our configuration that attempts to record trace of every record, the Opentelemetry collector cannot keep up with too high rate of traces. +To verify if it is the case you can compare rate of traces in Trace Explorer and metrics (or directly look at Opentelemetry collector's metrics). + +### Basic +For basic analysis use https://console.cloud.google.com/traces/tasks. + +Sample queries: +- `+root:"bigtablesink-kafka-topic publish" +span:sinkProcessing` - collect traces such that the first span is named `bigtablesink-kafka-topic publish` and a span `sinkProcessing` is present +- `+span:sinkProcessing` - collect all traces containing span named `sinkProcessing` + +Query language reference: https://cloud.google.com/trace/docs/trace-filters. + +Note that to compute total trace time this analysis sums time of all member spans rather than compute delta between end of the last span and start of the first one. +Still, sample traces are still very useful to look at. + +### Advanced +To aggregate wall time instead of CPU time, you can use https://console.cloud.google.com/logs/analytics. + +The following query calculates percentiles for three kinds of latencies described in [Measured Values](#measured-values). +Remember to adjust project, topic name, and timestamps. +Note that latency from the moment the message is in Kafka to the moment the data lands in Cloud Bigtable is calculated using the producer's trace since Kafka Connect consumer's span could start after the data arrives. +Unfortunately, it might be inaccurate if producer process lacks CPU and closes span with delay. +It's a good idea to exclude the first 60-90s and the last 30-60s of the test period so that outliers caused by scaling rate of messages up or down doesn't influence the results. + +```sql +SELECT + APPROX_QUANTILES(Process.duration_nano / 1000, 100)[OFFSET(50)] as sink_p50, + APPROX_QUANTILES(Process.duration_nano / 1000, 100)[OFFSET(75)] as sink_p75, + APPROX_QUANTILES(Process.duration_nano / 1000, 100)[OFFSET(90)] as sink_p90, + APPROX_QUANTILES(Process.duration_nano / 1000, 100)[OFFSET(95)] as sink_p95, + APPROX_QUANTILES(Process.duration_nano / 1000, 100)[OFFSET(99)] as sink_p99, + APPROX_QUANTILES(Process.duration_nano / 1000, 1000)[OFFSET(995)] as sink_p995, + APPROX_QUANTILES(Process.duration_nano / 1000, 1000)[OFFSET(999)] as sink_p999, + MIN(Process.duration_nano / 1000) as sink_min, + AVG(Process.duration_nano / 1000) as sink_avg, + MAX(Process.duration_nano / 1000) as sink_max, + APPROX_QUANTILES(TIMESTAMP_DIFF(Process.start_time, Publish.start_time, MICROSECOND) + (Process.duration_nano / 1000) - (Publish.duration_nano / 1000), 100)[OFFSET(50)] as kafka_to_bigtable_p50, + APPROX_QUANTILES(TIMESTAMP_DIFF(Process.start_time, Publish.start_time, MICROSECOND) + (Process.duration_nano / 1000) - (Publish.duration_nano / 1000), 100)[OFFSET(75)] as kafka_to_bigtable_p75, + APPROX_QUANTILES(TIMESTAMP_DIFF(Process.start_time, Publish.start_time, MICROSECOND) + (Process.duration_nano / 1000) - (Publish.duration_nano / 1000), 100)[OFFSET(90)] as kafka_to_bigtable_p90, + APPROX_QUANTILES(TIMESTAMP_DIFF(Process.start_time, Publish.start_time, MICROSECOND) + (Process.duration_nano / 1000) - (Publish.duration_nano / 1000), 100)[OFFSET(95)] as kafka_to_bigtable_p95, + APPROX_QUANTILES(TIMESTAMP_DIFF(Process.start_time, Publish.start_time, MICROSECOND) + (Process.duration_nano / 1000) - (Publish.duration_nano / 1000), 100)[OFFSET(99)] as kafka_to_bigtable_p99, + APPROX_QUANTILES(TIMESTAMP_DIFF(Process.start_time, Publish.start_time, MICROSECOND) + (Process.duration_nano / 1000) - (Publish.duration_nano / 1000), 1000)[OFFSET(995)] as kafka_to_bigtable_p995, + APPROX_QUANTILES(TIMESTAMP_DIFF(Process.start_time, Publish.start_time, MICROSECOND) + (Process.duration_nano / 1000) - (Publish.duration_nano / 1000), 1000)[OFFSET(999)] as kafka_to_bigtable_p999, + MIN(TIMESTAMP_DIFF(Process.start_time, Publish.start_time, MICROSECOND) + (Process.duration_nano / 1000) - (Publish.duration_nano / 1000)) as kafka_to_bigtable_min, + AVG(TIMESTAMP_DIFF(Process.start_time, Publish.start_time, MICROSECOND) + (Process.duration_nano / 1000) - (Publish.duration_nano / 1000)) as kafka_to_bigtable_avg, + MAX(TIMESTAMP_DIFF(Process.start_time, Publish.start_time, MICROSECOND) + (Process.duration_nano / 1000) - (Publish.duration_nano / 1000)) as kafka_to_bigtable_max, + APPROX_QUANTILES(TIMESTAMP_DIFF(Process.start_time, Publish.start_time, MICROSECOND) + (Process.duration_nano / 1000), 100)[OFFSET(50)] as total_p50, + APPROX_QUANTILES(TIMESTAMP_DIFF(Process.start_time, Publish.start_time, MICROSECOND) + (Process.duration_nano / 1000), 100)[OFFSET(75)] as total_p75, + APPROX_QUANTILES(TIMESTAMP_DIFF(Process.start_time, Publish.start_time, MICROSECOND) + (Process.duration_nano / 1000), 100)[OFFSET(90)] as total_p90, + APPROX_QUANTILES(TIMESTAMP_DIFF(Process.start_time, Publish.start_time, MICROSECOND) + (Process.duration_nano / 1000), 100)[OFFSET(95)] as total_p95, + APPROX_QUANTILES(TIMESTAMP_DIFF(Process.start_time, Publish.start_time, MICROSECOND) + (Process.duration_nano / 1000), 100)[OFFSET(99)] as total_p99, + APPROX_QUANTILES(TIMESTAMP_DIFF(Process.start_time, Publish.start_time, MICROSECOND) + (Process.duration_nano / 1000), 1000)[OFFSET(995)] as total_p995, + APPROX_QUANTILES(TIMESTAMP_DIFF(Process.start_time, Publish.start_time, MICROSECOND) + (Process.duration_nano / 1000), 1000)[OFFSET(999)] as total_p999, + MIN(TIMESTAMP_DIFF(Process.start_time, Publish.start_time, MICROSECOND) + (Process.duration_nano / 1000)) as total_min, + AVG(TIMESTAMP_DIFF(Process.start_time, Publish.start_time, MICROSECOND) + (Process.duration_nano / 1000)) as total_avg, + MAX(TIMESTAMP_DIFF(Process.start_time, Publish.start_time, MICROSECOND) + (Process.duration_nano / 1000)) as total_max +FROM `unoperate-test.global._Trace._AllSpans` Publish, `unoperate-test.global._Trace._AllSpans` Process +WHERE + Publish.name = "bigtablesink-kafka-topic publish" + AND Process.name = "sinkProcessing" + AND Publish.start_time > TIMESTAMP('2025-02-21T12:52:00 UTC') + AND Publish.start_time < TIMESTAMP('2025-02-21T13:03:00 UTC') + AND Publish.trace_id = Process.trace_id +``` diff --git a/google-cloud-bigtable-kafka-connect-sink/performance/dashboard.json b/google-cloud-bigtable-kafka-connect-sink/performance/dashboard.json new file mode 100644 index 0000000000..f43056c840 --- /dev/null +++ b/google-cloud-bigtable-kafka-connect-sink/performance/dashboard.json @@ -0,0 +1,1095 @@ +{ + "displayName": "Bigtable Sink Performance Tests", + "dashboardFilters": [], + "labels": {}, + "mosaicLayout": { + "columns": 48, + "tiles": [ + { + "height": 16, + "width": 24, + "widget": { + "title": "Kafka Topic", + "id": "", + "xyChart": { + "chartOptions": { + "displayHorizontal": false, + "mode": "COLOR", + "showLegend": false + }, + "dataSets": [ + { + "breakdowns": [], + "dimensions": [], + "legendTemplate": "", + "measures": [], + "minAlignmentPeriod": "60s", + "plotType": "LINE", + "targetAxis": "Y2", + "timeSeriesQuery": { + "outputFullDuration": false, + "timeSeriesFilter": { + "aggregation": { + "alignmentPeriod": "60s", + "crossSeriesReducer": "REDUCE_SUM", + "groupByFields": [], + "perSeriesAligner": "ALIGN_RATE" + }, + "filter": "metric.type=\"managedkafka.googleapis.com/message_in_count\" resource.type=\"managedkafka.googleapis.com/Topic\" resource.label.\"topic_id\"=\"bigtablesink-kafka-topic\"" + }, + "unitOverride": "" + } + }, + { + "breakdowns": [], + "dimensions": [], + "legendTemplate": "", + "measures": [], + "minAlignmentPeriod": "60s", + "plotType": "LINE", + "targetAxis": "Y1", + "timeSeriesQuery": { + "outputFullDuration": false, + "timeSeriesFilter": { + "aggregation": { + "alignmentPeriod": "60s", + "crossSeriesReducer": "REDUCE_SUM", + "groupByFields": [], + "perSeriesAligner": "ALIGN_RATE" + }, + "filter": "metric.type=\"managedkafka.googleapis.com/byte_in_count\" resource.type=\"managedkafka.googleapis.com/Topic\" resource.label.\"topic_id\"=\"bigtablesink-kafka-topic\"" + }, + "unitOverride": "" + } + }, + { + "breakdowns": [], + "dimensions": [], + "legendTemplate": "", + "measures": [], + "minAlignmentPeriod": "60s", + "plotType": "LINE", + "targetAxis": "Y1", + "timeSeriesQuery": { + "outputFullDuration": false, + "timeSeriesFilter": { + "aggregation": { + "alignmentPeriod": "60s", + "crossSeriesReducer": "REDUCE_SUM", + "groupByFields": [], + "perSeriesAligner": "ALIGN_RATE" + }, + "filter": "metric.type=\"managedkafka.googleapis.com/byte_out_count\" resource.type=\"managedkafka.googleapis.com/Topic\" resource.label.\"topic_id\"=\"bigtablesink-kafka-topic\"" + }, + "unitOverride": "" + } + } + ], + "thresholds": [], + "y2Axis": { + "label": "", + "scale": "LINEAR" + }, + "yAxis": { + "label": "", + "scale": "LINEAR" + } + } + } + }, + { + "xPos": 24, + "height": 16, + "width": 24, + "widget": { + "title": "Kafka Connect & Load Generator Resources", + "id": "", + "xyChart": { + "chartOptions": { + "displayHorizontal": false, + "mode": "COLOR", + "showLegend": false + }, + "dataSets": [ + { + "breakdowns": [], + "dimensions": [], + "legendTemplate": "", + "measures": [], + "plotType": "LINE", + "targetAxis": "Y1", + "timeSeriesQuery": { + "outputFullDuration": false, + "timeSeriesQueryLanguage": "fetch k8s_container\n| metric 'kubernetes.io/container/cpu/core_usage_time'\n| filter (resource.project_id == 'unoperate-test')\n| filter (resource.cluster_name == 'bigtablesink-k8s-cluster')\n| filter (resource.pod_name == 'load-generator' || resource.container_name == 'my-connect-cluster-connect')\n| align rate(1m)\n| every 1m\n| group_by [resource.location, resource.pod_name],\n [value_core_usage_time_aggregate: aggregate(value.core_usage_time)] \n", + "unitOverride": "" + } + }, + { + "breakdowns": [], + "dimensions": [], + "legendTemplate": "", + "measures": [], + "plotType": "LINE", + "targetAxis": "Y2", + "timeSeriesQuery": { + "outputFullDuration": false, + "timeSeriesQueryLanguage": "fetch k8s_container\n| metric 'kubernetes.io/container/memory/used_bytes'\n| filter (resource.project_id == 'unoperate-test')\n| filter (resource.cluster_name == 'bigtablesink-k8s-cluster')\n| filter (resource.pod_name == 'load-generator' || resource.container_name == 'my-connect-cluster-connect')\n| align next_older(1m)\n| every 1m\n| group_by [resource.location, resource.pod_name],\n [value_used_bytes_aggregate: aggregate(value.used_bytes)]\n", + "unitOverride": "" + } + } + ], + "thresholds": [], + "timeshiftDuration": "0s", + "y2Axis": { + "label": "", + "scale": "LINEAR" + }, + "yAxis": { + "label": "", + "scale": "LINEAR" + } + } + } + }, + { + "yPos": 16, + "height": 16, + "width": 24, + "widget": { + "title": "Kafka Connect Sink Read/Send Rates", + "id": "", + "xyChart": { + "chartOptions": { + "displayHorizontal": false, + "mode": "COLOR", + "showLegend": false + }, + "dataSets": [ + { + "breakdowns": [], + "dimensions": [], + "legendTemplate": "", + "measures": [], + "minAlignmentPeriod": "60s", + "plotType": "LINE", + "targetAxis": "Y1", + "timeSeriesQuery": { + "outputFullDuration": false, + "timeSeriesFilter": { + "aggregation": { + "alignmentPeriod": "60s", + "crossSeriesReducer": "REDUCE_SUM", + "groupByFields": [], + "perSeriesAligner": "ALIGN_MEAN" + }, + "filter": "metric.type=\"prometheus.googleapis.com/kafka_connect_sink_task_sink_record_read_rate/gauge\" resource.type=\"prometheus_target\"" + }, + "unitOverride": "" + } + }, + { + "breakdowns": [], + "dimensions": [], + "legendTemplate": "", + "measures": [], + "minAlignmentPeriod": "60s", + "plotType": "LINE", + "targetAxis": "Y1", + "timeSeriesQuery": { + "outputFullDuration": false, + "timeSeriesFilter": { + "aggregation": { + "alignmentPeriod": "60s", + "crossSeriesReducer": "REDUCE_SUM", + "groupByFields": [], + "perSeriesAligner": "ALIGN_MEAN" + }, + "filter": "metric.type=\"prometheus.googleapis.com/kafka_connect_sink_task_sink_record_send_rate/gauge\" resource.type=\"prometheus_target\"" + }, + "unitOverride": "" + } + } + ], + "thresholds": [], + "yAxis": { + "label": "", + "scale": "LINEAR" + } + } + } + }, + { + "yPos": 16, + "xPos": 24, + "height": 16, + "width": 24, + "widget": { + "title": "Kafka Resources", + "id": "", + "xyChart": { + "chartOptions": { + "displayHorizontal": false, + "mode": "COLOR", + "showLegend": false + }, + "dataSets": [ + { + "breakdowns": [], + "dimensions": [], + "legendTemplate": "", + "measures": [], + "minAlignmentPeriod": "60s", + "plotType": "LINE", + "targetAxis": "Y1", + "timeSeriesQuery": { + "outputFullDuration": false, + "timeSeriesFilter": { + "aggregation": { + "alignmentPeriod": "60s", + "crossSeriesReducer": "REDUCE_MAX", + "groupByFields": [], + "perSeriesAligner": "ALIGN_RATE" + }, + "filter": "metric.type=\"managedkafka.googleapis.com/cpu/core_usage_time\" resource.type=\"managedkafka.googleapis.com/Cluster\"" + }, + "unitOverride": "" + } + }, + { + "breakdowns": [], + "dimensions": [], + "legendTemplate": "", + "measures": [], + "minAlignmentPeriod": "60s", + "plotType": "LINE", + "targetAxis": "Y2", + "timeSeriesQuery": { + "outputFullDuration": false, + "timeSeriesFilter": { + "aggregation": { + "alignmentPeriod": "60s", + "crossSeriesReducer": "REDUCE_MAX", + "groupByFields": [], + "perSeriesAligner": "ALIGN_MAX" + }, + "filter": "metric.type=\"managedkafka.googleapis.com/memory/usage\" resource.type=\"managedkafka.googleapis.com/Cluster\"" + }, + "unitOverride": "" + } + }, + { + "breakdowns": [], + "dimensions": [], + "legendTemplate": "", + "measures": [], + "minAlignmentPeriod": "60s", + "plotType": "LINE", + "targetAxis": "Y1", + "timeSeriesQuery": { + "outputFullDuration": false, + "timeSeriesFilter": { + "aggregation": { + "alignmentPeriod": "60s", + "crossSeriesReducer": "REDUCE_MEAN", + "groupByFields": [], + "perSeriesAligner": "ALIGN_RATE" + }, + "filter": "metric.type=\"managedkafka.googleapis.com/cpu/core_usage_time\" resource.type=\"managedkafka.googleapis.com/Cluster\"" + }, + "unitOverride": "" + } + }, + { + "breakdowns": [], + "dimensions": [], + "legendTemplate": "", + "measures": [], + "minAlignmentPeriod": "60s", + "plotType": "LINE", + "targetAxis": "Y1", + "timeSeriesQuery": { + "outputFullDuration": false, + "timeSeriesFilter": { + "aggregation": { + "alignmentPeriod": "60s", + "crossSeriesReducer": "REDUCE_MEAN", + "groupByFields": [], + "perSeriesAligner": "ALIGN_MEAN" + }, + "filter": "metric.type=\"managedkafka.googleapis.com/memory/usage\" resource.type=\"managedkafka.googleapis.com/Cluster\"" + }, + "unitOverride": "" + } + } + ], + "thresholds": [], + "y2Axis": { + "label": "", + "scale": "LINEAR" + }, + "yAxis": { + "label": "", + "scale": "LINEAR" + } + } + } + }, + { + "yPos": 32, + "height": 16, + "width": 24, + "widget": { + "title": "Kafka Connect Batch Latencies", + "id": "", + "xyChart": { + "chartOptions": { + "displayHorizontal": false, + "mode": "COLOR", + "showLegend": false + }, + "dataSets": [ + { + "breakdowns": [], + "dimensions": [], + "legendTemplate": "", + "measures": [], + "minAlignmentPeriod": "60s", + "plotType": "LINE", + "targetAxis": "Y1", + "timeSeriesQuery": { + "outputFullDuration": false, + "timeSeriesFilter": { + "aggregation": { + "alignmentPeriod": "60s", + "crossSeriesReducer": "REDUCE_MAX", + "groupByFields": [], + "perSeriesAligner": "ALIGN_MAX" + }, + "filter": "metric.type=\"prometheus.googleapis.com/kafka_connect_sink_task_put_batch_max_time_ms/gauge\" resource.type=\"prometheus_target\"" + }, + "unitOverride": "" + } + }, + { + "breakdowns": [], + "dimensions": [], + "legendTemplate": "", + "measures": [], + "minAlignmentPeriod": "60s", + "plotType": "LINE", + "targetAxis": "Y1", + "timeSeriesQuery": { + "outputFullDuration": false, + "timeSeriesFilter": { + "aggregation": { + "alignmentPeriod": "60s", + "crossSeriesReducer": "REDUCE_MEAN", + "groupByFields": [], + "perSeriesAligner": "ALIGN_MEAN" + }, + "filter": "metric.type=\"prometheus.googleapis.com/kafka_connect_sink_task_put_batch_avg_time_ms/gauge\" resource.type=\"prometheus_target\"" + }, + "unitOverride": "" + } + } + ], + "thresholds": [], + "yAxis": { + "label": "", + "scale": "LINEAR" + } + } + } + }, + { + "yPos": 32, + "xPos": 24, + "height": 16, + "width": 24, + "widget": { + "title": "Bigtable CPU", + "id": "", + "xyChart": { + "chartOptions": { + "displayHorizontal": false, + "mode": "COLOR", + "showLegend": false + }, + "dataSets": [ + { + "breakdowns": [], + "dimensions": [], + "legendTemplate": "", + "measures": [], + "minAlignmentPeriod": "60s", + "plotType": "LINE", + "targetAxis": "Y1", + "timeSeriesQuery": { + "outputFullDuration": false, + "timeSeriesFilter": { + "aggregation": { + "alignmentPeriod": "60s", + "groupByFields": [], + "perSeriesAligner": "ALIGN_MEAN" + }, + "filter": "metric.type=\"bigtable.googleapis.com/cluster/cpu_load_hottest_node_high_granularity\" resource.type=\"bigtable_cluster\"" + }, + "unitOverride": "" + } + }, + { + "breakdowns": [], + "dimensions": [], + "legendTemplate": "", + "measures": [], + "minAlignmentPeriod": "60s", + "plotType": "LINE", + "targetAxis": "Y1", + "timeSeriesQuery": { + "outputFullDuration": false, + "timeSeriesFilter": { + "aggregation": { + "alignmentPeriod": "60s", + "groupByFields": [], + "perSeriesAligner": "ALIGN_MEAN" + }, + "filter": "metric.type=\"bigtable.googleapis.com/cluster/cpu_load_by_app_profile_by_method_by_table\" resource.type=\"bigtable_cluster\"" + }, + "unitOverride": "" + } + } + ], + "thresholds": [], + "yAxis": { + "label": "", + "scale": "LINEAR" + } + } + } + }, + { + "yPos": 48, + "height": 16, + "width": 24, + "widget": { + "title": "Kafka Connect Batch Sizes", + "id": "", + "xyChart": { + "chartOptions": { + "displayHorizontal": false, + "mode": "COLOR", + "showLegend": false + }, + "dataSets": [ + { + "breakdowns": [], + "dimensions": [], + "legendTemplate": "", + "measures": [], + "minAlignmentPeriod": "60s", + "plotType": "LINE", + "targetAxis": "Y1", + "timeSeriesQuery": { + "outputFullDuration": false, + "timeSeriesFilter": { + "aggregation": { + "alignmentPeriod": "60s", + "crossSeriesReducer": "REDUCE_MEAN", + "groupByFields": [], + "perSeriesAligner": "ALIGN_MEAN" + }, + "filter": "metric.type=\"prometheus.googleapis.com/kafka_connect_connector_task_batch_size_avg/gauge\" resource.type=\"prometheus_target\"" + }, + "unitOverride": "" + } + }, + { + "breakdowns": [], + "dimensions": [], + "legendTemplate": "", + "measures": [], + "minAlignmentPeriod": "60s", + "plotType": "LINE", + "targetAxis": "Y1", + "timeSeriesQuery": { + "outputFullDuration": false, + "timeSeriesFilter": { + "aggregation": { + "alignmentPeriod": "60s", + "crossSeriesReducer": "REDUCE_MAX", + "groupByFields": [], + "perSeriesAligner": "ALIGN_MAX" + }, + "filter": "metric.type=\"prometheus.googleapis.com/kafka_connect_connector_task_batch_size_max/gauge\" resource.type=\"prometheus_target\"" + }, + "unitOverride": "" + } + }, + { + "breakdowns": [], + "dimensions": [], + "legendTemplate": "", + "measures": [], + "minAlignmentPeriod": "60s", + "plotType": "LINE", + "targetAxis": "Y1", + "timeSeriesQuery": { + "outputFullDuration": false, + "timeSeriesFilter": { + "aggregation": { + "alignmentPeriod": "60s", + "crossSeriesReducer": "REDUCE_MEAN", + "groupByFields": [], + "perSeriesAligner": "ALIGN_MEAN" + }, + "filter": "metric.type=\"prometheus.googleapis.com/kafka_connect_sink_task_sink_record_active_count/gauge\" resource.type=\"prometheus_target\"" + }, + "unitOverride": "" + } + } + ], + "thresholds": [], + "yAxis": { + "label": "", + "scale": "LINEAR" + } + } + } + }, + { + "yPos": 48, + "xPos": 24, + "height": 16, + "width": 24, + "widget": { + "title": "Kubernetes Network", + "id": "", + "xyChart": { + "chartOptions": { + "displayHorizontal": false, + "mode": "COLOR", + "showLegend": false + }, + "dataSets": [ + { + "breakdowns": [], + "dimensions": [], + "legendTemplate": "", + "measures": [], + "minAlignmentPeriod": "60s", + "plotType": "LINE", + "targetAxis": "Y1", + "timeSeriesQuery": { + "outputFullDuration": false, + "timeSeriesFilter": { + "aggregation": { + "alignmentPeriod": "60s", + "crossSeriesReducer": "REDUCE_SUM", + "groupByFields": [ + "metadata.user_labels.\"app.kubernetes.io/name\"" + ], + "perSeriesAligner": "ALIGN_RATE" + }, + "filter": "metric.type=\"kubernetes.io/pod/network/received_bytes_count\" resource.type=\"k8s_pod\"" + }, + "unitOverride": "" + } + }, + { + "breakdowns": [], + "dimensions": [], + "legendTemplate": "", + "measures": [], + "minAlignmentPeriod": "60s", + "plotType": "LINE", + "targetAxis": "Y1", + "timeSeriesQuery": { + "outputFullDuration": false, + "timeSeriesFilter": { + "aggregation": { + "alignmentPeriod": "60s", + "crossSeriesReducer": "REDUCE_SUM", + "groupByFields": [ + "metadata.user_labels.\"app.kubernetes.io/name\"" + ], + "perSeriesAligner": "ALIGN_RATE" + }, + "filter": "metric.type=\"kubernetes.io/pod/network/sent_bytes_count\" resource.type=\"k8s_pod\"" + }, + "unitOverride": "" + } + }, + { + "breakdowns": [], + "dimensions": [], + "legendTemplate": "", + "measures": [], + "minAlignmentPeriod": "60s", + "plotType": "LINE", + "targetAxis": "Y1", + "timeSeriesQuery": { + "outputFullDuration": false, + "timeSeriesFilter": { + "aggregation": { + "alignmentPeriod": "60s", + "crossSeriesReducer": "REDUCE_SUM", + "groupByFields": [ + "resource.label.\"pod_name\"" + ], + "perSeriesAligner": "ALIGN_RATE" + }, + "filter": "metric.type=\"kubernetes.io/pod/network/received_bytes_count\" resource.type=\"k8s_pod\" resource.label.\"pod_name\"=\"load-generator\"" + }, + "unitOverride": "" + } + }, + { + "breakdowns": [], + "dimensions": [], + "legendTemplate": "", + "measures": [], + "minAlignmentPeriod": "60s", + "plotType": "LINE", + "targetAxis": "Y1", + "timeSeriesQuery": { + "outputFullDuration": false, + "timeSeriesFilter": { + "aggregation": { + "alignmentPeriod": "60s", + "crossSeriesReducer": "REDUCE_SUM", + "groupByFields": [ + "resource.label.\"pod_name\"" + ], + "perSeriesAligner": "ALIGN_RATE" + }, + "filter": "metric.type=\"kubernetes.io/pod/network/sent_bytes_count\" resource.type=\"k8s_pod\" resource.label.\"pod_name\"=\"load-generator\"" + }, + "unitOverride": "" + } + } + ], + "thresholds": [], + "yAxis": { + "label": "", + "scale": "LINEAR" + } + } + } + }, + { + "yPos": 64, + "height": 16, + "width": 24, + "widget": { + "title": "Bigtable Data API 50p Latencies", + "id": "", + "xyChart": { + "chartOptions": { + "displayHorizontal": false, + "mode": "COLOR", + "showLegend": false + }, + "dataSets": [ + { + "breakdowns": [], + "dimensions": [], + "legendTemplate": "", + "measures": [], + "minAlignmentPeriod": "60s", + "plotType": "LINE", + "targetAxis": "Y1", + "timeSeriesQuery": { + "outputFullDuration": false, + "timeSeriesFilter": { + "aggregation": { + "alignmentPeriod": "60s", + "crossSeriesReducer": "REDUCE_PERCENTILE_50", + "groupByFields": [ + "resource.label.\"method\"" + ], + "perSeriesAligner": "ALIGN_SUM" + }, + "filter": "metric.type=\"serviceruntime.googleapis.com/api/request_latencies\" resource.type=\"consumed_api\" resource.label.\"project_id\"=\"unoperate-test\" resource.label.\"service\"=\"bigtable.googleapis.com\"" + }, + "unitOverride": "" + } + } + ], + "thresholds": [], + "yAxis": { + "label": "", + "scale": "LINEAR" + } + } + } + }, + { + "yPos": 64, + "xPos": 24, + "height": 16, + "width": 24, + "widget": { + "title": "Kubernetes CPU", + "id": "", + "xyChart": { + "chartOptions": { + "displayHorizontal": false, + "mode": "COLOR", + "showLegend": false + }, + "dataSets": [ + { + "breakdowns": [], + "dimensions": [], + "legendTemplate": "", + "measures": [], + "minAlignmentPeriod": "60s", + "plotType": "LINE", + "targetAxis": "Y1", + "timeSeriesQuery": { + "outputFullDuration": false, + "timeSeriesFilter": { + "aggregation": { + "alignmentPeriod": "60s", + "crossSeriesReducer": "REDUCE_SUM", + "groupByFields": [ + "metadata.user_labels.\"app.kubernetes.io/name\"" + ], + "perSeriesAligner": "ALIGN_RATE" + }, + "filter": "metric.type=\"kubernetes.io/container/cpu/core_usage_time\" resource.type=\"k8s_container\"" + }, + "unitOverride": "" + } + } + ], + "thresholds": [], + "yAxis": { + "label": "", + "scale": "LINEAR" + } + } + } + }, + { + "yPos": 80, + "height": 16, + "width": 24, + "widget": { + "title": "Bigtable Data API Error%", + "id": "", + "xyChart": { + "chartOptions": { + "displayHorizontal": false, + "mode": "COLOR", + "showLegend": false + }, + "dataSets": [ + { + "breakdowns": [], + "dimensions": [], + "legendTemplate": "", + "measures": [], + "plotType": "LINE", + "targetAxis": "Y1", + "timeSeriesQuery": { + "outputFullDuration": false, + "timeSeriesFilterRatio": { + "denominator": { + "aggregation": { + "crossSeriesReducer": "REDUCE_SUM", + "groupByFields": [ + "resource.label.\"method\"" + ], + "perSeriesAligner": "ALIGN_RATE" + }, + "filter": "metric.type=\"serviceruntime.googleapis.com/api/request_count\" resource.type=\"consumed_api\" resource.label.\"project_id\"=\"unoperate-test\" resource.label.\"service\"=\"bigtable.googleapis.com\"" + }, + "numerator": { + "aggregation": { + "alignmentPeriod": "60s", + "crossSeriesReducer": "REDUCE_SUM", + "groupByFields": [ + "resource.label.\"method\"" + ], + "perSeriesAligner": "ALIGN_RATE" + }, + "filter": "resource.type=\"consumed_api\" AND metric.type=\"serviceruntime.googleapis.com/api/request_count\" AND project=\"unoperate-test\" AND resource.labels.service=\"bigtable.googleapis.com\" AND (metric.labels.response_code_class=one_of(\"4xx\", \"5xx\"))" + } + }, + "unitOverride": "" + } + } + ], + "thresholds": [], + "yAxis": { + "label": "", + "scale": "LINEAR" + } + } + } + }, + { + "yPos": 80, + "xPos": 24, + "height": 16, + "width": 24, + "widget": { + "title": "Kafka Connect Kubernetes Limits", + "id": "", + "xyChart": { + "chartOptions": { + "displayHorizontal": false, + "mode": "COLOR", + "showLegend": false + }, + "dataSets": [ + { + "breakdowns": [], + "dimensions": [], + "legendTemplate": "", + "measures": [], + "minAlignmentPeriod": "60s", + "plotType": "LINE", + "targetAxis": "Y1", + "timeSeriesQuery": { + "outputFullDuration": false, + "timeSeriesFilter": { + "aggregation": { + "alignmentPeriod": "60s", + "groupByFields": [], + "perSeriesAligner": "ALIGN_MEAN" + }, + "filter": "metric.type=\"kubernetes.io/container/cpu/limit_utilization\" resource.type=\"k8s_container\" resource.label.\"container_name\"=\"my-connect-cluster-connect\"" + }, + "unitOverride": "" + } + }, + { + "breakdowns": [], + "dimensions": [], + "legendTemplate": "", + "measures": [], + "minAlignmentPeriod": "60s", + "plotType": "LINE", + "targetAxis": "Y1", + "timeSeriesQuery": { + "outputFullDuration": false, + "timeSeriesFilter": { + "aggregation": { + "alignmentPeriod": "60s", + "groupByFields": [], + "perSeriesAligner": "ALIGN_MEAN" + }, + "filter": "metric.type=\"kubernetes.io/container/memory/limit_utilization\" resource.type=\"k8s_container\" resource.label.\"container_name\"=\"my-connect-cluster-connect\"" + }, + "unitOverride": "" + } + } + ], + "thresholds": [], + "yAxis": { + "label": "", + "scale": "LINEAR" + } + } + } + }, + { + "yPos": 96, + "height": 16, + "width": 24, + "widget": { + "title": "Kafka Connect #Errors", + "id": "", + "xyChart": { + "chartOptions": { + "displayHorizontal": false, + "mode": "COLOR", + "showLegend": false + }, + "dataSets": [ + { + "breakdowns": [], + "dimensions": [], + "legendTemplate": "", + "measures": [], + "minAlignmentPeriod": "60s", + "plotType": "LINE", + "targetAxis": "Y1", + "timeSeriesQuery": { + "outputFullDuration": false, + "timeSeriesFilter": { + "aggregation": { + "alignmentPeriod": "60s", + "crossSeriesReducer": "REDUCE_SUM", + "groupByFields": [], + "perSeriesAligner": "ALIGN_MEAN" + }, + "filter": "metric.type=\"prometheus.googleapis.com/kafka_connect_task_error_total_record_errors/gauge\" resource.type=\"prometheus_target\"" + }, + "unitOverride": "" + } + }, + { + "breakdowns": [], + "dimensions": [], + "legendTemplate": "", + "measures": [], + "minAlignmentPeriod": "60s", + "plotType": "LINE", + "targetAxis": "Y1", + "timeSeriesQuery": { + "outputFullDuration": false, + "timeSeriesFilter": { + "aggregation": { + "alignmentPeriod": "60s", + "crossSeriesReducer": "REDUCE_SUM", + "groupByFields": [], + "perSeriesAligner": "ALIGN_MEAN" + }, + "filter": "metric.type=\"prometheus.googleapis.com/kafka_connect_task_error_total_record_failures/gauge\" resource.type=\"prometheus_target\"" + }, + "unitOverride": "" + } + }, + { + "breakdowns": [], + "dimensions": [], + "legendTemplate": "", + "measures": [], + "minAlignmentPeriod": "60s", + "plotType": "LINE", + "targetAxis": "Y1", + "timeSeriesQuery": { + "outputFullDuration": false, + "timeSeriesFilter": { + "aggregation": { + "alignmentPeriod": "60s", + "crossSeriesReducer": "REDUCE_SUM", + "groupByFields": [], + "perSeriesAligner": "ALIGN_MEAN" + }, + "filter": "metric.type=\"prometheus.googleapis.com/kafka_connect_task_error_total_errors_logged/gauge\" resource.type=\"prometheus_target\"" + }, + "unitOverride": "" + } + } + ], + "thresholds": [], + "yAxis": { + "label": "", + "scale": "LINEAR" + } + } + } + }, + { + "yPos": 96, + "xPos": 24, + "height": 16, + "width": 24, + "widget": { + "title": "Kubernetes Limits", + "xyChart": { + "chartOptions": { + "displayHorizontal": false, + "mode": "COLOR" + }, + "dataSets": [ + { + "minAlignmentPeriod": "60s", + "plotType": "LINE", + "targetAxis": "Y1", + "timeSeriesQuery": { + "timeSeriesFilter": { + "aggregation": { + "crossSeriesReducer": "REDUCE_MAX", + "groupByFields": [ + "resource.label.\"pod_name\"", + "resource.label.\"container_name\"" + ], + "perSeriesAligner": "ALIGN_MAX" + }, + "filter": "metric.type=\"kubernetes.io/container/cpu/limit_utilization\" resource.type=\"k8s_container\"", + "pickTimeSeriesFilter": { + "direction": "TOP", + "numTimeSeries": 8, + "rankingMethod": "METHOD_MAX" + } + } + } + }, + { + "minAlignmentPeriod": "60s", + "plotType": "LINE", + "targetAxis": "Y1", + "timeSeriesQuery": { + "timeSeriesFilter": { + "aggregation": { + "crossSeriesReducer": "REDUCE_MAX", + "groupByFields": [], + "perSeriesAligner": "ALIGN_MAX" + }, + "filter": "metric.type=\"kubernetes.io/container/memory/limit_utilization\" resource.type=\"k8s_container\"" + } + } + } + ], + "thresholds": [], + "yAxis": { + "scale": "LINEAR" + } + } + } + }, + { + "yPos": 112, + "height": 16, + "width": 24, + "widget": { + "title": "Kafka Connect Up", + "id": "", + "xyChart": { + "chartOptions": { + "displayHorizontal": false, + "mode": "COLOR", + "showLegend": false + }, + "dataSets": [ + { + "breakdowns": [], + "dimensions": [], + "legendTemplate": "", + "measures": [], + "minAlignmentPeriod": "60s", + "plotType": "LINE", + "targetAxis": "Y1", + "timeSeriesQuery": { + "outputFullDuration": false, + "timeSeriesFilter": { + "aggregation": { + "alignmentPeriod": "60s", + "crossSeriesReducer": "REDUCE_MIN", + "groupByFields": [ + "metric.label.\"pod\"" + ], + "perSeriesAligner": "ALIGN_MIN" + }, + "filter": "metric.type=\"prometheus.googleapis.com/up/gauge\" resource.type=\"prometheus_target\" metric.label.\"pod\"=monitoring.regex.full_match(\".*\")" + }, + "unitOverride": "" + } + } + ], + "thresholds": [], + "yAxis": { + "label": "", + "scale": "LINEAR" + } + } + } + } + ] + } +} \ No newline at end of file diff --git a/google-cloud-bigtable-kafka-connect-sink/performance/enable_tracing.patch b/google-cloud-bigtable-kafka-connect-sink/performance/enable_tracing.patch new file mode 100644 index 0000000000..f23ef2925a --- /dev/null +++ b/google-cloud-bigtable-kafka-connect-sink/performance/enable_tracing.patch @@ -0,0 +1,69 @@ +diff --git a/google-cloud-bigtable-kafka-connect-sink/src/main/java/com/google/cloud/kafka/connect/bigtable/BigtableSinkTask.java b/google-cloud-bigtable-kafka-connect-sink/src/main/java/com/google/cloud/kafka/connect/bigtable/BigtableSinkTask.java +index 80bea508..3dcf2b27 100644 +--- a/google-cloud-bigtable-kafka-connect-sink/src/main/java/com/google/cloud/kafka/connect/bigtable/BigtableSinkTask.java ++++ b/google-cloud-bigtable-kafka-connect-sink/src/main/java/com/google/cloud/kafka/connect/bigtable/BigtableSinkTask.java +@@ -37,10 +37,13 @@ import com.google.cloud.kafka.connect.bigtable.mapping.KeyMapper; + import com.google.cloud.kafka.connect.bigtable.mapping.MutationData; + import com.google.cloud.kafka.connect.bigtable.mapping.MutationDataBuilder; + import com.google.cloud.kafka.connect.bigtable.mapping.ValueMapper; ++import com.google.cloud.kafka.connect.bigtable.tracing.MessageTracer; + import com.google.cloud.kafka.connect.bigtable.version.PackageMetadata; + import com.google.common.annotations.VisibleForTesting; + import com.google.common.collect.Lists; + import com.google.protobuf.ByteString; ++import io.opentelemetry.api.trace.Span; ++import io.opentelemetry.api.trace.StatusCode; + import java.util.ArrayList; + import java.util.Collection; + import java.util.HashMap; +@@ -75,6 +78,7 @@ public class BigtableSinkTask extends SinkTask { + private BigtableSchemaManager schemaManager; + @VisibleForTesting protected final Map> batchers; + @VisibleForTesting protected Logger logger = LoggerFactory.getLogger(BigtableSinkTask.class); ++ private Map spans; + + /** + * A default empty constructor. Initialization methods such as {@link BigtableSinkTask#start(Map)} +@@ -103,6 +107,8 @@ public class BigtableSinkTask extends SinkTask { + this.schemaManager = schemaManager; + this.context = context; + this.batchers = new HashMap<>(); ++ // TODO: use a Map with limited capacity when tracing prod code. ++ this.spans = new HashMap<>(); + } + + @Override +@@ -166,6 +172,15 @@ public class BigtableSinkTask extends SinkTask { + if (records.isEmpty()) { + return; + } ++ records.forEach( ++ r -> { ++ Span span = MessageTracer.getRecordSpan(r, "sinkProcessing"); ++ Span oldSpan = spans.put(r, span); ++ if (oldSpan != null) { ++ span.setStatus(StatusCode.ERROR, "Span not ended. Probably a previous put() failed."); ++ span.end(); ++ } ++ }); + + Map mutations = prepareRecords(records); + if (config.getBoolean(BigtableSinkTaskConfig.AUTO_CREATE_TABLES_CONFIG)) { +@@ -494,11 +509,16 @@ public class BigtableSinkTask extends SinkTask { + void handleResults(Map> perRecordResults) { + logger.trace("handleResults(#records={})", perRecordResults.size()); + for (Map.Entry> recordResult : perRecordResults.entrySet()) { ++ SinkRecord record = recordResult.getKey(); ++ Optional span = Optional.ofNullable(spans.remove(record)); + try { + recordResult.getValue().get(); ++ span.map(s -> s.setStatus(StatusCode.OK)); + } catch (ExecutionException | InterruptedException e) { +- SinkRecord record = recordResult.getKey(); ++ span.map(s -> s.setStatus(StatusCode.ERROR, e.getClass().getName())); + reportError(record, e); ++ } finally { ++ span.ifPresent(Span::end); + } + } + } diff --git a/google-cloud-bigtable-kafka-connect-sink/performance/kubernetes/kafka-connect-managed-prometheus.yaml b/google-cloud-bigtable-kafka-connect-sink/performance/kubernetes/kafka-connect-managed-prometheus.yaml new file mode 100644 index 0000000000..6340763aed --- /dev/null +++ b/google-cloud-bigtable-kafka-connect-sink/performance/kubernetes/kafka-connect-managed-prometheus.yaml @@ -0,0 +1,11 @@ +apiVersion: monitoring.googleapis.com/v1 +kind: PodMonitoring +metadata: + name: kafka-connect-managed-prometheus +spec: + selector: + matchLabels: + app.kubernetes.io/name: kafka-connect + endpoints: + - port: tcp-prometheus + interval: 30s diff --git a/google-cloud-bigtable-kafka-connect-sink/performance/kubernetes/otlp/1_configmap.yaml b/google-cloud-bigtable-kafka-connect-sink/performance/kubernetes/otlp/1_configmap.yaml index b8168a90c1..2bbc6a25bf 100644 --- a/google-cloud-bigtable-kafka-connect-sink/performance/kubernetes/otlp/1_configmap.yaml +++ b/google-cloud-bigtable-kafka-connect-sink/performance/kubernetes/otlp/1_configmap.yaml @@ -22,6 +22,10 @@ data: user_agent: Google-Cloud-OTLP manifests:0.1.0 otel/opentelemetry-collector-contrib:0.118.0 googlemanagedprometheus: user_agent: Google-Cloud-OTLP manifests:0.1.0 otel/opentelemetry-collector-contrib:0.118.0 + debug: + verbosity: detailed + sampling_initial: 1000 + sampling_thereafter: 1 extensions: health_check: @@ -110,6 +114,10 @@ data: - http://* - https://* endpoint: ${env:MY_POD_IP}:4318 + otlp/debug: + protocols: + grpc: + endpoint: ${env:MY_POD_IP}:43170 otlp/self-metrics: protocols: grpc: @@ -162,6 +170,16 @@ data: - batch receivers: - otlp + traces/debug: + exporters: + - debug + processors: + - k8sattributes + - memory_limiter + - resourcedetection + - batch + receivers: + - otlp/debug telemetry: logs: encoding: json diff --git a/google-cloud-bigtable-kafka-connect-sink/performance/kubernetes/otlp/4_deployment.yaml b/google-cloud-bigtable-kafka-connect-sink/performance/kubernetes/otlp/4_deployment.yaml index dbec84a661..9c96e93ca5 100644 --- a/google-cloud-bigtable-kafka-connect-sink/performance/kubernetes/otlp/4_deployment.yaml +++ b/google-cloud-bigtable-kafka-connect-sink/performance/kubernetes/otlp/4_deployment.yaml @@ -54,10 +54,10 @@ spec: fieldPath: status.podIP resources: requests: - memory: "200Mi" + memory: "400Mi" cpu: "200m" limits: - memory: "400Mi" + memory: "800Mi" volumeMounts: - name: collector-config mountPath: /conf diff --git a/google-cloud-bigtable-kafka-connect-sink/performance/kubernetes/otlp/5_hpa.yaml b/google-cloud-bigtable-kafka-connect-sink/performance/kubernetes/otlp/5_hpa.yaml index 2aede1fd71..42386630c8 100644 --- a/google-cloud-bigtable-kafka-connect-sink/performance/kubernetes/otlp/5_hpa.yaml +++ b/google-cloud-bigtable-kafka-connect-sink/performance/kubernetes/otlp/5_hpa.yaml @@ -22,7 +22,7 @@ spec: apiVersion: apps/v1 kind: Deployment name: opentelemetry-collector - minReplicas: 1 + minReplicas: 6 maxReplicas: 10 metrics: - type: Resource diff --git a/google-cloud-bigtable-kafka-connect-sink/performance/terraform/.gitignore b/google-cloud-bigtable-kafka-connect-sink/performance/terraform/.gitignore new file mode 100644 index 0000000000..7fa95564c8 --- /dev/null +++ b/google-cloud-bigtable-kafka-connect-sink/performance/terraform/.gitignore @@ -0,0 +1,6 @@ +.terraform +*terraform.tfstate* +# Directory needed for building the Docker image. +bigtable-sink +# These files are generated by terraform +*.yaml diff --git a/google-cloud-bigtable-kafka-connect-sink/performance/terraform/.terraform.lock.hcl b/google-cloud-bigtable-kafka-connect-sink/performance/terraform/.terraform.lock.hcl new file mode 100644 index 0000000000..0eb1686daf --- /dev/null +++ b/google-cloud-bigtable-kafka-connect-sink/performance/terraform/.terraform.lock.hcl @@ -0,0 +1,42 @@ +# This file is maintained automatically by "terraform init". +# Manual edits may be lost in future updates. + +provider "registry.terraform.io/hashicorp/google" { + version = "6.19.0" + constraints = "~> 6.19" + hashes = [ + "h1:Mug/ens8Xqf8uyAL249htDwsfWiDbzeJROygk7BXMkc=", + "zh:32ffeedd1131d81f290f4e4213b948c221f40d6b3b5e54b22781c2ed6e8ad3a5", + "zh:40b388e0356d849e6bf3f03be950f6bb7fa3e26a9a85977662f58f693a93901b", + "zh:5775262765dd66dae4886f3e8c85a39d4954892e352ce077e21f9310ffd6a9f4", + "zh:5e4626bdde902e35c97179a421fb303fa76e08ef89271275a40468d26f93a05e", + "zh:5fcc9482ec0b697f0d2223117f770e7eb6146837bf5d509b1ed59ae24ffd623f", + "zh:829e8bb61e4ac47e70138f7a381e9f0e6e51dc76d4f373fd0eb2da7d9d3d5968", + "zh:8baeab5b3bcafb03cb567302df6047580300b5881de9694c3fa40f4b6f6bf714", + "zh:a6a800d89e3dcbdcb5ba8bd87e981a1336785cdb206610581ba072828b2a83c4", + "zh:bf7620009f0a1b89756f2aa2a748db8bd0683947f26bb1999ba1c8c6479d1149", + "zh:d93b4202012672becf64473775330928c3cdf4471eb3db4502e7cca1badfbe6a", + "zh:dbeae2a0a11062d285964c0360c8f090cf260d163823c0f237396ce64900fb43", + "zh:f569b65999264a9416862bca5cd2a6177d94ccb0424f3a4ef424428912b9cb3c", + ] +} + +provider "registry.terraform.io/hashicorp/kubernetes" { + version = "2.35.1" + constraints = "~> 2.35" + hashes = [ + "h1:Av0Wk8g2XjY2oap7nyWNHEgfCRfphdJvrkqJjEM2ZKM=", + "zh:12212ca5ae47823ce14bfafb909eeb6861faf1e2435fb2fc4a8b334b3544b5f5", + "zh:3f49b3d77182df06b225ab266667de69681c2e75d296867eb2cf06a8f8db768c", + "zh:40832494d19f8a2b3cd0c18b80294d0b23ef6b82f6f6897b5fe00248a9997460", + "zh:739a5ddea61a77925ee7006a29c8717377a2e9d0a79a0bbd98738d92eec12c0d", + "zh:a02b472021753627c5c39447a56d125a32214c29ff9108fc499f2dcdf4f1cc4f", + "zh:b78865b3867065aa266d6758c9601a2756741478f5735a838c20d633d65e085b", + "zh:d362e87464683f5632790e66920ea803adb54c2bc0cb24b6fd9a314d2b1efffd", + "zh:d98206fe88c2c9a52b8d2d0cb2c877c812a4a51d19f9d8428e63cbd5fd8a304d", + "zh:dfa320946b1ce3f3615c42b3447a28dc9f604c06d8b9a6fe289855ab2ade4d11", + "zh:f569b65999264a9416862bca5cd2a6177d94ccb0424f3a4ef424428912b9cb3c", + "zh:fc1debd2e695b5222d2ccc8b24dab65baba4ee2418ecce944e64d42e79474cb5", + "zh:fdaf960443720a238c09e519aeb30faf74f027ac5d1e0a309c3b326888e031d7", + ] +} diff --git a/google-cloud-bigtable-kafka-connect-sink/performance/terraform/Dockerfile b/google-cloud-bigtable-kafka-connect-sink/performance/terraform/Dockerfile new file mode 100644 index 0000000000..f0ea6ebcac --- /dev/null +++ b/google-cloud-bigtable-kafka-connect-sink/performance/terraform/Dockerfile @@ -0,0 +1,17 @@ +FROM rust:1.84-bullseye AS build +WORKDIR /root +COPY main.rs main.rs +RUN rustc main.rs -o main + +# https://quay.io/repository/strimzi/kafka/manifest/sha256:752d3dcf2bb117b617705488b56e1102a24c33836ac257acb7da63d0a9d01759 +# Keep it in sync with Kafka version as described in kubernetes.tf. +FROM quay.io/strimzi/kafka:0.45.0-kafka-3.8.1 +USER root:root +RUN rpm -i https://dl.fedoraproject.org/pub/epel/9/Everything/x86_64/Packages/p/pv-1.6.20-1.el9.x86_64.rpm +RUN curl -L https://github.com/open-telemetry/opentelemetry-java-instrumentation/releases/download/v2.12.0/opentelemetry-javaagent.jar > /opt/kafka/libs/opentelemetry-javaagent.jar +COPY bigtable-sink /opt/kafka/plugins/bigtable-sink +# Needed to force the connector to use opentelemetry jars from the host. +RUN rm -rf /opt/kafka/plugins/bigtable-sink/sink-1.0.0-SNAPSHOT-package/opentelemetry* +COPY --from=build /root/main /usr/bin/kafka-console-generator +COPY generate_load.sh /usr/bin/generate_load.sh +USER 1001 diff --git a/google-cloud-bigtable-kafka-connect-sink/performance/terraform/bigtable.tf b/google-cloud-bigtable-kafka-connect-sink/performance/terraform/bigtable.tf new file mode 100644 index 0000000000..61a3756a48 --- /dev/null +++ b/google-cloud-bigtable-kafka-connect-sink/performance/terraform/bigtable.tf @@ -0,0 +1,50 @@ +locals { + cluster_id = "${local.name}-cluster" + bigtable_table_name = "${local.name}-table" + bigtable_default_column_family = "default_column_family" + bigtable_column_families = 21 + // Note that it must match `main.rs`'s prefix for column families. + bigtable_column_family_prefix = "cf" +} + +resource "google_bigtable_instance" "bigtable" { + name = "${local.name}-instance" + deletion_protection = false + + cluster { + cluster_id = local.cluster_id + num_nodes = 1 + storage_type = "SSD" + zone = local.subregion + } + + depends_on = [google_project_service.bigtable] +} + +resource "google_bigtable_table" "table" { + name = local.bigtable_table_name + instance_name = google_bigtable_instance.bigtable.name + + dynamic "column_family" { + for_each = toset([for i in range(local.bigtable_column_families) : "${local.bigtable_column_family_prefix}${i}"]) + content { + family = column_family.value + } + } + + column_family { + family = local.bigtable_default_column_family + } +} + +resource "google_bigtable_app_profile" "profile" { + instance = google_bigtable_instance.bigtable.name + app_profile_id = "${local.name}-profile" + + single_cluster_routing { + cluster_id = local.cluster_id + allow_transactional_writes = true + } + + ignore_warnings = true +} diff --git a/google-cloud-bigtable-kafka-connect-sink/performance/terraform/container_registry.tf b/google-cloud-bigtable-kafka-connect-sink/performance/terraform/container_registry.tf new file mode 100644 index 0000000000..f5bd55bac4 --- /dev/null +++ b/google-cloud-bigtable-kafka-connect-sink/performance/terraform/container_registry.tf @@ -0,0 +1,9 @@ +locals { + kafka_connect_docker_registry_url = "${google_artifact_registry_repository.kafka_connect.location}-docker.pkg.dev/${google_artifact_registry_repository.kafka_connect.project}/${google_artifact_registry_repository.kafka_connect.name}/kafkaconnect" +} + +resource "google_artifact_registry_repository" "kafka_connect" { + location = local.region + repository_id = "${local.name}-kafka-connect" + format = "DOCKER" +} diff --git a/google-cloud-bigtable-kafka-connect-sink/performance/terraform/gcp_apis.tf b/google-cloud-bigtable-kafka-connect-sink/performance/terraform/gcp_apis.tf new file mode 100644 index 0000000000..fdcad953ba --- /dev/null +++ b/google-cloud-bigtable-kafka-connect-sink/performance/terraform/gcp_apis.tf @@ -0,0 +1,14 @@ +resource "google_project_service" "kubernetes" { + service = "container.googleapis.com" + disable_on_destroy = false +} + +resource "google_project_service" "kafka" { + service = "managedkafka.googleapis.com" + disable_on_destroy = false +} + +resource "google_project_service" "bigtable" { + service = "bigtable.googleapis.com" + disable_on_destroy = false +} diff --git a/google-cloud-bigtable-kafka-connect-sink/performance/terraform/generate_load.sh b/google-cloud-bigtable-kafka-connect-sink/performance/terraform/generate_load.sh new file mode 100755 index 0000000000..716948735a --- /dev/null +++ b/google-cloud-bigtable-kafka-connect-sink/performance/terraform/generate_load.sh @@ -0,0 +1,23 @@ +#!/bin/bash +set -euo pipefail + +CONFIG_FILE="$(mktemp)" +THROUGHPUT="$THROUGHPUT" +TIMEOUT="$TIMEOUT" +TOPIC="$TOPIC" +/opt/kafka/kafka_connect_config_generator.sh > "$CONFIG_FILE" + +# https://opentelemetry.io/docs/zero-code/java/agent/getting-started/#configuring-the-agent +export JAVA_TOOL_OPTIONS="-javaagent:/opt/kafka/libs/opentelemetry-javaagent.jar" +# export OTEL_JAVAAGENT_DEBUG=true +export OTEL_EXPORTER_OTLP_PROTOCOL=grpc +kafka-console-generator | pv --line-mode --rate-limit "$THROUGHPUT" | timeout -v -s TERM -k "$TIMEOUT" "$TIMEOUT" /opt/kafka/bin/kafka-console-producer.sh \ + --producer.config "$CONFIG_FILE" \ + --topic "$TOPIC" \ + --bootstrap-server "$KAFKA_CONNECT_BOOTSTRAP_SERVERS" \ + --property parse.key=true \ + --property key.separator="|" \ + --compression-codec none \ + --request-required-acks 1 \ + --timeout 0 \ + || true diff --git a/google-cloud-bigtable-kafka-connect-sink/performance/terraform/gke.tf b/google-cloud-bigtable-kafka-connect-sink/performance/terraform/gke.tf new file mode 100644 index 0000000000..dfca242a54 --- /dev/null +++ b/google-cloud-bigtable-kafka-connect-sink/performance/terraform/gke.tf @@ -0,0 +1,52 @@ +locals { + workload_identity_pool = "${data.google_project.project.project_id}.svc.id.goog" +} + +resource "google_service_account" "kubernetes" { + account_id = "${local.name}-kubernetes" + display_name = "${local.name} Kubernetes Service Account" +} + +resource "google_container_cluster" "kubernetes" { + name = "${local.name}-k8s-cluster" + location = local.region + + # We can't create a cluster with no node pool defined, but we want to only use + # separately managed node pools. So we create the smallest possible default + # node pool and immediately delete it. + remove_default_node_pool = true + initial_node_count = 1 + + workload_identity_config { + workload_pool = local.workload_identity_pool + } + + deletion_protection = false + depends_on = [google_project_service.kubernetes] +} + +resource "google_container_node_pool" "pool" { + name = "${local.name}-k8s-pool" + location = local.region + node_locations = [local.subregion] + cluster = google_container_cluster.kubernetes.name + # +1 for load generator and possibly other tools. + node_count = local.kafka_connect_nodes + 1 + + node_config { + preemptible = false + machine_type = "n1-standard-4" + + service_account = google_service_account.kubernetes.email + oauth_scopes = [ + "https://www.googleapis.com/auth/cloud-platform" + ] + workload_metadata_config { + mode = "GKE_METADATA" + } + } +} + +output "gke_cluster_name" { + value = google_container_cluster.kubernetes.name +} diff --git a/google-cloud-bigtable-kafka-connect-sink/performance/terraform/kafka.tf b/google-cloud-bigtable-kafka-connect-sink/performance/terraform/kafka.tf new file mode 100644 index 0000000000..0cb68bdb41 --- /dev/null +++ b/google-cloud-bigtable-kafka-connect-sink/performance/terraform/kafka.tf @@ -0,0 +1,32 @@ +locals { + kafka_url = "bootstrap.${google_managed_kafka_cluster.kafka.cluster_id}.${local.region}.managedkafka.${local.project}.cloud.goog:9092" + kafka_topic = "${local.name}-kafka-topic" +} + +resource "google_managed_kafka_cluster" "kafka" { + cluster_id = "${local.name}-kafka" + location = local.region + capacity_config { + vcpu_count = local.kafka_vcpus + memory_bytes = local.kafka_ram_gbs * 1073741824 + } + gcp_config { + access_config { + network_configs { + subnet = data.google_compute_subnetwork.default_subnet.id + } + } + } + depends_on = [google_project_service.kafka] +} + +resource "google_managed_kafka_topic" "topic" { + topic_id = local.kafka_topic + cluster = google_managed_kafka_cluster.kafka.cluster_id + location = local.region + partition_count = local.kafka_partitions + replication_factor = local.kafka_connect_nodes + configs = { + "cleanup.policy" = "compact" + } +} diff --git a/google-cloud-bigtable-kafka-connect-sink/performance/terraform/kubernetes.tf b/google-cloud-bigtable-kafka-connect-sink/performance/terraform/kubernetes.tf new file mode 100644 index 0000000000..63125fc1af --- /dev/null +++ b/google-cloud-bigtable-kafka-connect-sink/performance/terraform/kubernetes.tf @@ -0,0 +1,362 @@ +locals { + kubernetes_kafka_connect_service_key = base64decode(google_service_account_key.kubernetes_kafka_connect_key.private_key) + kafka_namespace = kubernetes_namespace.kafka.metadata[0].name + kafka_connect_crd_name = "my-connect-cluster" + kafka_secret_service_key_secret_field = "service_key" + kafka_service_key_email = jsondecode(local.kubernetes_kafka_connect_service_key)["client_email"] + # Must match contents of ../kubernetes/otlp + otel_endpoint_url = "http://opentelemetry-collector.opentelemetry.svc.cluster.local:4317" + otel_namespace = kubernetes_namespace.otel.metadata[0].name + + kafka_credentials_secret = kubernetes_secret.kafka_credentials.metadata[0].name + kafka_image = "${local.kafka_connect_docker_registry_url}:latest" + + # Test parameters, adjust them here. + kafka_vcpus = 3 + kafka_ram_gbs = 3 + kafka_partitions = 60 + kafka_connect_nodes = 3 + kafka_connect_vcpus = "2.66" + kafka_connect_ram = "6Gi" + kafka_connect_version = "3.8.1" + load_generation_messages_per_second = "3000" + load_generation_seconds = "600" + load_generation_field_value_size = "50" + load_generation_column_families = "1" + load_generation_columns_per_family = "2" + load_generation_vcpus = "2" + load_generation_ram = "8Gi" + connector_batch_size = "1000" +} + +resource "kubernetes_namespace" "kafka" { + metadata { + name = "kafka" + } +} + +resource "kubernetes_secret" "kafka_credentials" { + metadata { + name = "bigtable-kafka-connect-credentials" + namespace = local.kafka_namespace + } + + data = { + username = local.kafka_service_key_email + (local.kafka_secret_service_key_secret_field) = base64encode(local.kubernetes_kafka_connect_service_key) + } + + type = "generic" +} + +resource "kubernetes_service_account" "kafka_connect_service_account" { + metadata { + name = "${local.kafka_connect_crd_name}-connect" + namespace = local.kafka_namespace + } + # To avoid clashing with Strimzi Operator. + lifecycle { + ignore_changes = [metadata["labels"], automount_service_account_token] + } +} + +resource "google_project_iam_member" "kafka_connect_connectors_bigtable_permission" { + project = local.project + role = "roles/bigtable.admin" + member = "principal://iam.googleapis.com/projects/${data.google_project.project.number}/locations/global/workloadIdentityPools/${local.workload_identity_pool}/subject/ns/${local.kafka_namespace}/sa/${kubernetes_service_account.kafka_connect_service_account.metadata[0].name}" +} + +resource "kubernetes_namespace" "otel" { + metadata { + name = "opentelemetry" + } +} + +resource "kubernetes_service_account" "otel_service_account" { + metadata { + name = "opentelemetry-collector" + namespace = local.otel_namespace + } + # To avoid clashing with OTEL manifest. + lifecycle { + ignore_changes = [metadata["labels"], automount_service_account_token] + } +} + +# https://cloud.google.com/stackdriver/docs/instrumentation/opentelemetry-collector-gke#configure_permissions_for_the_collector +resource "google_project_iam_member" "otel_permissions" { + for_each = toset(["roles/logging.logWriter", "roles/monitoring.metricWriter", "roles/cloudtrace.agent"]) + + project = local.project + role = each.value + member = "principal://iam.googleapis.com/projects/${data.google_project.project.number}/locations/global/workloadIdentityPools/${local.workload_identity_pool}/subject/ns/${local.otel_namespace}/sa/${kubernetes_service_account.otel_service_account.metadata[0].name}" +} + +output "kubernetes_kafka_namespace" { + value = local.kafka_namespace +} + +output "kafka_connect_manifest" { + value = yamlencode({ + "apiVersion" = "kafka.strimzi.io/v1beta2" + "kind" = "KafkaConnect" + "metadata" = { + "annotations" = { + # So that we can configure the connector using the operator's custom resources. + "strimzi.io/use-connector-resources" = "true" + } + "name" = local.kafka_connect_crd_name + "namespace" = local.kafka_namespace + } + "spec" = { + "authentication" = { + "passwordSecret" = { + "password" = local.kafka_secret_service_key_secret_field + "secretName" = local.kafka_credentials_secret + } + "type" = "plain" + "username" = local.kafka_service_key_email + } + "bootstrapServers" = local.kafka_url + "config" = { + "config.storage.replication.factor" = -1 + "config.storage.topic" = "connect-cluster-configs" + "fetch.min.bytes" = 1 + "group.id" = "connect-cluster" + "key.converter" = "org.apache.kafka.connect.json.JsonConverter" + "key.converter.schemas.enable" = false + "offset.storage.replication.factor" = -1 + "offset.storage.topic" = "connect-cluster-offsets" + "status.storage.replication.factor" = -1 + "status.storage.topic" = "connect-cluster-status" + "value.converter" = "org.apache.kafka.connect.json.JsonConverter" + "value.converter.schemas.enable" = true + } + "image" = local.kafka_image + "logging" = { + "loggers" = { + "connect.root.logger.level" = "INFO" + "log4j.logger.com.google.cloud.kafka.connect.bigtable" = "INFO" + "log4j.logger.io.opentelemetry" = "TRACE" + } + "type" = "inline" + } + "metricsConfig" = { + "type" = "jmxPrometheusExporter" + "valueFrom" = { + "configMapKeyRef" = { + # They must match value from ../kubernetes/kafka-connec-metrics.config.yaml + "name" = "connect-metrics" + "key" = "metrics-config.yml" + } + } + } + "replicas" = local.kafka_connect_nodes + "resources" = { + "limits" = { + "cpu" = local.kafka_connect_vcpus + "memory" = local.kafka_connect_ram + } + } + "template" = { + "connectContainer" = { + "env" = [ + { + "name" = "OTEL_SERVICE_NAME" + "value" = "kafka-connect" + }, + { + "name" = "OTEL_EXPORTER_OTLP_ENDPOINT" + "value" = local.otel_endpoint_url + }, + # Set up GlobalOpenTelemetry in java code + { + "name" = "OTEL_JAVA_GLOBAL_AUTOCONFIGURE_ENABLED" + "value" = "true" + }, + { + "name" = "OTEL_EXPORTER_OTLP_PROTOCOL" + "value" = "grpc" + }, + { + "name" = "OTEL_TRACES_EXPORTER" + "value" = "otlp" + }, + { + "name" = "OTEL_JAVAAGENT_DEBUG" + "value" = "true" + } + ] + } + } + "tls" = { + "trustedCertificates" = [] + } + "tracing" = { + "type" = "opentelemetry" + } + "version" = local.kafka_connect_version + } + }) + sensitive = true +} + +output "kafka_connect_connector_manifest" { + value = yamlencode({ + "apiVersion" = "kafka.strimzi.io/v1beta2" + "kind" = "KafkaConnector" + "metadata" = { + "labels" = { + "strimzi.io/cluster" = local.kafka_connect_crd_name + } + "name" = "${local.name}-connector" + } + "spec" = { + "class" = "com.google.cloud.kafka.connect.bigtable.BigtableSinkConnector" + "config" = { + "auto.create.column.families" = "false", + "auto.create.tables" = "false", + "connector.class" = "com.google.cloud.kafka.connect.bigtable.BigtableSinkConnector", + "default.column.family" = local.bigtable_default_column_family, + "default.column.qualifier" = "default_column", + "error.mode" = "FAIL", + "gcp.bigtable.instance.id" = google_bigtable_instance.bigtable.name, + "gcp.bigtable.project.id" = local.project, + "insert.mode" = "upsert", + "max.batch.size" = local.connector_batch_size, + "retry.timeout.ms" = "90000", + "row.key.definition" = "", + "row.key.delimiter" = "#", + "table.name.format" = local.bigtable_table_name, + "topics" = local.kafka_topic, + "value.null.mode" = "write" + } + "tasksMax" = local.kafka_partitions + } + }) + sensitive = true +} + +output "load_generator_manifest" { + value = yamlencode({ + "apiVersion" = "v1" + "kind" = "Pod" + "metadata" = { + "name" = "load-generator" + "namespace" = local.kafka_namespace + } + "spec" = { + "containers" = [ + { + "args" = ["/usr/bin/generate_load.sh"] + "env" = [ + { + "name" = "KAFKA_CONNECT_BOOTSTRAP_SERVERS" + "value" = local.kafka_url + }, + { + "name" = "KAFKA_CONNECT_TLS" + "value" = "true" + }, + { + "name" = "KAFKA_CONNECT_SASL_USERNAME" + "value" = local.kafka_service_key_email + }, + { + "name" = "KAFKA_CONNECT_SASL_PASSWORD_FILE" + "value" = "${local.kafka_credentials_secret}/${local.kafka_secret_service_key_secret_field}" + }, + { + "name" = "KAFKA_CONNECT_SASL_MECHANISM" + "value" = "plain" + }, + # Load generation args + { + "name" = "THROUGHPUT" + "value" = local.load_generation_messages_per_second + }, + { + "name" = "TIMEOUT" + "value" = local.load_generation_seconds + }, + { + "name" = "TOPIC" + "value" = local.kafka_topic + }, + { + "name" = "FIELD_VALUE_SIZE" + "value" = local.load_generation_field_value_size + }, + { + "name" = "COLUMN_FAMILIES" + "value" = local.load_generation_column_families + }, + { + "name" = "COLUMNS_PER_FAMILY" + "value" = local.load_generation_columns_per_family + }, + # Tracing + { + "name" = "OTEL_SERVICE_NAME" + "value" = "load-generator" + }, + { + "name" = "OTEL_EXPORTER_OTLP_ENDPOINT" + "value" = local.otel_endpoint_url + }, + ] + "image" = local.kafka_image + "imagePullPolicy" = "Always" + "name" = "kafka-load-generator" + "resources" = { + "requests" = { + "cpu" = local.load_generation_vcpus + "memory" = local.load_generation_ram + } + } + "volumeMounts" = [ + { + "mountPath" = "/opt/kafka/connect-password/${local.kafka_credentials_secret}" + "name" = local.kafka_credentials_secret + }, + ] + }, + ] + "restartPolicy" = "Never" + "volumes" = [ + { + "name" = local.kafka_credentials_secret + "secret" = { + "defaultMode" = 292 # 0x124 + "secretName" = local.kafka_credentials_secret + } + }, + ] + } + }) + sensitive = true +} + +output "kafka_connect_docker_registry_url" { + value = local.kafka_connect_docker_registry_url +} + +output "perf_test_config" { + value = { + kafka_vcpus = local.kafka_vcpus + kafka_ram_gbs = local.kafka_ram_gbs + kafka_partitions = local.kafka_partitions + kafka_connect_nodes = local.kafka_connect_nodes + kafka_connect_vcpus = local.kafka_connect_vcpus + kafka_connect_ram = local.kafka_connect_ram + kafka_connect_version = local.kafka_connect_version + load_generation_messages_per_second = local.load_generation_messages_per_second + load_generation_seconds = local.load_generation_seconds + load_generation_field_value_size = local.load_generation_field_value_size + load_generation_column_families = local.load_generation_column_families + load_generation_columns_per_family = local.load_generation_columns_per_family + load_generation_vcpus = local.load_generation_vcpus + load_generation_ram = local.load_generation_ram + connector_batch_size = local.connector_batch_size + } +} diff --git a/google-cloud-bigtable-kafka-connect-sink/performance/terraform/main.rs b/google-cloud-bigtable-kafka-connect-sink/performance/terraform/main.rs new file mode 100644 index 0000000000..a08d26ebfd --- /dev/null +++ b/google-cloud-bigtable-kafka-connect-sink/performance/terraform/main.rs @@ -0,0 +1,96 @@ +use std::env; + +const FIELD_VALUE_SIZE_ENV: &str = "FIELD_VALUE_SIZE"; +const COLUMN_FAMILIES_ENV: &str = "COLUMN_FAMILIES"; +const COLUMNS_PER_FAMILI_ENV: &str = "COLUMNS_PER_FAMILY"; + +const DELIMITER: &str = "|"; +const VALUE_CHAR: &str = "0"; + +fn main() { + let field_value_size: usize = get_env_var_u64(FIELD_VALUE_SIZE_ENV) as usize; + let column_families: u64 = get_env_var_u64(COLUMN_FAMILIES_ENV); + let columns_per_family: u64 = get_env_var_u64(COLUMNS_PER_FAMILI_ENV); + + let field_value: String = VALUE_CHAR.repeat(field_value_size); + + let (value_schema, value) = if column_families == 0 { + string_schema_and_value(field_value) + } else { + struct_schema_and_value(column_families, columns_per_family, field_value) + }; + let schema_and_value: String = format!("{{\"schema\":{},\"payload\":{}}}", value_schema, value); + let mut i: u64 = 0; + loop { + println!("\"{}\"{}{}", i, DELIMITER, schema_and_value); + i += 1; + } +} + +fn get_env_var_u64(name: &str) -> u64 { + let missing_error: String = format!("Missing env var: {}.", name); + let invalid_error: String = format!("Non-u64 env var: {}.", name); + env::var(name) + .expect(&missing_error) + .parse::() + .expect(&invalid_error) +} + +fn struct_schema_and_value( + column_families: u64, + columns_per_family: u64, + field_value: String, +) -> (String, String) { + let columns: Vec = (1..=columns_per_family) + .map(|i| format!("c{}", i)) + .collect(); + let column_families: Vec = (1..=column_families).map(|i| format!("cf{}", i)).collect(); + let column_family_schemas: Vec = column_families + .iter() + .map(|cf| { + format!( + "{{\"type\":\"struct\",\"optional\":true,\"field\":\"{}\",\"fields\":[{}]}}", + cf, + columns + .iter() + .map(|c| format!( + "{{\"type\":\"string\",\"optional\":true,\"field\":\"{}\"}}", + c + )) + .collect::>() + .join(",") + ) + }) + .collect(); + let schema: String = format!( + "{{\"name\":\"record\",\"type\":\"struct\",\"optional\":true,\"fields\":[{}]}}", + column_family_schemas.join(",") + ); + let value: String = format!( + "{{{}}}", + column_families + .iter() + .map(|cf| { + format!( + "\"{}\":{{{}}}", + cf, + columns + .iter() + .map(|c| format!("\"{}\":\"{}\"", c, field_value)) + .collect::>() + .join(",") + ) + }) + .collect::>() + .join(",") + ); + + (schema, value) +} + +fn string_schema_and_value(field_value: String) -> (String, String) { + ( + "{\"type\":\"string\",\"optional\":true}".to_string(), + format!("\"{}\"", field_value), + ) +} diff --git a/google-cloud-bigtable-kafka-connect-sink/performance/terraform/main.tf b/google-cloud-bigtable-kafka-connect-sink/performance/terraform/main.tf new file mode 100644 index 0000000000..dd5f8548a5 --- /dev/null +++ b/google-cloud-bigtable-kafka-connect-sink/performance/terraform/main.tf @@ -0,0 +1,17 @@ +locals { + project = "unoperate-test" + region = "europe-central2" + subregion = "europe-central2-a" + name = "bigtablesink" +} + +data "google_compute_subnetwork" "default_subnet" { + name = "default" + region = local.region +} + +data "google_project" "project" {} + +output "region" { + value = local.region +} diff --git a/google-cloud-bigtable-kafka-connect-sink/performance/terraform/perf_test.sh b/google-cloud-bigtable-kafka-connect-sink/performance/terraform/perf_test.sh new file mode 100755 index 0000000000..e05296c1d8 --- /dev/null +++ b/google-cloud-bigtable-kafka-connect-sink/performance/terraform/perf_test.sh @@ -0,0 +1,136 @@ +#!/bin/bash + +set -euo pipefail + +KAFKA_CONNECT_CR_PATH="kafka-connect.yaml" +KAFKA_CONNECT_CONNECTOR_CR_PATH="kafka-connect-connector.yaml" +LOAD_GENERATOR_MANIFEST_PATH="load-generator.yaml" +# Must match `Dockerfile` +BIGTABLE_PLUGIN_PATH=bigtable-sink + +print_delimiter() { + yes "=" | head -80 | tr -d '\n' || : + echo +} + +print_timestamp() { + date --utc --iso-8601=seconds +} + +do_prepare() { + terraform apply -auto-approve + TERRAFORM_OUTPUT="$(terraform output -json)" + KAFKA_CONNECT_DOCKER_REPO="$(echo "$TERRAFORM_OUTPUT" | jq -rc .kafka_connect_docker_registry_url.value)" + KAFKA_NAMESPACE="$(echo "$TERRAFORM_OUTPUT" | jq -rc .kubernetes_kafka_namespace.value)" + GKE_CLUSTER_NAME="$(echo "$TERRAFORM_OUTPUT" | jq -rc .gke_cluster_name.value)" + GCP_REGION="$(echo "$TERRAFORM_OUTPUT" | jq -rc .region.value)" + + gcloud container clusters get-credentials "$GKE_CLUSTER_NAME" "--region=$GCP_REGION" + gcloud auth configure-docker "$GCP_REGION-docker.pkg.dev" + + pushd ../.. + mvn clean package -Dmaven.test.skip + popd + rm -rf --preserve-root $BIGTABLE_PLUGIN_PATH || true + mkdir -p $BIGTABLE_PLUGIN_PATH + cp -rf ../../target/sink-*-SNAPSHOT{.jar,-package} $BIGTABLE_PLUGIN_PATH + + docker build . -t "$KAFKA_CONNECT_DOCKER_REPO" --push + + # As per https://cloud.google.com/stackdriver/docs/instrumentation/opentelemetry-collector-gke#deploy_the_collector + # For whathever reason, the GCLOUD_PROJECT seems not to be used. + GCLOUD_PROJECT="$(gcloud config get-value project)" + export GCLOUD_PROJECT + kubectl kustomize ../kubernetes/otlp | envsubst | kubectl apply -f - + + kubectl apply -f ../kubernetes/strimzi-cluster-operator-0.45.0.yaml + kubectl apply -f ../kubernetes/kafka-connect-metrics-config.yaml -n "$KAFKA_NAMESPACE" + kubectl apply -f ../kubernetes/kafka-connect-managed-prometheus.yaml -n "$KAFKA_NAMESPACE" +} + +do_run() { + terraform apply -auto-approve + TERRAFORM_OUTPUT="$(terraform output -json)" + KAFKA_NAMESPACE="$(echo "$TERRAFORM_OUTPUT" | jq -rc .kubernetes_kafka_namespace.value)" + CONNECTOR_TASKS="$(echo "$TERRAFORM_OUTPUT" | jq -rc .perf_test_config.value.kafka_partitions)" + + echo "$TERRAFORM_OUTPUT" | jq -rc .kafka_connect_manifest.value > $KAFKA_CONNECT_CR_PATH + echo "$TERRAFORM_OUTPUT" | jq -rc .kafka_connect_connector_manifest.value > $KAFKA_CONNECT_CONNECTOR_CR_PATH + echo "$TERRAFORM_OUTPUT" | jq -rc .load_generator_manifest.value > $LOAD_GENERATOR_MANIFEST_PATH + + kubectl apply -f $KAFKA_CONNECT_CR_PATH -n "$KAFKA_NAMESPACE" + kubectl apply -f $KAFKA_CONNECT_CONNECTOR_CR_PATH -n "$KAFKA_NAMESPACE" + + echo "Waiting for all connector tasks to be running..." + while : + do + RUNNING="$(kubectl describe KafkaConnector -n "$KAFKA_NAMESPACE" | { grep -c RUNNING || :; })" + # `gt` rather than `ge` since there is one extra "RUNNING" comming from the connector itself + if [[ "$RUNNING" -gt "$CONNECTOR_TASKS" ]] ; then + break + fi + # A bit untrue, in reality `$RUNNING - 1` tasks are running - one RUNNING comes from the connector. + # Still, it is good enough - we're probably running tens of tasks. + printf "Only %s out of %s tasks are running. Sleeping...\n" "$RUNNING" "$CONNECTOR_TASKS" + sleep 10 + done + + kubectl apply -f $LOAD_GENERATOR_MANIFEST_PATH -n "$KAFKA_NAMESPACE" + + print_delimiter + echo "Perf test config:" + echo "$TERRAFORM_OUTPUT" | jq .perf_test_config.value + echo "Start time" + print_timestamp + echo "Waiting for the load generator to be done..." + kubectl wait --for=condition=ready=True --timeout=-1s pod/load-generator -n "$KAFKA_NAMESPACE" + kubectl wait --for=condition=ready=False --timeout=-1s pod/load-generator -n "$KAFKA_NAMESPACE" + echo "Load generator done." + echo "End time" + print_timestamp + print_delimiter +} + +do_cleanup() { + kubectl delete --ignore-not-found=true -f $KAFKA_CONNECT_CR_PATH -f $KAFKA_CONNECT_CONNECTOR_CR_PATH -f $LOAD_GENERATOR_MANIFEST_PATH -n kafka + # Let's give time to the connector to spin down since Kafka Connect might otherwise recreate the topic. + sleep 20 + terraform destroy -auto-approve -target google_bigtable_table.table -target google_managed_kafka_topic.topic +} + +do_destroy() { + # Some operator leaves some resources preventing destroy from working. Let's just ignore it and manually remove problematic resources. + # They will disappear once the whole kubernetes cluster is destroyed. + terraform state rm kubernetes_namespace.otel + terraform state rm kubernetes_namespace.kafka + # No auto-approve since it takes 15-20 minutes to be recreated. + terraform destroy +} + +usage() { + printf "USAGE: %s [prepare|run|cleanup|destroy|rerun]" "$0" + exit 1 +} + +[[ $# -eq 1 ]] || usage +case "$1" in + prepare) + do_prepare + ;; + run) + do_run + ;; + rerun) + do_cleanup + do_run + ;; + cleanup) + do_cleanup + ;; + destroy) + do_destroy + ;; + *) + usage + ;; +esac diff --git a/google-cloud-bigtable-kafka-connect-sink/performance/terraform/providers.tf b/google-cloud-bigtable-kafka-connect-sink/performance/terraform/providers.tf new file mode 100644 index 0000000000..9083ab49a8 --- /dev/null +++ b/google-cloud-bigtable-kafka-connect-sink/performance/terraform/providers.tf @@ -0,0 +1,19 @@ +provider "google" { + project = local.project + region = local.region +} + +# As per https://registry.terraform.io/providers/hashicorp/google/latest/docs/guides/using_gke_with_terraform +data "google_client_config" "provider" {} + +provider "kubernetes" { + host = "https://${google_container_cluster.kubernetes.endpoint}" + token = data.google_client_config.provider.access_token + cluster_ca_certificate = base64decode( + google_container_cluster.kubernetes.master_auth[0].cluster_ca_certificate, + ) + exec { + api_version = "client.authentication.k8s.io/v1beta1" + command = "gke-gcloud-auth-plugin" + } +} diff --git a/google-cloud-bigtable-kafka-connect-sink/performance/terraform/service_account.tf b/google-cloud-bigtable-kafka-connect-sink/performance/terraform/service_account.tf new file mode 100644 index 0000000000..ee125de513 --- /dev/null +++ b/google-cloud-bigtable-kafka-connect-sink/performance/terraform/service_account.tf @@ -0,0 +1,33 @@ +resource "google_service_account" "kubernetes_kafka_connect" { + account_id = "${local.name}-k8s-kafka-connect" + display_name = "${local.name}-k8s-kafka-connect-SA" +} + +resource "google_project_iam_member" "kubernetes_kafka_connect_permissions" { + for_each = toset([ + "roles/managedkafka.admin", + ]) + + project = local.project + role = each.key + member = "serviceAccount:${google_service_account.kubernetes_kafka_connect.email}" +} + +resource "google_service_account_key" "kubernetes_kafka_connect_key" { + service_account_id = google_service_account.kubernetes_kafka_connect.name +} + +# https://cloud.google.com/kubernetes-engine/docs/troubleshooting/dashboards#write_permissions +resource "google_project_iam_member" "gke_monitoring" { + for_each = toset(["roles/monitoring.metricWriter", "roles/monitoring.editor", "roles/logging.logWriter", "roles/stackdriver.resourceMetadata.writer"]) + project = local.project + role = each.value + member = "serviceAccount:${google_service_account.kubernetes.email}" +} + +resource "google_project_iam_member" "kafka_connect_registry" { + project = local.project + role = "roles/artifactregistry.reader" + member = "serviceAccount:${google_service_account.kubernetes.email}" +} + diff --git a/google-cloud-bigtable-kafka-connect-sink/performance/terraform/terraform.tf b/google-cloud-bigtable-kafka-connect-sink/performance/terraform/terraform.tf new file mode 100644 index 0000000000..324b5c2df6 --- /dev/null +++ b/google-cloud-bigtable-kafka-connect-sink/performance/terraform/terraform.tf @@ -0,0 +1,12 @@ +terraform { + required_providers { + google = { + source = "hashicorp/google" + version = "~> 6.19" + } + kubernetes = { + source = "hashicorp/kubernetes" + version = "~> 2.35" + } + } +} From d4c1570e9c1fa895b91ceb87baf315021591d0d6 Mon Sep 17 00:00:00 2001 From: Adam Czajkowski Date: Tue, 25 Feb 2025 14:57:05 +0100 Subject: [PATCH 56/76] MultipleConnectorTasksIT#testRestartPauseStop --- .../integration/MultipleConnectorTasksIT.java | 61 +++++++++++++++++++ 1 file changed, 61 insertions(+) diff --git a/google-cloud-bigtable-kafka-connect-sink/src/test/java/com/google/cloud/kafka/connect/bigtable/integration/MultipleConnectorTasksIT.java b/google-cloud-bigtable-kafka-connect-sink/src/test/java/com/google/cloud/kafka/connect/bigtable/integration/MultipleConnectorTasksIT.java index 936f028967..188c8ffe7f 100644 --- a/google-cloud-bigtable-kafka-connect-sink/src/test/java/com/google/cloud/kafka/connect/bigtable/integration/MultipleConnectorTasksIT.java +++ b/google-cloud-bigtable-kafka-connect-sink/src/test/java/com/google/cloud/kafka/connect/bigtable/integration/MultipleConnectorTasksIT.java @@ -15,6 +15,9 @@ */ package com.google.cloud.kafka.connect.bigtable.integration; +import static org.junit.Assert.assertEquals; + +import com.google.cloud.bigtable.admin.v2.models.CreateTableRequest; import com.google.cloud.kafka.connect.bigtable.config.BigtableSinkConfig; import com.google.cloud.kafka.connect.bigtable.config.InsertMode; import io.confluent.connect.avro.AvroConverter; @@ -70,4 +73,62 @@ public void testMultipleTasks() throws InterruptedException { waitUntilBigtableContainsNumberOfRows(testId, numRecords); assertConnectorAndAllTasksAreRunning(testId); } + + @Test + public void testRestartPauseStop() throws InterruptedException { + numTasks = 10; + int expectedRowsInBigtable = 0; + String defaultColumnFamily = "default"; + String value = "1"; + Map connectorProps = baseConnectorProps(); + connectorProps.put(BigtableSinkConfig.DEFAULT_COLUMN_FAMILY_CONFIG, defaultColumnFamily); + String testId = startSingleTopicConnector(connectorProps); + bigtableAdmin.createTable(CreateTableRequest.of(testId).addFamily(defaultColumnFamily)); + + connect + .assertions() + .assertConnectorAndAtLeastNumTasksAreRunning(testId, numTasks, "Connector start timeout"); + assertEquals(expectedRowsInBigtable, readAllRows(bigtableData, testId).size()); + connect.kafka().produce(testId, "started", value); + expectedRowsInBigtable += 1; + waitUntilBigtableContainsNumberOfRows(testId, expectedRowsInBigtable); + + connect.restartConnectorAndTasks(testId, false, true, false); + connect + .assertions() + .assertConnectorAndAtLeastNumTasksAreRunning(testId, numTasks, "Connector restart timeout"); + assertEquals(expectedRowsInBigtable, readAllRows(bigtableData, testId).size()); + connect.kafka().produce(testId, "restarted", value); + expectedRowsInBigtable += 1; + waitUntilBigtableContainsNumberOfRows(testId, expectedRowsInBigtable); + + connect.pauseConnector(testId); + connect + .assertions() + .assertConnectorAndExactlyNumTasksArePaused(testId, numTasks, "Connector pause timeout"); + connect.resumeConnector(testId); + connect + .assertions() + .assertConnectorAndAtLeastNumTasksAreRunning( + testId, numTasks, "Connector post-pause resume timeout"); + assertEquals(expectedRowsInBigtable, readAllRows(bigtableData, testId).size()); + connect.kafka().produce(testId, "pause", value); + expectedRowsInBigtable += 1; + waitUntilBigtableContainsNumberOfRows(testId, expectedRowsInBigtable); + + connect.stopConnector(testId); + connect.assertions().assertConnectorIsStopped(testId, "Connector stop timeout"); + connect.resumeConnector(testId); + connect + .assertions() + .assertConnectorAndAtLeastNumTasksAreRunning( + testId, numTasks, "Connector post-stop resume timeout"); + assertEquals(expectedRowsInBigtable, readAllRows(bigtableData, testId).size()); + connect.kafka().produce(testId, "stop", value); + expectedRowsInBigtable += 1; + waitUntilBigtableContainsNumberOfRows(testId, expectedRowsInBigtable); + + connect.deleteConnector(testId); + connect.assertions().assertConnectorAndTasksAreNotRunning(testId, "Connector deletion timeout"); + } } From d2c8b100b5b0ccf0753de62851c0ebabfd462da6 Mon Sep 17 00:00:00 2001 From: Adam Czajkowski Date: Tue, 25 Feb 2025 14:59:18 +0100 Subject: [PATCH 57/76] Remove at-least assertions with exact ones --- .../bigtable/integration/BaseKafkaConnectIT.java | 4 ++-- .../kafka/connect/bigtable/integration/ConfigIT.java | 2 +- .../bigtable/integration/ConfluentCompatibilityIT.java | 2 +- .../bigtable/integration/DifferentConvertersIT.java | 2 +- .../bigtable/integration/MultipleConnectorTasksIT.java | 10 +++++----- .../connect/bigtable/integration/NullHandlingIT.java | 6 +++--- 6 files changed, 13 insertions(+), 13 deletions(-) diff --git a/google-cloud-bigtable-kafka-connect-sink/src/test/java/com/google/cloud/kafka/connect/bigtable/integration/BaseKafkaConnectIT.java b/google-cloud-bigtable-kafka-connect-sink/src/test/java/com/google/cloud/kafka/connect/bigtable/integration/BaseKafkaConnectIT.java index 2b07add438..6fadbd3899 100644 --- a/google-cloud-bigtable-kafka-connect-sink/src/test/java/com/google/cloud/kafka/connect/bigtable/integration/BaseKafkaConnectIT.java +++ b/google-cloud-bigtable-kafka-connect-sink/src/test/java/com/google/cloud/kafka/connect/bigtable/integration/BaseKafkaConnectIT.java @@ -112,7 +112,7 @@ public void startConnect() { try { connect .assertions() - .assertAtLeastNumWorkersAreUp(1, "Initial group of workers did not start in time."); + .assertExactlyNumWorkersAreUp(1, "Initial group of workers did not start in time."); } catch (InterruptedException e) { throw new RuntimeException(e); } @@ -174,7 +174,7 @@ private String startConnector(Map configProps, Set topic connect.configureConnector(id, configProps); connect .assertions() - .assertConnectorAndAtLeastNumTasksAreRunning(id, numTasks, "Connector start timeout"); + .assertConnectorAndExactlyNumTasksAreRunning(id, numTasks, "Connector start timeout"); return id; } diff --git a/google-cloud-bigtable-kafka-connect-sink/src/test/java/com/google/cloud/kafka/connect/bigtable/integration/ConfigIT.java b/google-cloud-bigtable-kafka-connect-sink/src/test/java/com/google/cloud/kafka/connect/bigtable/integration/ConfigIT.java index 45306295f5..9872562483 100644 --- a/google-cloud-bigtable-kafka-connect-sink/src/test/java/com/google/cloud/kafka/connect/bigtable/integration/ConfigIT.java +++ b/google-cloud-bigtable-kafka-connect-sink/src/test/java/com/google/cloud/kafka/connect/bigtable/integration/ConfigIT.java @@ -34,7 +34,7 @@ public void testBaseSuccess() throws InterruptedException { connect.configureConnector(connectorName, props); connect .assertions() - .assertConnectorAndAtLeastNumTasksAreRunning( + .assertConnectorAndExactlyNumTasksAreRunning( connectorName, numTasks, "Connector start timeout"); } diff --git a/google-cloud-bigtable-kafka-connect-sink/src/test/java/com/google/cloud/kafka/connect/bigtable/integration/ConfluentCompatibilityIT.java b/google-cloud-bigtable-kafka-connect-sink/src/test/java/com/google/cloud/kafka/connect/bigtable/integration/ConfluentCompatibilityIT.java index b123189a8f..7401f0ab64 100644 --- a/google-cloud-bigtable-kafka-connect-sink/src/test/java/com/google/cloud/kafka/connect/bigtable/integration/ConfluentCompatibilityIT.java +++ b/google-cloud-bigtable-kafka-connect-sink/src/test/java/com/google/cloud/kafka/connect/bigtable/integration/ConfluentCompatibilityIT.java @@ -201,7 +201,7 @@ public String startConnector(Map connectorProps) throws Interrup String topic = startSingleTopicConnector(connectorProps); connect .assertions() - .assertConnectorAndAtLeastNumTasksAreRunning(topic, numTasks, "Connector start timeout"); + .assertConnectorAndExactlyNumTasksAreRunning(topic, numTasks, "Connector start timeout"); return topic; } diff --git a/google-cloud-bigtable-kafka-connect-sink/src/test/java/com/google/cloud/kafka/connect/bigtable/integration/DifferentConvertersIT.java b/google-cloud-bigtable-kafka-connect-sink/src/test/java/com/google/cloud/kafka/connect/bigtable/integration/DifferentConvertersIT.java index 5d4549c6bb..86d7d5e96a 100644 --- a/google-cloud-bigtable-kafka-connect-sink/src/test/java/com/google/cloud/kafka/connect/bigtable/integration/DifferentConvertersIT.java +++ b/google-cloud-bigtable-kafka-connect-sink/src/test/java/com/google/cloud/kafka/connect/bigtable/integration/DifferentConvertersIT.java @@ -95,7 +95,7 @@ public void testConverter() throws InterruptedException { String topic = startSingleTopicConnector(connectorProps); connect .assertions() - .assertConnectorAndAtLeastNumTasksAreRunning(topic, numTasks, "Connector start timeout"); + .assertConnectorAndExactlyNumTasksAreRunning(topic, numTasks, "Connector start timeout"); populateKafkaTopic(topic, numRecords, keyConverter, valueConverter); waitUntilBigtableContainsNumberOfRows(topic, numRecords); diff --git a/google-cloud-bigtable-kafka-connect-sink/src/test/java/com/google/cloud/kafka/connect/bigtable/integration/MultipleConnectorTasksIT.java b/google-cloud-bigtable-kafka-connect-sink/src/test/java/com/google/cloud/kafka/connect/bigtable/integration/MultipleConnectorTasksIT.java index 188c8ffe7f..17ee4f64b6 100644 --- a/google-cloud-bigtable-kafka-connect-sink/src/test/java/com/google/cloud/kafka/connect/bigtable/integration/MultipleConnectorTasksIT.java +++ b/google-cloud-bigtable-kafka-connect-sink/src/test/java/com/google/cloud/kafka/connect/bigtable/integration/MultipleConnectorTasksIT.java @@ -67,7 +67,7 @@ public void testMultipleTasks() throws InterruptedException { String testId = startSingleTopicConnector(connectorProps); connect .assertions() - .assertConnectorAndAtLeastNumTasksAreRunning(testId, numTasks, "Connector start timeout"); + .assertConnectorAndExactlyNumTasksAreRunning(testId, numTasks, "Connector start timeout"); populateKafkaTopic(testId, numRecords, keyConverter, valueConverter); waitUntilBigtableContainsNumberOfRows(testId, numRecords); @@ -87,7 +87,7 @@ public void testRestartPauseStop() throws InterruptedException { connect .assertions() - .assertConnectorAndAtLeastNumTasksAreRunning(testId, numTasks, "Connector start timeout"); + .assertConnectorAndExactlyNumTasksAreRunning(testId, numTasks, "Connector start timeout"); assertEquals(expectedRowsInBigtable, readAllRows(bigtableData, testId).size()); connect.kafka().produce(testId, "started", value); expectedRowsInBigtable += 1; @@ -96,7 +96,7 @@ public void testRestartPauseStop() throws InterruptedException { connect.restartConnectorAndTasks(testId, false, true, false); connect .assertions() - .assertConnectorAndAtLeastNumTasksAreRunning(testId, numTasks, "Connector restart timeout"); + .assertConnectorAndExactlyNumTasksAreRunning(testId, numTasks, "Connector restart timeout"); assertEquals(expectedRowsInBigtable, readAllRows(bigtableData, testId).size()); connect.kafka().produce(testId, "restarted", value); expectedRowsInBigtable += 1; @@ -109,7 +109,7 @@ public void testRestartPauseStop() throws InterruptedException { connect.resumeConnector(testId); connect .assertions() - .assertConnectorAndAtLeastNumTasksAreRunning( + .assertConnectorAndExactlyNumTasksAreRunning( testId, numTasks, "Connector post-pause resume timeout"); assertEquals(expectedRowsInBigtable, readAllRows(bigtableData, testId).size()); connect.kafka().produce(testId, "pause", value); @@ -121,7 +121,7 @@ public void testRestartPauseStop() throws InterruptedException { connect.resumeConnector(testId); connect .assertions() - .assertConnectorAndAtLeastNumTasksAreRunning( + .assertConnectorAndExactlyNumTasksAreRunning( testId, numTasks, "Connector post-stop resume timeout"); assertEquals(expectedRowsInBigtable, readAllRows(bigtableData, testId).size()); connect.kafka().produce(testId, "stop", value); diff --git a/google-cloud-bigtable-kafka-connect-sink/src/test/java/com/google/cloud/kafka/connect/bigtable/integration/NullHandlingIT.java b/google-cloud-bigtable-kafka-connect-sink/src/test/java/com/google/cloud/kafka/connect/bigtable/integration/NullHandlingIT.java index 5b163be92b..2bd4a1a3ee 100644 --- a/google-cloud-bigtable-kafka-connect-sink/src/test/java/com/google/cloud/kafka/connect/bigtable/integration/NullHandlingIT.java +++ b/google-cloud-bigtable-kafka-connect-sink/src/test/java/com/google/cloud/kafka/connect/bigtable/integration/NullHandlingIT.java @@ -80,7 +80,7 @@ public void testIgnoreMode() throws InterruptedException { String testId = startSingleTopicConnector(connectorProps); connect .assertions() - .assertConnectorAndAtLeastNumTasksAreRunning(testId, numTasks, "Connector start timeout"); + .assertConnectorAndExactlyNumTasksAreRunning(testId, numTasks, "Connector start timeout"); String keyFinish = "key_finish"; ByteString keyFinishBytes = ByteString.copyFrom(keyFinish.getBytes(StandardCharsets.UTF_8)); @@ -138,7 +138,7 @@ public void testWriteMode() throws InterruptedException { String testId = startSingleTopicConnector(connectorProps); connect .assertions() - .assertConnectorAndAtLeastNumTasksAreRunning(testId, numTasks, "Connector start timeout"); + .assertConnectorAndExactlyNumTasksAreRunning(testId, numTasks, "Connector start timeout"); List> records = new ArrayList<>(); records.add( @@ -198,7 +198,7 @@ public void testDeleteMode() throws InterruptedException { String testId = startSingleTopicConnector(connectorProps); connect .assertions() - .assertConnectorAndAtLeastNumTasksAreRunning(testId, numTasks, "Connector start timeout"); + .assertConnectorAndExactlyNumTasksAreRunning(testId, numTasks, "Connector start timeout"); String columnFamily1 = "cf1"; String columnFamily2 = "cf2"; From 38b3ff9653b7052ee14b9d211a54725a08a04eb2 Mon Sep 17 00:00:00 2001 From: Adam Czajkowski Date: Fri, 7 Mar 2025 12:12:29 +0100 Subject: [PATCH 58/76] Configure retries for CheckAndMutate --- .../cloud/kafka/connect/bigtable/config/BigtableSinkConfig.java | 1 + 1 file changed, 1 insertion(+) diff --git a/google-cloud-bigtable-kafka-connect-sink/src/main/java/com/google/cloud/kafka/connect/bigtable/config/BigtableSinkConfig.java b/google-cloud-bigtable-kafka-connect-sink/src/main/java/com/google/cloud/kafka/connect/bigtable/config/BigtableSinkConfig.java index 75628296cd..0511eb78c2 100644 --- a/google-cloud-bigtable-kafka-connect-sink/src/main/java/com/google/cloud/kafka/connect/bigtable/config/BigtableSinkConfig.java +++ b/google-cloud-bigtable-kafka-connect-sink/src/main/java/com/google/cloud/kafka/connect/bigtable/config/BigtableSinkConfig.java @@ -501,6 +501,7 @@ public BigtableDataClient getBigtableDataClient() { EnhancedBigtableStubSettings.Builder dataStubSettings = dataSettingsBuilder.stubSettings(); dataStubSettings.mutateRowSettings().setRetrySettings(retrySettings); + dataStubSettings.checkAndMutateRowSettings().setRetrySettings(retrySettings); dataStubSettings.bulkMutateRowsSettings().setRetrySettings(retrySettings); dataStubSettings.readRowSettings().setRetrySettings(retrySettings); dataStubSettings.readRowsSettings().setRetrySettings(retrySettings); From c6879dc966fa21a46ca7a26373f2f690684c4501 Mon Sep 17 00:00:00 2001 From: Adam Czajkowski Date: Fri, 7 Mar 2025 12:13:06 +0100 Subject: [PATCH 59/76] Update READMEs --- .../performance/README.md | 6 +++- .../performance/terraform/perf_test.sh | 1 + .../src/test/README.md | 30 ++++++++++--------- 3 files changed, 22 insertions(+), 15 deletions(-) diff --git a/google-cloud-bigtable-kafka-connect-sink/performance/README.md b/google-cloud-bigtable-kafka-connect-sink/performance/README.md index 91266d50be..454007d2ba 100644 --- a/google-cloud-bigtable-kafka-connect-sink/performance/README.md +++ b/google-cloud-bigtable-kafka-connect-sink/performance/README.md @@ -169,7 +169,11 @@ A single test run consists of just running a load generator for a stretch of tim - `kubectl` - `jq` - `bash` + coreutils -- `gcloud` (logged into the desired project) +- `envsubst` +- `docker` +- `gcloud` (logged into the desired project, with Application Default Credentials configured) +- `java` 11, 17, or 21 +- `maven` ### Preparation - Adjust variables such as project name and region in [`main.tf`](terraform/main.tf) diff --git a/google-cloud-bigtable-kafka-connect-sink/performance/terraform/perf_test.sh b/google-cloud-bigtable-kafka-connect-sink/performance/terraform/perf_test.sh index e05296c1d8..f71ff814e6 100755 --- a/google-cloud-bigtable-kafka-connect-sink/performance/terraform/perf_test.sh +++ b/google-cloud-bigtable-kafka-connect-sink/performance/terraform/perf_test.sh @@ -18,6 +18,7 @@ print_timestamp() { } do_prepare() { + terraform init terraform apply -auto-approve TERRAFORM_OUTPUT="$(terraform output -json)" KAFKA_CONNECT_DOCKER_REPO="$(echo "$TERRAFORM_OUTPUT" | jq -rc .kafka_connect_docker_registry_url.value)" diff --git a/google-cloud-bigtable-kafka-connect-sink/src/test/README.md b/google-cloud-bigtable-kafka-connect-sink/src/test/README.md index 903a70d214..8ac9a5be71 100644 --- a/google-cloud-bigtable-kafka-connect-sink/src/test/README.md +++ b/google-cloud-bigtable-kafka-connect-sink/src/test/README.md @@ -30,7 +30,7 @@ resource "google_bigtable_instance" "bigtable" { cluster { cluster_id = "kafka-connect-bigtable-sink-test-cluster" num_nodes = 1 - storage_type = "HDD" + storage_type = "SSD" zone = "europe-central2-a" } } @@ -40,31 +40,33 @@ resource "google_bigtable_instance" "bigtable" { This section is optional, you can skip it if you want to use Application Default Credentials. Create a service account and grant it Bigtable Administrator (`roles/bigtable.admin`) permissions (such wide permissions are needed for table and column family auto creation). +Download its key. + +##### Configure the permissions for integration tests +Note that we provide the credentials not only to our sink, but also to Confluent's (in ConfluentCompatibilityIT). + + +If you want to use Application Default Credentials, configure the machine appropriately (on a workstation, log in with `gcloud` into an account with Bigtable Administrator permissions to the instance created in one of the previous steps). + +Otherwise, you need to use service account's permissions. ##### Configure the integration tests to use the created instance -Ensure that the sink's [pom.xml](../../pom.xml) does **not** contain the following section in Failsafe's `` section: +- Ensure that the sink's [pom.xml](../../pom.xml) does **not** contain `BIGTABLE_EMULATOR_HOST` variable in Failsafe's `configuration` section within `environmentVariables` subsection: +- Ensure that `GOOGLE_APPLICATION_CREDENTIALS` in the same subsection points to the appropriate account's key. +The code below shows how to use configured Application Default Credentials on a UNIX workstation. +Alternatively, you could point it to [the service account key created before](#optional-create-service-account-with-required-permissions). ```xml - target/test-classes/fake_service_key.json - localhost:8086 + ${user.home}/.config/gcloud/application_default_credentials.json ``` - -Replace the following TODO values with you GCP project ID and Cloud Bigtable instance ID in `BaseIT#baseConnectorProps()` function: +- Replace the following TODO values with you GCP project ID and Cloud Bigtable instance ID in `BaseIT#baseConnectorProps()` function: ```java result.put(GCP_PROJECT_ID_CONFIG, "todotodo"); result.put(BIGTABLE_INSTANCE_ID_CONFIG, "todotodo"); ``` -##### [optional] Configure the permissions for integration tests - -If you want to use Application Default Credentials, configure the machine (on a workstation, log in with `gcloud` into an account with Bigtable Administrator permissions to the instance created in one of the previous steps). - -Otherwise, you need to use service account's permissions: -- download a service account key. -- put that key or path to it to the properties in `BaseIT#baseConnectorProps()` with a key defined in `BigtableSinkConfig`. - #### Emulator Start the emulator using `gcloud` directly: ```bash From 06c9bd8189a78483aa1d14bdffa8ab68c4621d6c Mon Sep 17 00:00:00 2001 From: Adam Czajkowski Date: Fri, 7 Mar 2025 12:20:21 +0100 Subject: [PATCH 60/76] Use required dependency versions --- .../pom.xml | 28 ++++++++++++------- .../bigtable/BigtableSinkTaskTest.java | 2 +- .../autocreate/BigtableSchemaManagerTest.java | 2 +- .../config/BigtableSinkConfigTest.java | 2 +- .../integration/ConfluentCompatibilityIT.java | 2 +- .../bigtable/integration/ErrorHandlingIT.java | 2 +- .../integration/MultipleConnectorTasksIT.java | 2 +- .../integration/ResourceAutoCreationIT.java | 2 +- .../bigtable/integration/VersionIT.java | 2 +- .../bigtable/mapping/KeyMapperTest.java | 2 +- 10 files changed, 27 insertions(+), 19 deletions(-) diff --git a/google-cloud-bigtable-kafka-connect-sink/pom.xml b/google-cloud-bigtable-kafka-connect-sink/pom.xml index 888d99c5ba..61681a2f81 100644 --- a/google-cloud-bigtable-kafka-connect-sink/pom.xml +++ b/google-cloud-bigtable-kafka-connect-sink/pom.xml @@ -11,18 +11,24 @@ Google Bigtable sink connector for Apache Kafka Connect - 26.54.0 - 3.6.1 + 26.53.0 + 2.52.0 + 3.7.1 2.12 2.6.1 - 2.0.16 - 2.14.2 + 1.7.33 + 1.7.36 + 2.12.7 - 5.14.2 - 4.13.2 - + 5.15.2 + 4.8.2 + 5.11.3 - 7.6.0 + 7.7.0 2.43.0 1.19.2 3.11.2 @@ -69,10 +75,12 @@ org.apache.kafka kafka-clients ${kafka.version} + provided com.google.cloud google-cloud-bigtable + ${google.bigtable.version} org.apache.hbase @@ -82,12 +90,12 @@ org.slf4j slf4j-api - ${slf4j.version} + ${slf4j.api.version} org.slf4j slf4j-reload4j - ${slf4j.version} + ${slf4j.reload4j.version} test diff --git a/google-cloud-bigtable-kafka-connect-sink/src/test/java/com/google/cloud/kafka/connect/bigtable/BigtableSinkTaskTest.java b/google-cloud-bigtable-kafka-connect-sink/src/test/java/com/google/cloud/kafka/connect/bigtable/BigtableSinkTaskTest.java index ff2d573737..7f9a7c8177 100644 --- a/google-cloud-bigtable-kafka-connect-sink/src/test/java/com/google/cloud/kafka/connect/bigtable/BigtableSinkTaskTest.java +++ b/google-cloud-bigtable-kafka-connect-sink/src/test/java/com/google/cloud/kafka/connect/bigtable/BigtableSinkTaskTest.java @@ -25,8 +25,8 @@ import static org.junit.Assert.assertEquals; import static org.junit.Assert.assertFalse; import static org.junit.Assert.assertNotNull; -import static org.junit.Assert.assertThrows; import static org.junit.Assert.assertTrue; +import static org.junit.jupiter.api.Assertions.assertThrows; import static org.mockito.Mockito.any; import static org.mockito.Mockito.anyLong; import static org.mockito.Mockito.anyString; diff --git a/google-cloud-bigtable-kafka-connect-sink/src/test/java/com/google/cloud/kafka/connect/bigtable/autocreate/BigtableSchemaManagerTest.java b/google-cloud-bigtable-kafka-connect-sink/src/test/java/com/google/cloud/kafka/connect/bigtable/autocreate/BigtableSchemaManagerTest.java index 1b4bebae32..af34b98b87 100644 --- a/google-cloud-bigtable-kafka-connect-sink/src/test/java/com/google/cloud/kafka/connect/bigtable/autocreate/BigtableSchemaManagerTest.java +++ b/google-cloud-bigtable-kafka-connect-sink/src/test/java/com/google/cloud/kafka/connect/bigtable/autocreate/BigtableSchemaManagerTest.java @@ -20,8 +20,8 @@ import static com.google.cloud.kafka.connect.bigtable.util.MockUtil.assertTotalNumberOfInvocations; import static org.junit.Assert.assertEquals; import static org.junit.Assert.assertFalse; -import static org.junit.Assert.assertThrows; import static org.junit.Assert.assertTrue; +import static org.junit.jupiter.api.Assertions.assertThrows; import static org.mockito.ArgumentMatchers.any; import static org.mockito.ArgumentMatchers.eq; import static org.mockito.Mockito.argThat; diff --git a/google-cloud-bigtable-kafka-connect-sink/src/test/java/com/google/cloud/kafka/connect/bigtable/config/BigtableSinkConfigTest.java b/google-cloud-bigtable-kafka-connect-sink/src/test/java/com/google/cloud/kafka/connect/bigtable/config/BigtableSinkConfigTest.java index b7e7590b9d..4ea19f278a 100644 --- a/google-cloud-bigtable-kafka-connect-sink/src/test/java/com/google/cloud/kafka/connect/bigtable/config/BigtableSinkConfigTest.java +++ b/google-cloud-bigtable-kafka-connect-sink/src/test/java/com/google/cloud/kafka/connect/bigtable/config/BigtableSinkConfigTest.java @@ -25,8 +25,8 @@ import static java.util.Collections.emptyList; import static org.junit.Assert.assertEquals; import static org.junit.Assert.assertFalse; -import static org.junit.Assert.assertThrows; import static org.junit.Assert.assertTrue; +import static org.junit.jupiter.api.Assertions.assertThrows; import static org.mockito.Mockito.any; import static org.mockito.Mockito.doReturn; import static org.mockito.Mockito.doThrow; diff --git a/google-cloud-bigtable-kafka-connect-sink/src/test/java/com/google/cloud/kafka/connect/bigtable/integration/ConfluentCompatibilityIT.java b/google-cloud-bigtable-kafka-connect-sink/src/test/java/com/google/cloud/kafka/connect/bigtable/integration/ConfluentCompatibilityIT.java index 7401f0ab64..60787c11e6 100644 --- a/google-cloud-bigtable-kafka-connect-sink/src/test/java/com/google/cloud/kafka/connect/bigtable/integration/ConfluentCompatibilityIT.java +++ b/google-cloud-bigtable-kafka-connect-sink/src/test/java/com/google/cloud/kafka/connect/bigtable/integration/ConfluentCompatibilityIT.java @@ -24,7 +24,7 @@ import static org.apache.kafka.connect.runtime.ConnectorConfig.CONNECTOR_CLASS_CONFIG; import static org.junit.Assert.assertEquals; -import static org.junit.Assert.assertNotEquals; +import static org.junit.jupiter.api.Assertions.assertNotEquals; import com.google.cloud.bigtable.admin.v2.models.CreateTableRequest; import com.google.cloud.bigtable.data.v2.models.Row; diff --git a/google-cloud-bigtable-kafka-connect-sink/src/test/java/com/google/cloud/kafka/connect/bigtable/integration/ErrorHandlingIT.java b/google-cloud-bigtable-kafka-connect-sink/src/test/java/com/google/cloud/kafka/connect/bigtable/integration/ErrorHandlingIT.java index e664f99273..9d6d572b00 100644 --- a/google-cloud-bigtable-kafka-connect-sink/src/test/java/com/google/cloud/kafka/connect/bigtable/integration/ErrorHandlingIT.java +++ b/google-cloud-bigtable-kafka-connect-sink/src/test/java/com/google/cloud/kafka/connect/bigtable/integration/ErrorHandlingIT.java @@ -17,8 +17,8 @@ import static org.junit.Assert.assertArrayEquals; import static org.junit.Assert.assertEquals; -import static org.junit.Assert.assertThrows; import static org.junit.Assert.assertTrue; +import static org.junit.jupiter.api.Assertions.assertThrows; import com.google.cloud.bigtable.data.v2.models.Row; import com.google.cloud.bigtable.data.v2.models.RowCell; diff --git a/google-cloud-bigtable-kafka-connect-sink/src/test/java/com/google/cloud/kafka/connect/bigtable/integration/MultipleConnectorTasksIT.java b/google-cloud-bigtable-kafka-connect-sink/src/test/java/com/google/cloud/kafka/connect/bigtable/integration/MultipleConnectorTasksIT.java index 17ee4f64b6..84985e7f17 100644 --- a/google-cloud-bigtable-kafka-connect-sink/src/test/java/com/google/cloud/kafka/connect/bigtable/integration/MultipleConnectorTasksIT.java +++ b/google-cloud-bigtable-kafka-connect-sink/src/test/java/com/google/cloud/kafka/connect/bigtable/integration/MultipleConnectorTasksIT.java @@ -129,6 +129,6 @@ public void testRestartPauseStop() throws InterruptedException { waitUntilBigtableContainsNumberOfRows(testId, expectedRowsInBigtable); connect.deleteConnector(testId); - connect.assertions().assertConnectorAndTasksAreNotRunning(testId, "Connector deletion timeout"); + connect.assertions().assertConnectorDoesNotExist(testId, "Connector deletion timeout"); } } diff --git a/google-cloud-bigtable-kafka-connect-sink/src/test/java/com/google/cloud/kafka/connect/bigtable/integration/ResourceAutoCreationIT.java b/google-cloud-bigtable-kafka-connect-sink/src/test/java/com/google/cloud/kafka/connect/bigtable/integration/ResourceAutoCreationIT.java index e82d364532..4744b82cc3 100644 --- a/google-cloud-bigtable-kafka-connect-sink/src/test/java/com/google/cloud/kafka/connect/bigtable/integration/ResourceAutoCreationIT.java +++ b/google-cloud-bigtable-kafka-connect-sink/src/test/java/com/google/cloud/kafka/connect/bigtable/integration/ResourceAutoCreationIT.java @@ -17,8 +17,8 @@ import static org.junit.Assert.assertEquals; import static org.junit.Assert.assertFalse; -import static org.junit.Assert.assertThrows; import static org.junit.Assert.assertTrue; +import static org.junit.jupiter.api.Assertions.assertThrows; import com.google.cloud.bigtable.admin.v2.models.ColumnFamily; import com.google.cloud.bigtable.admin.v2.models.CreateTableRequest; diff --git a/google-cloud-bigtable-kafka-connect-sink/src/test/java/com/google/cloud/kafka/connect/bigtable/integration/VersionIT.java b/google-cloud-bigtable-kafka-connect-sink/src/test/java/com/google/cloud/kafka/connect/bigtable/integration/VersionIT.java index 0500f0e11e..a039ead92f 100644 --- a/google-cloud-bigtable-kafka-connect-sink/src/test/java/com/google/cloud/kafka/connect/bigtable/integration/VersionIT.java +++ b/google-cloud-bigtable-kafka-connect-sink/src/test/java/com/google/cloud/kafka/connect/bigtable/integration/VersionIT.java @@ -15,7 +15,7 @@ */ package com.google.cloud.kafka.connect.bigtable.integration; -import static org.junit.Assert.assertNotEquals; +import static org.junit.jupiter.api.Assertions.assertNotEquals; import com.fasterxml.jackson.databind.MapperFeature; import com.fasterxml.jackson.databind.ObjectMapper; diff --git a/google-cloud-bigtable-kafka-connect-sink/src/test/java/com/google/cloud/kafka/connect/bigtable/mapping/KeyMapperTest.java b/google-cloud-bigtable-kafka-connect-sink/src/test/java/com/google/cloud/kafka/connect/bigtable/mapping/KeyMapperTest.java index 39aa05efac..3c523f723e 100644 --- a/google-cloud-bigtable-kafka-connect-sink/src/test/java/com/google/cloud/kafka/connect/bigtable/mapping/KeyMapperTest.java +++ b/google-cloud-bigtable-kafka-connect-sink/src/test/java/com/google/cloud/kafka/connect/bigtable/mapping/KeyMapperTest.java @@ -24,7 +24,7 @@ import static org.junit.Assert.assertArrayEquals; import static org.junit.Assert.assertEquals; -import static org.junit.Assert.assertThrows; +import static org.junit.jupiter.api.Assertions.assertThrows; import com.google.cloud.kafka.connect.bigtable.util.JsonConverterFactory; import java.io.ByteArrayOutputStream; From 46ddca3104d15907810b3bd50e024028d39dbe3d Mon Sep 17 00:00:00 2001 From: Adam Czajkowski Date: Fri, 7 Mar 2025 12:37:31 +0100 Subject: [PATCH 61/76] Include integration test downloadables in .gitgnore --- google-cloud-bigtable-kafka-connect-sink/.gitignore | 2 ++ 1 file changed, 2 insertions(+) diff --git a/google-cloud-bigtable-kafka-connect-sink/.gitignore b/google-cloud-bigtable-kafka-connect-sink/.gitignore index 612c5bc968..2e4f1cbfb8 100644 --- a/google-cloud-bigtable-kafka-connect-sink/.gitignore +++ b/google-cloud-bigtable-kafka-connect-sink/.gitignore @@ -1,3 +1,5 @@ target .idea *.iml +*.zip +integration_test_plugins/ From c380d9f00ae663bd9ba3fdb59fdce226d601d480 Mon Sep 17 00:00:00 2001 From: Adam Czajkowski Date: Fri, 7 Mar 2025 12:39:21 +0100 Subject: [PATCH 62/76] Use getName() for getting class name everywhere --- .../google/cloud/kafka/connect/bigtable/integration/BaseIT.java | 2 +- .../cloud/kafka/connect/bigtable/integration/VersionIT.java | 2 +- 2 files changed, 2 insertions(+), 2 deletions(-) diff --git a/google-cloud-bigtable-kafka-connect-sink/src/test/java/com/google/cloud/kafka/connect/bigtable/integration/BaseIT.java b/google-cloud-bigtable-kafka-connect-sink/src/test/java/com/google/cloud/kafka/connect/bigtable/integration/BaseIT.java index 581a128d08..fda893fae3 100644 --- a/google-cloud-bigtable-kafka-connect-sink/src/test/java/com/google/cloud/kafka/connect/bigtable/integration/BaseIT.java +++ b/google-cloud-bigtable-kafka-connect-sink/src/test/java/com/google/cloud/kafka/connect/bigtable/integration/BaseIT.java @@ -40,7 +40,7 @@ public abstract class BaseIT { public Map baseConnectorProps() { Map result = new HashMap<>(); - result.put(CONNECTOR_CLASS_CONFIG, BigtableSinkConnector.class.getCanonicalName()); + result.put(CONNECTOR_CLASS_CONFIG, BigtableSinkConnector.class.getName()); result.put(TASKS_MAX_CONFIG, Integer.toString(numTasks)); result.put(KEY_CONVERTER_CLASS_CONFIG, StringConverter.class.getName()); result.put(VALUE_CONVERTER_CLASS_CONFIG, StringConverter.class.getName()); diff --git a/google-cloud-bigtable-kafka-connect-sink/src/test/java/com/google/cloud/kafka/connect/bigtable/integration/VersionIT.java b/google-cloud-bigtable-kafka-connect-sink/src/test/java/com/google/cloud/kafka/connect/bigtable/integration/VersionIT.java index a039ead92f..7f3453781c 100644 --- a/google-cloud-bigtable-kafka-connect-sink/src/test/java/com/google/cloud/kafka/connect/bigtable/integration/VersionIT.java +++ b/google-cloud-bigtable-kafka-connect-sink/src/test/java/com/google/cloud/kafka/connect/bigtable/integration/VersionIT.java @@ -49,7 +49,7 @@ public void testVersionIsSet() throws IOException, InterruptedException { PluginInfo[] pluginInfos = mapper.readValue(response.body(), PluginInfo[].class); PluginInfo pluginInfo = Arrays.stream(pluginInfos) - .filter(i -> i.className().equals(BigtableSinkConnector.class.getCanonicalName())) + .filter(i -> i.className().equals(BigtableSinkConnector.class.getName())) .findFirst() .get(); assertNotEquals(PackageMetadata.UNKNOWN_VERSION, pluginInfo.version()); From 036a4368bf4b78cb80aaba5241a9caabdd3f732e Mon Sep 17 00:00:00 2001 From: Adam Czajkowski Date: Fri, 7 Mar 2025 12:46:50 +0100 Subject: [PATCH 63/76] Use file-based credentials for all integration tests to avoid special casing ConfluentCompatibilityIT --- .../cloud/kafka/connect/bigtable/integration/BaseIT.java | 9 +++++++++ .../bigtable/integration/ConfluentCompatibilityIT.java | 4 ---- 2 files changed, 9 insertions(+), 4 deletions(-) diff --git a/google-cloud-bigtable-kafka-connect-sink/src/test/java/com/google/cloud/kafka/connect/bigtable/integration/BaseIT.java b/google-cloud-bigtable-kafka-connect-sink/src/test/java/com/google/cloud/kafka/connect/bigtable/integration/BaseIT.java index fda893fae3..81e9ea7ea4 100644 --- a/google-cloud-bigtable-kafka-connect-sink/src/test/java/com/google/cloud/kafka/connect/bigtable/integration/BaseIT.java +++ b/google-cloud-bigtable-kafka-connect-sink/src/test/java/com/google/cloud/kafka/connect/bigtable/integration/BaseIT.java @@ -29,11 +29,16 @@ import com.google.cloud.kafka.connect.bigtable.util.TestId; import java.util.HashMap; import java.util.Map; +import java.util.Objects; import org.apache.kafka.clients.producer.ProducerConfig; import org.apache.kafka.connect.runtime.ConnectorConfig; import org.apache.kafka.connect.storage.StringConverter; public abstract class BaseIT { + // The Confluent's sink cannot use Application Default Credentials. We set the credentials + // explicitly (rather than use ADC) to avoid having a special case for ConfluentCompatibilityIT. + public String CREDENTIALS_PATH_ENV_VAR = "GOOGLE_APPLICATION_CREDENTIALS"; + public int numTasks = 1; public int maxKafkaMessageSizeBytes = 300 * 1024 * 1024; @@ -58,6 +63,10 @@ public Map baseConnectorProps() { // TODO: get it from environment variables after migrating to kokoro. result.put(GCP_PROJECT_ID_CONFIG, "todotodo"); result.put(BIGTABLE_INSTANCE_ID_CONFIG, "todotodo"); + // TODO: fix it when transitioning to kokoro. + result.put( + BigtableSinkConfig.GCP_CREDENTIALS_PATH_CONFIG, + Objects.requireNonNull(System.getenv(CREDENTIALS_PATH_ENV_VAR))); return result; } diff --git a/google-cloud-bigtable-kafka-connect-sink/src/test/java/com/google/cloud/kafka/connect/bigtable/integration/ConfluentCompatibilityIT.java b/google-cloud-bigtable-kafka-connect-sink/src/test/java/com/google/cloud/kafka/connect/bigtable/integration/ConfluentCompatibilityIT.java index 60787c11e6..86635f3227 100644 --- a/google-cloud-bigtable-kafka-connect-sink/src/test/java/com/google/cloud/kafka/connect/bigtable/integration/ConfluentCompatibilityIT.java +++ b/google-cloud-bigtable-kafka-connect-sink/src/test/java/com/google/cloud/kafka/connect/bigtable/integration/ConfluentCompatibilityIT.java @@ -169,10 +169,6 @@ public String startConfluentConnector() throws InterruptedException { connectorProps.put("confluent.license", ""); connectorProps.put("confluent.topic.bootstrap.servers", connect.kafka().bootstrapServers()); connectorProps.put("confluent.topic.replication.factor", "1"); - // TODO: fix it when transitioning to kokoro. - connectorProps.put( - BigtableSinkConfig.GCP_CREDENTIALS_PATH_CONFIG, - Objects.requireNonNull(System.getenv("GOOGLE_APPLICATION_CREDENTIALS"))); return startConnector(connectorProps); } From f58c32b672c63017c172e2cc70a8ad65e4d15230 Mon Sep 17 00:00:00 2001 From: Adam Czajkowski Date: Fri, 7 Mar 2025 12:54:57 +0100 Subject: [PATCH 64/76] Handle temporary errors in integration test helpers --- .../BaseKafkaConnectBigtableIT.java | 37 ++++++++++++++----- 1 file changed, 28 insertions(+), 9 deletions(-) diff --git a/google-cloud-bigtable-kafka-connect-sink/src/test/java/com/google/cloud/kafka/connect/bigtable/integration/BaseKafkaConnectBigtableIT.java b/google-cloud-bigtable-kafka-connect-sink/src/test/java/com/google/cloud/kafka/connect/bigtable/integration/BaseKafkaConnectBigtableIT.java index 88fed5dc60..4ff80f68fa 100644 --- a/google-cloud-bigtable-kafka-connect-sink/src/test/java/com/google/cloud/kafka/connect/bigtable/integration/BaseKafkaConnectBigtableIT.java +++ b/google-cloud-bigtable-kafka-connect-sink/src/test/java/com/google/cloud/kafka/connect/bigtable/integration/BaseKafkaConnectBigtableIT.java @@ -17,13 +17,17 @@ import static org.apache.kafka.test.TestUtils.waitForCondition; +import com.google.api.gax.rpc.FailedPreconditionException; +import com.google.api.gax.rpc.NotFoundException; import com.google.cloud.bigtable.admin.v2.BigtableTableAdminClient; import com.google.cloud.bigtable.data.v2.BigtableDataClient; import com.google.cloud.bigtable.data.v2.models.Query; import com.google.cloud.bigtable.data.v2.models.Row; import com.google.protobuf.ByteString; import java.util.Map; +import java.util.function.Supplier; import java.util.stream.Collectors; +import org.apache.kafka.test.TestCondition; import org.junit.After; import org.junit.Before; import org.slf4j.Logger; @@ -77,7 +81,8 @@ public long cellCount(Map rows) { public void waitUntilBigtableContainsNumberOfRows(String tableId, long numberOfRows) throws InterruptedException { waitForCondition( - () -> readAllRows(bigtableData, tableId).size() == numberOfRows, + testConditionIgnoringTransientErrors( + () -> readAllRows(bigtableData, tableId).size() == numberOfRows), DEFAULT_BIGTABLE_RETRY_TIMEOUT_MILLIS, "Records not consumed in time."); } @@ -85,17 +90,19 @@ public void waitUntilBigtableContainsNumberOfRows(String tableId, long numberOfR public void waitUntilBigtableContainsNumberOfCells(String tableId, long numberOfCells) throws InterruptedException { waitForCondition( - () -> cellCount(readAllRows(bigtableData, tableId)) == numberOfCells, + testConditionIgnoringTransientErrors( + () -> cellCount(readAllRows(bigtableData, tableId)) == numberOfCells), DEFAULT_BIGTABLE_RETRY_TIMEOUT_MILLIS, "Records not consumed in time"); } public void waitUntilBigtableTableExists(String tableId) throws InterruptedException { waitForCondition( - () -> { - bigtableAdmin.getTable(tableId); - return true; - }, + testConditionIgnoringTransientErrors( + () -> { + bigtableAdmin.getTable(tableId); + return true; + }), DEFAULT_BIGTABLE_RETRY_TIMEOUT_MILLIS, "Table not created in time."); } @@ -103,10 +110,22 @@ public void waitUntilBigtableTableExists(String tableId) throws InterruptedExcep public void waitUntilBigtableTableHasColumnFamily(String tableId, String columnFamily) throws InterruptedException { waitForCondition( - () -> - bigtableAdmin.getTable(tableId).getColumnFamilies().stream() - .anyMatch(cf -> cf.getId().equals(columnFamily)), + testConditionIgnoringTransientErrors( + () -> + bigtableAdmin.getTable(tableId).getColumnFamilies().stream() + .anyMatch(cf -> cf.getId().equals(columnFamily))), DEFAULT_BIGTABLE_RETRY_TIMEOUT_MILLIS, "Column Family not created in time."); } + + // These exceptions are thrown around the moment of a table or column family creation. + private TestCondition testConditionIgnoringTransientErrors(Supplier supplier) { + return () -> { + try { + return supplier.get(); + } catch (NotFoundException | FailedPreconditionException e) { + return false; + } + }; + } } From 768ac9ec5caac2e58d72f1419751f533e7b7fdaf Mon Sep 17 00:00:00 2001 From: Adam Czajkowski Date: Mon, 10 Mar 2025 09:32:58 +0100 Subject: [PATCH 65/76] Fix comment about retries in Bigtable Admin API --- .../kafka/connect/bigtable/config/BigtableSinkConfig.java | 6 +++--- 1 file changed, 3 insertions(+), 3 deletions(-) diff --git a/google-cloud-bigtable-kafka-connect-sink/src/main/java/com/google/cloud/kafka/connect/bigtable/config/BigtableSinkConfig.java b/google-cloud-bigtable-kafka-connect-sink/src/main/java/com/google/cloud/kafka/connect/bigtable/config/BigtableSinkConfig.java index 0511eb78c2..e1a931a8f5 100644 --- a/google-cloud-bigtable-kafka-connect-sink/src/main/java/com/google/cloud/kafka/connect/bigtable/config/BigtableSinkConfig.java +++ b/google-cloud-bigtable-kafka-connect-sink/src/main/java/com/google/cloud/kafka/connect/bigtable/config/BigtableSinkConfig.java @@ -458,9 +458,9 @@ BigtableTableAdminClient getBigtableAdminClient( adminStubSettings .modifyColumnFamiliesSettings() .setRetrySettings(adminApiWriteRetrySettings) - // Retry createTable() for status codes other admin operations retry by default as - // seen in BigtableTableAdminStubSettings and for FAILED_PRECONDITION which is - // returned when concurrent column family creation is detected. + // Retry modifyColumnFamilies() for status codes other admin operations retry by + // default as seen in BigtableTableAdminStubSettings and for FAILED_PRECONDITION, + // which is returned when concurrent column family creation is detected. .setRetryableCodes( StatusCode.Code.UNAVAILABLE, StatusCode.Code.DEADLINE_EXCEEDED, From 67596417e505529493fb6589ca04693651158284 Mon Sep 17 00:00:00 2001 From: Adam Czajkowski Date: Mon, 10 Mar 2025 13:34:17 +0100 Subject: [PATCH 66/76] Test that ServiceLoader works as expected --- .../connect/bigtable/BigtableSinkConnectorTest.java | 9 +++++++++ 1 file changed, 9 insertions(+) diff --git a/google-cloud-bigtable-kafka-connect-sink/src/test/java/com/google/cloud/kafka/connect/bigtable/BigtableSinkConnectorTest.java b/google-cloud-bigtable-kafka-connect-sink/src/test/java/com/google/cloud/kafka/connect/bigtable/BigtableSinkConnectorTest.java index 9d04c010d7..7c105e3cd8 100644 --- a/google-cloud-bigtable-kafka-connect-sink/src/test/java/com/google/cloud/kafka/connect/bigtable/BigtableSinkConnectorTest.java +++ b/google-cloud-bigtable-kafka-connect-sink/src/test/java/com/google/cloud/kafka/connect/bigtable/BigtableSinkConnectorTest.java @@ -18,11 +18,14 @@ import static com.google.cloud.kafka.connect.bigtable.config.BigtableSinkTaskConfig.TASK_ID_CONFIG; import static org.junit.Assert.assertEquals; import static org.junit.Assert.assertNotNull; +import static org.junit.Assert.assertTrue; import com.google.cloud.kafka.connect.bigtable.util.BasicPropertiesFactory; import java.util.HashMap; import java.util.List; import java.util.Map; +import java.util.ServiceLoader; +import org.apache.kafka.connect.sink.SinkConnector; import org.junit.Before; import org.junit.Test; import org.junit.runner.RunWith; @@ -81,4 +84,10 @@ public void testTaskConfigs() { public void testVersion() { assertNotNull(connector.version()); } + + @Test + public void testServiceLoader() { + ServiceLoader serviceLoader = ServiceLoader.load(SinkConnector.class); + assertTrue(serviceLoader.stream().anyMatch(x -> BigtableSinkConnector.class.equals(x.type()))); + } } From 0bf4d1b7e38986e3955ac01314d0147d1b0a2504 Mon Sep 17 00:00:00 2001 From: Adam Czajkowski Date: Mon, 10 Mar 2025 14:13:15 +0100 Subject: [PATCH 67/76] Get rid of test flakiness by creating the Bigtable tables and column families before the integration tests --- .../integration/BaseDataGeneratorIT.java | 39 +++++++++++++++---- .../BaseKafkaConnectBigtableIT.java | 27 +++++++++++++ .../connect/bigtable/integration/BasicIT.java | 7 ++-- .../integration/ConfluentCompatibilityIT.java | 26 +++++++++---- .../integration/DifferentConvertersIT.java | 19 ++++----- .../bigtable/integration/ErrorHandlingIT.java | 24 +++++------- .../bigtable/integration/InsertUpsertIT.java | 11 +++--- .../integration/MultipleConnectorTasksIT.java | 12 +++--- .../bigtable/integration/NullHandlingIT.java | 26 +++++++------ .../integration/ResourceAutoCreationIT.java | 9 ++--- .../key_bytes/column-families.strings | 0 .../key_containers/column-families.strings | 0 .../key_logicals/column-families.strings | 0 .../key_matryoshkas/column-families.strings | 0 .../column-families.strings | 0 .../key_primitives/column-families.strings | 0 .../column-families.strings | 0 .../key_union/column-families.strings | 0 .../value_bytes/column-families.strings | 0 .../value_containers/column-families.strings | 3 ++ .../value_logicals/column-families.strings | 0 .../value_matryoshkas/column-families.strings | 2 + .../column-families.strings | 1 + .../value_nulls/column-families.strings | 0 .../value_primitives/column-families.strings | 0 .../column-families.strings | 0 .../value_union/column-families.strings | 0 27 files changed, 132 insertions(+), 74 deletions(-) create mode 100644 google-cloud-bigtable-kafka-connect-sink/src/test/resources/compatibility_test_cases/key_bytes/column-families.strings create mode 100644 google-cloud-bigtable-kafka-connect-sink/src/test/resources/compatibility_test_cases/key_containers/column-families.strings create mode 100644 google-cloud-bigtable-kafka-connect-sink/src/test/resources/compatibility_test_cases/key_logicals/column-families.strings create mode 100644 google-cloud-bigtable-kafka-connect-sink/src/test/resources/compatibility_test_cases/key_matryoshkas/column-families.strings create mode 100644 google-cloud-bigtable-kafka-connect-sink/src/test/resources/compatibility_test_cases/key_nestedlogicals/column-families.strings create mode 100644 google-cloud-bigtable-kafka-connect-sink/src/test/resources/compatibility_test_cases/key_primitives/column-families.strings create mode 100644 google-cloud-bigtable-kafka-connect-sink/src/test/resources/compatibility_test_cases/key_root_primitives/column-families.strings create mode 100644 google-cloud-bigtable-kafka-connect-sink/src/test/resources/compatibility_test_cases/key_union/column-families.strings create mode 100644 google-cloud-bigtable-kafka-connect-sink/src/test/resources/compatibility_test_cases/value_bytes/column-families.strings create mode 100644 google-cloud-bigtable-kafka-connect-sink/src/test/resources/compatibility_test_cases/value_containers/column-families.strings create mode 100644 google-cloud-bigtable-kafka-connect-sink/src/test/resources/compatibility_test_cases/value_logicals/column-families.strings create mode 100644 google-cloud-bigtable-kafka-connect-sink/src/test/resources/compatibility_test_cases/value_matryoshkas/column-families.strings create mode 100644 google-cloud-bigtable-kafka-connect-sink/src/test/resources/compatibility_test_cases/value_nestedlogicals/column-families.strings create mode 100644 google-cloud-bigtable-kafka-connect-sink/src/test/resources/compatibility_test_cases/value_nulls/column-families.strings create mode 100644 google-cloud-bigtable-kafka-connect-sink/src/test/resources/compatibility_test_cases/value_primitives/column-families.strings create mode 100644 google-cloud-bigtable-kafka-connect-sink/src/test/resources/compatibility_test_cases/value_root_primitives/column-families.strings create mode 100644 google-cloud-bigtable-kafka-connect-sink/src/test/resources/compatibility_test_cases/value_union/column-families.strings diff --git a/google-cloud-bigtable-kafka-connect-sink/src/test/java/com/google/cloud/kafka/connect/bigtable/integration/BaseDataGeneratorIT.java b/google-cloud-bigtable-kafka-connect-sink/src/test/java/com/google/cloud/kafka/connect/bigtable/integration/BaseDataGeneratorIT.java index cef7ad855a..728540c578 100644 --- a/google-cloud-bigtable-kafka-connect-sink/src/test/java/com/google/cloud/kafka/connect/bigtable/integration/BaseDataGeneratorIT.java +++ b/google-cloud-bigtable-kafka-connect-sink/src/test/java/com/google/cloud/kafka/connect/bigtable/integration/BaseDataGeneratorIT.java @@ -27,6 +27,7 @@ import java.util.ArrayList; import java.util.List; import java.util.Map; +import java.util.Set; import java.util.stream.Collectors; import java.util.stream.LongStream; import org.apache.kafka.connect.data.Date; @@ -40,6 +41,14 @@ import org.apache.kafka.connect.storage.Converter; public class BaseDataGeneratorIT extends BaseKafkaConnectBigtableSchemaRegistryIT { + private static final String FIELD_STRING = "f1"; + private static final String FIELD_BOOL = "f2"; + private static final String FIELD_FLOAT = "f3"; + private static final String FIELD_BYTES = "bytes_field"; + private static final String FIELD_NESTED = "nested_field"; + private static final String FIELD_PRIMITIVES = "primitives_field"; + private static final String FIELD_LOGICALS = "logicals_field"; + private static final String FIELD_ARRAY = "array_field"; private static final Schema SUB_STRUCT_SCHEMA = SchemaBuilder.struct() .field("ssf1", Schema.INT64_SCHEMA) @@ -74,15 +83,29 @@ public class BaseDataGeneratorIT extends BaseKafkaConnectBigtableSchemaRegistryI private static final Schema VALUE_SCHEMA = SchemaBuilder.struct() .optional() - .field("f1", Schema.STRING_SCHEMA) - .field("f2", Schema.BOOLEAN_SCHEMA) - .field("f3", Schema.FLOAT64_SCHEMA) - .field("bytes_field", Schema.OPTIONAL_BYTES_SCHEMA) - .field("nested_field", NESTED_STRUCT_SCHEMA) - .field("primitives_field", PRIMITIVES_SCHEMA) - .field("logicals_field", LOGICALS_SCHEMA) - .field("array_field", ARRAY_SCHEMA) + .field(FIELD_STRING, Schema.STRING_SCHEMA) + .field(FIELD_BOOL, Schema.BOOLEAN_SCHEMA) + .field(FIELD_FLOAT, Schema.FLOAT64_SCHEMA) + .field(FIELD_BYTES, Schema.OPTIONAL_BYTES_SCHEMA) + .field(FIELD_NESTED, NESTED_STRUCT_SCHEMA) + .field(FIELD_PRIMITIVES, PRIMITIVES_SCHEMA) + .field(FIELD_LOGICALS, LOGICALS_SCHEMA) + .field(FIELD_ARRAY, ARRAY_SCHEMA) .build(); + + public static Set valueFields(String defaultColumnFamily) { + return Set.of( + FIELD_STRING, + FIELD_BOOL, + FIELD_FLOAT, + FIELD_BYTES, + FIELD_NESTED, + FIELD_PRIMITIVES, + FIELD_LOGICALS, + FIELD_ARRAY, + defaultColumnFamily); + } + public long numRecords = 100L; public void populateKafkaTopic( diff --git a/google-cloud-bigtable-kafka-connect-sink/src/test/java/com/google/cloud/kafka/connect/bigtable/integration/BaseKafkaConnectBigtableIT.java b/google-cloud-bigtable-kafka-connect-sink/src/test/java/com/google/cloud/kafka/connect/bigtable/integration/BaseKafkaConnectBigtableIT.java index 4ff80f68fa..5c3e031691 100644 --- a/google-cloud-bigtable-kafka-connect-sink/src/test/java/com/google/cloud/kafka/connect/bigtable/integration/BaseKafkaConnectBigtableIT.java +++ b/google-cloud-bigtable-kafka-connect-sink/src/test/java/com/google/cloud/kafka/connect/bigtable/integration/BaseKafkaConnectBigtableIT.java @@ -20,11 +20,17 @@ import com.google.api.gax.rpc.FailedPreconditionException; import com.google.api.gax.rpc.NotFoundException; import com.google.cloud.bigtable.admin.v2.BigtableTableAdminClient; +import com.google.cloud.bigtable.admin.v2.models.CreateTableRequest; +import com.google.cloud.bigtable.admin.v2.models.Table; import com.google.cloud.bigtable.data.v2.BigtableDataClient; import com.google.cloud.bigtable.data.v2.models.Query; import com.google.cloud.bigtable.data.v2.models.Row; import com.google.protobuf.ByteString; +import java.util.List; import java.util.Map; +import java.util.Set; +import java.util.concurrent.ExecutionException; +import java.util.concurrent.Future; import java.util.function.Supplier; import java.util.stream.Collectors; import org.apache.kafka.test.TestCondition; @@ -59,6 +65,27 @@ public void tearDownBigtable() { } } + public void createTablesAndColumnFamilies(String testId) + throws ExecutionException, InterruptedException { + createTablesAndColumnFamilies(Map.of(testId, Set.of(testId))); + } + + public void createTablesAndColumnFamilies(Map> tablesAndColumnFamilies) + throws ExecutionException, InterruptedException { + List> futures = + tablesAndColumnFamilies.entrySet().parallelStream() + .map( + e -> { + CreateTableRequest ctr = CreateTableRequest.of(e.getKey()); + e.getValue().forEach(ctr::addFamily); + return bigtableAdmin.createTableAsync(ctr); + }) + .collect(Collectors.toList()); + for (Future

    f : futures) { + f.get(); + } + } + public Map readAllRows(BigtableDataClient bigtable, String table) { Integer numRecords = null; try { diff --git a/google-cloud-bigtable-kafka-connect-sink/src/test/java/com/google/cloud/kafka/connect/bigtable/integration/BasicIT.java b/google-cloud-bigtable-kafka-connect-sink/src/test/java/com/google/cloud/kafka/connect/bigtable/integration/BasicIT.java index d474f4b9ab..509cd429a4 100644 --- a/google-cloud-bigtable-kafka-connect-sink/src/test/java/com/google/cloud/kafka/connect/bigtable/integration/BasicIT.java +++ b/google-cloud-bigtable-kafka-connect-sink/src/test/java/com/google/cloud/kafka/connect/bigtable/integration/BasicIT.java @@ -20,11 +20,11 @@ import com.google.cloud.bigtable.data.v2.models.Row; import com.google.cloud.bigtable.data.v2.models.RowCell; -import com.google.cloud.kafka.connect.bigtable.config.BigtableSinkConfig; import com.google.protobuf.ByteString; import java.nio.charset.StandardCharsets; import java.util.List; import java.util.Map; +import java.util.concurrent.ExecutionException; import org.junit.Test; import org.junit.runner.RunWith; import org.junit.runners.JUnit4; @@ -32,11 +32,10 @@ @RunWith(JUnit4.class) public class BasicIT extends BaseKafkaConnectBigtableIT { @Test - public void testSimpleWrite() throws InterruptedException { + public void testSimpleWrite() throws InterruptedException, ExecutionException { Map props = baseConnectorProps(); - props.put(BigtableSinkConfig.AUTO_CREATE_TABLES_CONFIG, "true"); - props.put(BigtableSinkConfig.AUTO_CREATE_COLUMN_FAMILIES_CONFIG, "true"); String topic = startSingleTopicConnector(props); + createTablesAndColumnFamilies(topic); String key = "key"; ByteString keyBytes = ByteString.copyFrom(key.getBytes(StandardCharsets.UTF_8)); diff --git a/google-cloud-bigtable-kafka-connect-sink/src/test/java/com/google/cloud/kafka/connect/bigtable/integration/ConfluentCompatibilityIT.java b/google-cloud-bigtable-kafka-connect-sink/src/test/java/com/google/cloud/kafka/connect/bigtable/integration/ConfluentCompatibilityIT.java index 86635f3227..d23b0d5fa9 100644 --- a/google-cloud-bigtable-kafka-connect-sink/src/test/java/com/google/cloud/kafka/connect/bigtable/integration/ConfluentCompatibilityIT.java +++ b/google-cloud-bigtable-kafka-connect-sink/src/test/java/com/google/cloud/kafka/connect/bigtable/integration/ConfluentCompatibilityIT.java @@ -26,7 +26,6 @@ import static org.junit.Assert.assertEquals; import static org.junit.jupiter.api.Assertions.assertNotEquals; -import com.google.cloud.bigtable.admin.v2.models.CreateTableRequest; import com.google.cloud.bigtable.data.v2.models.Row; import com.google.cloud.bigtable.data.v2.models.RowCell; import com.google.cloud.kafka.connect.bigtable.config.BigtableSinkConfig; @@ -44,9 +43,12 @@ import java.util.Map; import java.util.Objects; import java.util.Properties; +import java.util.Set; import java.util.concurrent.ExecutionException; import java.util.concurrent.TimeUnit; import java.util.concurrent.TimeoutException; +import java.util.stream.Collectors; +import java.util.stream.Stream; import kafka.common.MessageReader; import org.apache.kafka.clients.producer.Producer; import org.apache.kafka.clients.producer.ProducerRecord; @@ -101,13 +103,15 @@ public static Collection testCases() { } @Test - public void testCasesUsingSchemaRegistry() throws InterruptedException, IOException { + public void testCasesUsingSchemaRegistry() + throws InterruptedException, IOException, ExecutionException { String confluentTestId = startConfluentConnector(); String googleTestId = startThisConnector(confluentTestId); assertNotEquals(confluentTestId, googleTestId); - - bigtableAdmin.createTable(CreateTableRequest.of(confluentTestId)); - bigtableAdmin.createTable(CreateTableRequest.of(googleTestId)); + createTablesAndColumnFamilies( + Map.of( + confluentTestId, getNeededColumnFamilies(confluentTestId), + googleTestId, getNeededColumnFamilies(confluentTestId))); populateTopic(confluentTestId); populateTopic(googleTestId); @@ -191,8 +195,6 @@ public String startConnector(Map connectorProps) throws Interrup + "." + AbstractKafkaSchemaSerDeConfig.SCHEMA_REGISTRY_URL_CONFIG, schemaRegistry.schemaRegistryUrl()); - connectorProps.put(BigtableSinkConfig.AUTO_CREATE_TABLES_CONFIG, "false"); - connectorProps.put(BigtableSinkConfig.AUTO_CREATE_COLUMN_FAMILIES_CONFIG, "true"); connectorProps.put(BigtableSinkConfig.ROW_KEY_DELIMITER_CONFIG, "#"); String topic = startSingleTopicConnector(connectorProps); connect @@ -237,6 +239,16 @@ public void populateTopic(String topic) throws IOException { } } + public Set getNeededColumnFamilies(String defaultColumnFamily) throws IOException { + return Stream.concat( + Stream.of(defaultColumnFamily), + Arrays.stream( + readStringResource(getTestCaseDir() + "/column-families.strings").split("\n")) + .map(String::trim) + .filter(s -> !s.isEmpty())) + .collect(Collectors.toSet()); + } + private static String readStringResource(String resourceName) throws IOException { byte[] resourceBytes = getClassLoader().getResourceAsStream(resourceName).readAllBytes(); return new String(resourceBytes, StandardCharsets.UTF_8); diff --git a/google-cloud-bigtable-kafka-connect-sink/src/test/java/com/google/cloud/kafka/connect/bigtable/integration/DifferentConvertersIT.java b/google-cloud-bigtable-kafka-connect-sink/src/test/java/com/google/cloud/kafka/connect/bigtable/integration/DifferentConvertersIT.java index 86d7d5e96a..f906199a3e 100644 --- a/google-cloud-bigtable-kafka-connect-sink/src/test/java/com/google/cloud/kafka/connect/bigtable/integration/DifferentConvertersIT.java +++ b/google-cloud-bigtable-kafka-connect-sink/src/test/java/com/google/cloud/kafka/connect/bigtable/integration/DifferentConvertersIT.java @@ -15,15 +15,14 @@ */ package com.google.cloud.kafka.connect.bigtable.integration; -import com.google.cloud.kafka.connect.bigtable.config.BigtableSinkConfig; import io.confluent.connect.avro.AvroConverter; import io.confluent.connect.json.JsonSchemaConverter; -import io.confluent.connect.protobuf.ProtobufConverter; import io.confluent.kafka.serializers.AbstractKafkaSchemaSerDeConfig; import java.util.Arrays; import java.util.Collection; import java.util.HashMap; import java.util.Map; +import java.util.concurrent.ExecutionException; import java.util.function.Supplier; import org.apache.kafka.connect.json.JsonConverter; import org.apache.kafka.connect.json.JsonConverterConfig; @@ -44,7 +43,9 @@ public static Collection testCases() { return Arrays.asList( new Object[][] { {(Supplier) AvroConverter::new, Map.of(), true}, - {(Supplier) ProtobufConverter::new, Map.of(), true}, + // TODO: uncomment once it starts using java-protobuf 4.x. Broken due to a dependency + // version mismatch for now. + // {(Supplier) ProtobufConverter::new, Map.of(), true}, {(Supplier) JsonSchemaConverter::new, Map.of(), true}, { (Supplier) JsonConverter::new, @@ -69,7 +70,7 @@ public DifferentConvertersIT( } @Test - public void testConverter() throws InterruptedException { + public void testConverter() throws InterruptedException, ExecutionException { Map converterProps = new HashMap<>(converterBaseConfig); if (converterUsesSchemaRegistry) { converterProps.put( @@ -81,7 +82,7 @@ public void testConverter() throws InterruptedException { Converter valueConverter = converterConstructor.get(); valueConverter.configure(converterProps, false); - Map connectorProps = connectorProps(); + Map connectorProps = baseConnectorProps(); for (Map.Entry prop : converterProps.entrySet()) { connectorProps.put( ConnectorConfig.KEY_CONVERTER_CLASS_CONFIG + "." + prop.getKey(), prop.getValue()); @@ -93,6 +94,7 @@ public void testConverter() throws InterruptedException { connectorProps.put( ConnectorConfig.VALUE_CONVERTER_CLASS_CONFIG, valueConverter.getClass().getName()); String topic = startSingleTopicConnector(connectorProps); + createTablesAndColumnFamilies(Map.of(topic, valueFields(topic))); connect .assertions() .assertConnectorAndExactlyNumTasksAreRunning(topic, numTasks, "Connector start timeout"); @@ -100,11 +102,4 @@ public void testConverter() throws InterruptedException { waitUntilBigtableContainsNumberOfRows(topic, numRecords); } - - private Map connectorProps() { - Map props = baseConnectorProps(); - props.put(BigtableSinkConfig.AUTO_CREATE_TABLES_CONFIG, "true"); - props.put(BigtableSinkConfig.AUTO_CREATE_COLUMN_FAMILIES_CONFIG, "true"); - return props; - } } diff --git a/google-cloud-bigtable-kafka-connect-sink/src/test/java/com/google/cloud/kafka/connect/bigtable/integration/ErrorHandlingIT.java b/google-cloud-bigtable-kafka-connect-sink/src/test/java/com/google/cloud/kafka/connect/bigtable/integration/ErrorHandlingIT.java index 9d6d572b00..c4aaf7d07a 100644 --- a/google-cloud-bigtable-kafka-connect-sink/src/test/java/com/google/cloud/kafka/connect/bigtable/integration/ErrorHandlingIT.java +++ b/google-cloud-bigtable-kafka-connect-sink/src/test/java/com/google/cloud/kafka/connect/bigtable/integration/ErrorHandlingIT.java @@ -70,11 +70,10 @@ public void testBigtableCredentialsAreCheckedOnStartup() { public void testTooLargeData() throws InterruptedException, ExecutionException { String dlqTopic = createDlq(); Map props = baseConnectorProps(); - props.put(BigtableSinkConfig.AUTO_CREATE_TABLES_CONFIG, String.valueOf(true)); - props.put(BigtableSinkConfig.AUTO_CREATE_COLUMN_FAMILIES_CONFIG, String.valueOf(true)); props.put(ConnectorConfig.VALUE_CONVERTER_CLASS_CONFIG, ByteArrayConverter.class.getName()); configureDlq(props, dlqTopic); String testId = startSingleTopicConnector(props); + createTablesAndColumnFamilies(testId); byte[] key = "key".getBytes(StandardCharsets.UTF_8); // The hard limit is 100 MB as per https://cloud.google.com/bigtable/quotas#limits-data-size @@ -87,13 +86,13 @@ public void testTooLargeData() throws InterruptedException, ExecutionException { } @Test - public void testSecondInsertIntoARowCausesAnError() throws InterruptedException { + public void testSecondInsertIntoARowCausesAnError() + throws InterruptedException, ExecutionException { String dlqTopic = createDlq(); Map props = baseConnectorProps(); - props.put(BigtableSinkConfig.AUTO_CREATE_TABLES_CONFIG, String.valueOf(true)); - props.put(BigtableSinkConfig.AUTO_CREATE_COLUMN_FAMILIES_CONFIG, String.valueOf(true)); configureDlq(props, dlqTopic); String testId = startSingleTopicConnector(props); + createTablesAndColumnFamilies(testId); String key = "key"; String valueOk = "ok"; @@ -116,13 +115,12 @@ public void testSecondInsertIntoARowCausesAnError() throws InterruptedException } @Test - public void testPartialBatchErrorWhenRelyingOnInputOrdering() throws InterruptedException { + public void testPartialBatchErrorWhenRelyingOnInputOrdering() + throws InterruptedException, ExecutionException { long dataSize = 1000; String dlqTopic = createDlq(); Map props = baseConnectorProps(); - props.put(BigtableSinkConfig.AUTO_CREATE_TABLES_CONFIG, String.valueOf(true)); - props.put(BigtableSinkConfig.AUTO_CREATE_COLUMN_FAMILIES_CONFIG, String.valueOf(true)); props.put(BigtableSinkConfig.INSERT_MODE_CONFIG, InsertMode.INSERT.name()); props.put( ConnectorConfig.CONNECTOR_CLIENT_CONSUMER_OVERRIDES_PREFIX @@ -138,6 +136,7 @@ public void testPartialBatchErrorWhenRelyingOnInputOrdering() throws Interrupted Integer.toString(Integer.MAX_VALUE)); configureDlq(props, dlqTopic); String testId = startSingleTopicConnector(props); + createTablesAndColumnFamilies(testId); connect.pauseConnector(testId); List> keysAndValues = new ArrayList<>(); @@ -190,8 +189,6 @@ public void testDeletingARowTwiceWorks() throws InterruptedException, ExecutionException, TimeoutException { String dlqTopic = createDlq(); Map props = baseConnectorProps(); - props.put(BigtableSinkConfig.AUTO_CREATE_TABLES_CONFIG, String.valueOf(true)); - props.put(BigtableSinkConfig.AUTO_CREATE_COLUMN_FAMILIES_CONFIG, String.valueOf(true)); props.put(BigtableSinkConfig.VALUE_NULL_MODE_CONFIG, NullValueMode.DELETE.name()); props.put(BigtableSinkConfig.INSERT_MODE_CONFIG, InsertMode.UPSERT.name()); configureDlq(props, dlqTopic); @@ -202,6 +199,7 @@ public void testDeletingARowTwiceWorks() + JsonConverterConfig.SCHEMAS_ENABLE_CONFIG, String.valueOf(false)); String testId = startSingleTopicConnector(props); + createTablesAndColumnFamilies(testId); String key = "key"; String putValue = "1"; @@ -228,8 +226,6 @@ public void testNonexistentCellDeletionWorks() throws InterruptedException, ExecutionException, TimeoutException { String dlqTopic = createDlq(); Map props = baseConnectorProps(); - props.put(BigtableSinkConfig.AUTO_CREATE_TABLES_CONFIG, String.valueOf(true)); - props.put(BigtableSinkConfig.AUTO_CREATE_COLUMN_FAMILIES_CONFIG, String.valueOf(true)); props.put(BigtableSinkConfig.VALUE_NULL_MODE_CONFIG, NullValueMode.DELETE.name()); props.put(BigtableSinkConfig.INSERT_MODE_CONFIG, InsertMode.UPSERT.name()); configureDlq(props, dlqTopic); @@ -240,6 +236,7 @@ public void testNonexistentCellDeletionWorks() + JsonConverterConfig.SCHEMAS_ENABLE_CONFIG, String.valueOf(true)); String testId = startSingleTopicConnector(props); + createTablesAndColumnFamilies(testId); String key = "nonexistentKey"; Struct innerStruct = @@ -260,8 +257,6 @@ public void testNonexistentColumnFamilyDeletionWorks() throws ExecutionException, InterruptedException, TimeoutException { String dlqTopic = createDlq(); Map props = baseConnectorProps(); - props.put(BigtableSinkConfig.AUTO_CREATE_TABLES_CONFIG, String.valueOf(true)); - props.put(BigtableSinkConfig.AUTO_CREATE_COLUMN_FAMILIES_CONFIG, String.valueOf(true)); props.put(BigtableSinkConfig.VALUE_NULL_MODE_CONFIG, NullValueMode.DELETE.name()); props.put(BigtableSinkConfig.INSERT_MODE_CONFIG, InsertMode.UPSERT.name()); configureDlq(props, dlqTopic); @@ -272,6 +267,7 @@ public void testNonexistentColumnFamilyDeletionWorks() + JsonConverterConfig.SCHEMAS_ENABLE_CONFIG, String.valueOf(true)); String testId = startSingleTopicConnector(props); + createTablesAndColumnFamilies(testId); String key = "nonexistentKey"; Struct struct = diff --git a/google-cloud-bigtable-kafka-connect-sink/src/test/java/com/google/cloud/kafka/connect/bigtable/integration/InsertUpsertIT.java b/google-cloud-bigtable-kafka-connect-sink/src/test/java/com/google/cloud/kafka/connect/bigtable/integration/InsertUpsertIT.java index 70a932315b..c29de51e54 100644 --- a/google-cloud-bigtable-kafka-connect-sink/src/test/java/com/google/cloud/kafka/connect/bigtable/integration/InsertUpsertIT.java +++ b/google-cloud-bigtable-kafka-connect-sink/src/test/java/com/google/cloud/kafka/connect/bigtable/integration/InsertUpsertIT.java @@ -26,6 +26,7 @@ import java.nio.charset.StandardCharsets; import java.util.Map; import java.util.Set; +import java.util.concurrent.ExecutionException; import java.util.stream.Collectors; import org.junit.Test; import org.junit.runner.RunWith; @@ -50,15 +51,14 @@ public class InsertUpsertIT extends BaseKafkaConnectBigtableIT { ByteString.copyFrom(VALUE3.getBytes(StandardCharsets.UTF_8)); @Test - public void testInsert() throws InterruptedException { + public void testInsert() throws InterruptedException, ExecutionException { String dlqTopic = createDlq(); Map props = baseConnectorProps(); - props.put(BigtableSinkConfig.AUTO_CREATE_TABLES_CONFIG, "true"); - props.put(BigtableSinkConfig.AUTO_CREATE_COLUMN_FAMILIES_CONFIG, "true"); props.put(BigtableSinkConfig.INSERT_MODE_CONFIG, InsertMode.INSERT.name()); props.put(BigtableSinkConfig.ERROR_MODE_CONFIG, BigtableErrorMode.IGNORE.name()); configureDlq(props, dlqTopic); String testId = startSingleTopicConnector(props); + createTablesAndColumnFamilies(testId); connect.kafka().produce(testId, KEY1, VALUE1); waitUntilBigtableContainsNumberOfRows(testId, 1); @@ -79,12 +79,11 @@ public void testInsert() throws InterruptedException { } @Test - public void testUpsert() throws InterruptedException { + public void testUpsert() throws InterruptedException, ExecutionException { Map props = baseConnectorProps(); - props.put(BigtableSinkConfig.AUTO_CREATE_TABLES_CONFIG, "true"); - props.put(BigtableSinkConfig.AUTO_CREATE_COLUMN_FAMILIES_CONFIG, "true"); props.put(BigtableSinkConfig.INSERT_MODE_CONFIG, InsertMode.UPSERT.name()); String testId = startSingleTopicConnector(props); + createTablesAndColumnFamilies(testId); connect.kafka().produce(testId, KEY1, VALUE1); waitUntilBigtableContainsNumberOfRows(testId, 1); diff --git a/google-cloud-bigtable-kafka-connect-sink/src/test/java/com/google/cloud/kafka/connect/bigtable/integration/MultipleConnectorTasksIT.java b/google-cloud-bigtable-kafka-connect-sink/src/test/java/com/google/cloud/kafka/connect/bigtable/integration/MultipleConnectorTasksIT.java index 84985e7f17..3598ca7fb3 100644 --- a/google-cloud-bigtable-kafka-connect-sink/src/test/java/com/google/cloud/kafka/connect/bigtable/integration/MultipleConnectorTasksIT.java +++ b/google-cloud-bigtable-kafka-connect-sink/src/test/java/com/google/cloud/kafka/connect/bigtable/integration/MultipleConnectorTasksIT.java @@ -17,12 +17,13 @@ import static org.junit.Assert.assertEquals; -import com.google.cloud.bigtable.admin.v2.models.CreateTableRequest; import com.google.cloud.kafka.connect.bigtable.config.BigtableSinkConfig; import com.google.cloud.kafka.connect.bigtable.config.InsertMode; import io.confluent.connect.avro.AvroConverter; import io.confluent.kafka.serializers.AbstractKafkaSchemaSerDeConfig; import java.util.Map; +import java.util.Set; +import java.util.concurrent.ExecutionException; import org.apache.kafka.connect.runtime.ConnectorConfig; import org.apache.kafka.connect.storage.Converter; import org.junit.Test; @@ -35,7 +36,7 @@ public class MultipleConnectorTasksIT extends BaseDataGeneratorIT { // to a too low value and retries of table/column family creation exhausted the quota that should // be used for creation of different column families. @Test - public void testMultipleTasks() throws InterruptedException { + public void testMultipleTasks() throws InterruptedException, ExecutionException { numRecords = 1000L; numTasks = 10; @@ -49,8 +50,6 @@ public void testMultipleTasks() throws InterruptedException { valueConverter.configure(converterProps, false); Map connectorProps = baseConnectorProps(); - connectorProps.put(BigtableSinkConfig.AUTO_CREATE_TABLES_CONFIG, "true"); - connectorProps.put(BigtableSinkConfig.AUTO_CREATE_COLUMN_FAMILIES_CONFIG, "true"); connectorProps.put(BigtableSinkConfig.INSERT_MODE_CONFIG, InsertMode.UPSERT.name()); for (Map.Entry prop : converterProps.entrySet()) { @@ -65,6 +64,7 @@ public void testMultipleTasks() throws InterruptedException { ConnectorConfig.VALUE_CONVERTER_CLASS_CONFIG, valueConverter.getClass().getName()); String testId = startSingleTopicConnector(connectorProps); + createTablesAndColumnFamilies(Map.of(testId, valueFields(testId))); connect .assertions() .assertConnectorAndExactlyNumTasksAreRunning(testId, numTasks, "Connector start timeout"); @@ -75,7 +75,7 @@ public void testMultipleTasks() throws InterruptedException { } @Test - public void testRestartPauseStop() throws InterruptedException { + public void testRestartPauseStop() throws InterruptedException, ExecutionException { numTasks = 10; int expectedRowsInBigtable = 0; String defaultColumnFamily = "default"; @@ -83,7 +83,7 @@ public void testRestartPauseStop() throws InterruptedException { Map connectorProps = baseConnectorProps(); connectorProps.put(BigtableSinkConfig.DEFAULT_COLUMN_FAMILY_CONFIG, defaultColumnFamily); String testId = startSingleTopicConnector(connectorProps); - bigtableAdmin.createTable(CreateTableRequest.of(testId).addFamily(defaultColumnFamily)); + createTablesAndColumnFamilies(Map.of(testId, Set.of(defaultColumnFamily))); connect .assertions() diff --git a/google-cloud-bigtable-kafka-connect-sink/src/test/java/com/google/cloud/kafka/connect/bigtable/integration/NullHandlingIT.java b/google-cloud-bigtable-kafka-connect-sink/src/test/java/com/google/cloud/kafka/connect/bigtable/integration/NullHandlingIT.java index 2bd4a1a3ee..30d3057082 100644 --- a/google-cloud-bigtable-kafka-connect-sink/src/test/java/com/google/cloud/kafka/connect/bigtable/integration/NullHandlingIT.java +++ b/google-cloud-bigtable-kafka-connect-sink/src/test/java/com/google/cloud/kafka/connect/bigtable/integration/NullHandlingIT.java @@ -21,7 +21,6 @@ import static org.junit.Assert.assertEquals; import static org.junit.Assert.assertTrue; -import com.google.cloud.bigtable.admin.v2.models.CreateTableRequest; import com.google.cloud.bigtable.data.v2.models.Row; import com.google.cloud.bigtable.data.v2.models.RowCell; import com.google.cloud.bigtable.data.v2.models.RowMutation; @@ -36,6 +35,7 @@ import java.util.List; import java.util.Map; import java.util.Set; +import java.util.concurrent.ExecutionException; import java.util.stream.Collectors; import org.apache.kafka.connect.data.Schema; import org.apache.kafka.connect.data.SchemaAndValue; @@ -74,10 +74,11 @@ public class NullHandlingIT extends BaseKafkaConnectBigtableIT { } @Test - public void testIgnoreMode() throws InterruptedException { + public void testIgnoreMode() throws InterruptedException, ExecutionException { Map connectorProps = connectorProps(); connectorProps.put(BigtableSinkConfig.VALUE_NULL_MODE_CONFIG, NullValueMode.IGNORE.name()); String testId = startSingleTopicConnector(connectorProps); + createTablesAndColumnFamilies(Map.of(testId, Set.of(testId, NESTED_NULL_STRUCT_FIELD_NAME))); connect .assertions() .assertConnectorAndExactlyNumTasksAreRunning(testId, numTasks, "Connector start timeout"); @@ -125,7 +126,7 @@ public void testIgnoreMode() throws InterruptedException { } @Test - public void testWriteMode() throws InterruptedException { + public void testWriteMode() throws InterruptedException, ExecutionException { String defaultColumnFamily = "family"; String defaultColumnQualifier = "qualifier"; ByteString defaultColumnQualifierBytes = @@ -136,6 +137,8 @@ public void testWriteMode() throws InterruptedException { connectorProps.put(BigtableSinkConfig.DEFAULT_COLUMN_FAMILY_CONFIG, defaultColumnFamily); connectorProps.put(BigtableSinkConfig.DEFAULT_COLUMN_QUALIFIER_CONFIG, defaultColumnQualifier); String testId = startSingleTopicConnector(connectorProps); + createTablesAndColumnFamilies( + Map.of(testId, Set.of(testId, defaultColumnFamily, NESTED_NULL_STRUCT_FIELD_NAME))); connect .assertions() .assertConnectorAndExactlyNumTasksAreRunning(testId, numTasks, "Connector start timeout"); @@ -191,21 +194,22 @@ public void testWriteMode() throws InterruptedException { } @Test - public void testDeleteMode() throws InterruptedException { + public void testDeleteMode() throws InterruptedException, ExecutionException { + String columnFamily1 = "cf1"; + String columnFamily2 = "cf2"; + String columnQualifier1 = "cq1"; + String columnQualifier2 = "cq2"; Map connectorProps = connectorProps(); connectorProps.put(BigtableSinkConfig.VALUE_NULL_MODE_CONFIG, NullValueMode.DELETE.name()); connectorProps.put(BigtableSinkConfig.INSERT_MODE_CONFIG, InsertMode.UPSERT.name()); String testId = startSingleTopicConnector(connectorProps); + createTablesAndColumnFamilies( + Map.of( + testId, Set.of(testId, NESTED_NULL_STRUCT_FIELD_NAME, columnFamily1, columnFamily2))); connect .assertions() .assertConnectorAndExactlyNumTasksAreRunning(testId, numTasks, "Connector start timeout"); - String columnFamily1 = "cf1"; - String columnFamily2 = "cf2"; - bigtableAdmin.createTable( - CreateTableRequest.of(testId).addFamily(columnFamily1).addFamily(columnFamily2)); - String columnQualifier1 = "cq1"; - String columnQualifier2 = "cq2"; ByteString columnQualifierBytes1 = ByteString.copyFrom(columnQualifier1.getBytes(StandardCharsets.UTF_8)); ByteString columnQualifierBytes2 = @@ -327,8 +331,6 @@ public void testDeleteMode() throws InterruptedException { private Map connectorProps() { Map props = super.baseConnectorProps(); - props.put(BigtableSinkConfig.AUTO_CREATE_TABLES_CONFIG, "true"); - props.put(BigtableSinkConfig.AUTO_CREATE_COLUMN_FAMILIES_CONFIG, "true"); // We use JsonConverter since it doesn't care about schemas, so we may use differently-shaped // data within a single test. props.put(ConnectorConfig.VALUE_CONVERTER_CLASS_CONFIG, JsonConverter.class.getName()); diff --git a/google-cloud-bigtable-kafka-connect-sink/src/test/java/com/google/cloud/kafka/connect/bigtable/integration/ResourceAutoCreationIT.java b/google-cloud-bigtable-kafka-connect-sink/src/test/java/com/google/cloud/kafka/connect/bigtable/integration/ResourceAutoCreationIT.java index 4744b82cc3..e416400e0c 100644 --- a/google-cloud-bigtable-kafka-connect-sink/src/test/java/com/google/cloud/kafka/connect/bigtable/integration/ResourceAutoCreationIT.java +++ b/google-cloud-bigtable-kafka-connect-sink/src/test/java/com/google/cloud/kafka/connect/bigtable/integration/ResourceAutoCreationIT.java @@ -21,7 +21,6 @@ import static org.junit.jupiter.api.Assertions.assertThrows; import com.google.cloud.bigtable.admin.v2.models.ColumnFamily; -import com.google.cloud.bigtable.admin.v2.models.CreateTableRequest; import com.google.cloud.bigtable.admin.v2.models.ModifyColumnFamiliesRequest; import com.google.cloud.bigtable.data.v2.models.Range; import com.google.cloud.kafka.connect.bigtable.config.BigtableSinkConfig; @@ -63,7 +62,7 @@ public class ResourceAutoCreationIT extends BaseKafkaConnectBigtableIT { private static final String COLUMN_QUALIFIER = "cq"; @Test - public void testDisabledResourceAutoCreation() throws InterruptedException { + public void testDisabledResourceAutoCreation() throws InterruptedException, ExecutionException { String dlqTopic = createDlq(); Map props = baseConnectorProps(); configureDlq(props, dlqTopic); @@ -97,7 +96,7 @@ public void testDisabledResourceAutoCreation() throws InterruptedException { assertConnectorAndAllTasksAreRunning(testId); // With the table and column family created. - bigtableAdmin.createTable(CreateTableRequest.of(testId).addFamily(COLUMN_FAMILY1)); + createTablesAndColumnFamilies(Map.of(testId, Set.of(COLUMN_FAMILY1))); connect.kafka().produce(testId, KEY2, serializedValue1); waitUntilBigtableContainsNumberOfRows(testId, 1); assertTrue( @@ -152,7 +151,7 @@ public void testTableAutoCreationEnabledColumnFamilyAutoCreationDisabled() @Test public void testTableAutoCreationDisabledColumnFamilyAutoCreationEnabled() - throws InterruptedException { + throws InterruptedException, ExecutionException { String dlqTopic = createDlq(); Map props = baseConnectorProps(); configureDlq(props, dlqTopic); @@ -170,7 +169,7 @@ public void testTableAutoCreationDisabledColumnFamilyAutoCreationEnabled() connect.kafka().produce(testId, KEY1, value); assertSingleDlqEntry(dlqTopic, KEY1, value, null); - bigtableAdmin.createTable(CreateTableRequest.of(testId)); + createTablesAndColumnFamilies(Map.of(testId, Set.of())); assertTrue(bigtableAdmin.getTable(testId).getColumnFamilies().isEmpty()); connect.kafka().produce(testId, KEY2, value); waitUntilBigtableContainsNumberOfRows(testId, 1); diff --git a/google-cloud-bigtable-kafka-connect-sink/src/test/resources/compatibility_test_cases/key_bytes/column-families.strings b/google-cloud-bigtable-kafka-connect-sink/src/test/resources/compatibility_test_cases/key_bytes/column-families.strings new file mode 100644 index 0000000000..e69de29bb2 diff --git a/google-cloud-bigtable-kafka-connect-sink/src/test/resources/compatibility_test_cases/key_containers/column-families.strings b/google-cloud-bigtable-kafka-connect-sink/src/test/resources/compatibility_test_cases/key_containers/column-families.strings new file mode 100644 index 0000000000..e69de29bb2 diff --git a/google-cloud-bigtable-kafka-connect-sink/src/test/resources/compatibility_test_cases/key_logicals/column-families.strings b/google-cloud-bigtable-kafka-connect-sink/src/test/resources/compatibility_test_cases/key_logicals/column-families.strings new file mode 100644 index 0000000000..e69de29bb2 diff --git a/google-cloud-bigtable-kafka-connect-sink/src/test/resources/compatibility_test_cases/key_matryoshkas/column-families.strings b/google-cloud-bigtable-kafka-connect-sink/src/test/resources/compatibility_test_cases/key_matryoshkas/column-families.strings new file mode 100644 index 0000000000..e69de29bb2 diff --git a/google-cloud-bigtable-kafka-connect-sink/src/test/resources/compatibility_test_cases/key_nestedlogicals/column-families.strings b/google-cloud-bigtable-kafka-connect-sink/src/test/resources/compatibility_test_cases/key_nestedlogicals/column-families.strings new file mode 100644 index 0000000000..e69de29bb2 diff --git a/google-cloud-bigtable-kafka-connect-sink/src/test/resources/compatibility_test_cases/key_primitives/column-families.strings b/google-cloud-bigtable-kafka-connect-sink/src/test/resources/compatibility_test_cases/key_primitives/column-families.strings new file mode 100644 index 0000000000..e69de29bb2 diff --git a/google-cloud-bigtable-kafka-connect-sink/src/test/resources/compatibility_test_cases/key_root_primitives/column-families.strings b/google-cloud-bigtable-kafka-connect-sink/src/test/resources/compatibility_test_cases/key_root_primitives/column-families.strings new file mode 100644 index 0000000000..e69de29bb2 diff --git a/google-cloud-bigtable-kafka-connect-sink/src/test/resources/compatibility_test_cases/key_union/column-families.strings b/google-cloud-bigtable-kafka-connect-sink/src/test/resources/compatibility_test_cases/key_union/column-families.strings new file mode 100644 index 0000000000..e69de29bb2 diff --git a/google-cloud-bigtable-kafka-connect-sink/src/test/resources/compatibility_test_cases/value_bytes/column-families.strings b/google-cloud-bigtable-kafka-connect-sink/src/test/resources/compatibility_test_cases/value_bytes/column-families.strings new file mode 100644 index 0000000000..e69de29bb2 diff --git a/google-cloud-bigtable-kafka-connect-sink/src/test/resources/compatibility_test_cases/value_containers/column-families.strings b/google-cloud-bigtable-kafka-connect-sink/src/test/resources/compatibility_test_cases/value_containers/column-families.strings new file mode 100644 index 0000000000..f460184231 --- /dev/null +++ b/google-cloud-bigtable-kafka-connect-sink/src/test/resources/compatibility_test_cases/value_containers/column-families.strings @@ -0,0 +1,3 @@ +map +array +struct diff --git a/google-cloud-bigtable-kafka-connect-sink/src/test/resources/compatibility_test_cases/value_logicals/column-families.strings b/google-cloud-bigtable-kafka-connect-sink/src/test/resources/compatibility_test_cases/value_logicals/column-families.strings new file mode 100644 index 0000000000..e69de29bb2 diff --git a/google-cloud-bigtable-kafka-connect-sink/src/test/resources/compatibility_test_cases/value_matryoshkas/column-families.strings b/google-cloud-bigtable-kafka-connect-sink/src/test/resources/compatibility_test_cases/value_matryoshkas/column-families.strings new file mode 100644 index 0000000000..c26b3ff7df --- /dev/null +++ b/google-cloud-bigtable-kafka-connect-sink/src/test/resources/compatibility_test_cases/value_matryoshkas/column-families.strings @@ -0,0 +1,2 @@ +middle +inner diff --git a/google-cloud-bigtable-kafka-connect-sink/src/test/resources/compatibility_test_cases/value_nestedlogicals/column-families.strings b/google-cloud-bigtable-kafka-connect-sink/src/test/resources/compatibility_test_cases/value_nestedlogicals/column-families.strings new file mode 100644 index 0000000000..6d2abcd542 --- /dev/null +++ b/google-cloud-bigtable-kafka-connect-sink/src/test/resources/compatibility_test_cases/value_nestedlogicals/column-families.strings @@ -0,0 +1 @@ +outer diff --git a/google-cloud-bigtable-kafka-connect-sink/src/test/resources/compatibility_test_cases/value_nulls/column-families.strings b/google-cloud-bigtable-kafka-connect-sink/src/test/resources/compatibility_test_cases/value_nulls/column-families.strings new file mode 100644 index 0000000000..e69de29bb2 diff --git a/google-cloud-bigtable-kafka-connect-sink/src/test/resources/compatibility_test_cases/value_primitives/column-families.strings b/google-cloud-bigtable-kafka-connect-sink/src/test/resources/compatibility_test_cases/value_primitives/column-families.strings new file mode 100644 index 0000000000..e69de29bb2 diff --git a/google-cloud-bigtable-kafka-connect-sink/src/test/resources/compatibility_test_cases/value_root_primitives/column-families.strings b/google-cloud-bigtable-kafka-connect-sink/src/test/resources/compatibility_test_cases/value_root_primitives/column-families.strings new file mode 100644 index 0000000000..e69de29bb2 diff --git a/google-cloud-bigtable-kafka-connect-sink/src/test/resources/compatibility_test_cases/value_union/column-families.strings b/google-cloud-bigtable-kafka-connect-sink/src/test/resources/compatibility_test_cases/value_union/column-families.strings new file mode 100644 index 0000000000..e69de29bb2 From fb8b369c79383febc32805144dd15fdd3baeed3e Mon Sep 17 00:00:00 2001 From: Adam Czajkowski Date: Mon, 10 Mar 2025 14:59:06 +0100 Subject: [PATCH 68/76] Avoid using deprecated methods --- .../connect/bigtable/integration/BaseKafkaConnectIT.java | 2 +- .../bigtable/integration/ConfluentCompatibilityIT.java | 4 ++-- 2 files changed, 3 insertions(+), 3 deletions(-) diff --git a/google-cloud-bigtable-kafka-connect-sink/src/test/java/com/google/cloud/kafka/connect/bigtable/integration/BaseKafkaConnectIT.java b/google-cloud-bigtable-kafka-connect-sink/src/test/java/com/google/cloud/kafka/connect/bigtable/integration/BaseKafkaConnectIT.java index 6fadbd3899..d2ce0a1704 100644 --- a/google-cloud-bigtable-kafka-connect-sink/src/test/java/com/google/cloud/kafka/connect/bigtable/integration/BaseKafkaConnectIT.java +++ b/google-cloud-bigtable-kafka-connect-sink/src/test/java/com/google/cloud/kafka/connect/bigtable/integration/BaseKafkaConnectIT.java @@ -104,7 +104,7 @@ public void startConnect() { .numBrokers(numBrokers) .brokerProps(brokerProps) .workerProps(workerProps()) - .clientConfigs(clientConfigs) + .clientProps(clientConfigs) .build(); // Start the clusters diff --git a/google-cloud-bigtable-kafka-connect-sink/src/test/java/com/google/cloud/kafka/connect/bigtable/integration/ConfluentCompatibilityIT.java b/google-cloud-bigtable-kafka-connect-sink/src/test/java/com/google/cloud/kafka/connect/bigtable/integration/ConfluentCompatibilityIT.java index d23b0d5fa9..8e9ea58018 100644 --- a/google-cloud-bigtable-kafka-connect-sink/src/test/java/com/google/cloud/kafka/connect/bigtable/integration/ConfluentCompatibilityIT.java +++ b/google-cloud-bigtable-kafka-connect-sink/src/test/java/com/google/cloud/kafka/connect/bigtable/integration/ConfluentCompatibilityIT.java @@ -32,6 +32,7 @@ import com.google.protobuf.ByteString; import io.confluent.connect.avro.AvroConverter; import io.confluent.kafka.formatter.AvroMessageReader; +import io.confluent.kafka.formatter.SchemaMessageReader; import io.confluent.kafka.serializers.AbstractKafkaSchemaSerDeConfig; import java.io.IOException; import java.io.InputStream; @@ -49,7 +50,6 @@ import java.util.concurrent.TimeoutException; import java.util.stream.Collectors; import java.util.stream.Stream; -import kafka.common.MessageReader; import org.apache.kafka.clients.producer.Producer; import org.apache.kafka.clients.producer.ProducerRecord; import org.apache.kafka.connect.runtime.ConnectorConfig; @@ -224,7 +224,7 @@ public void populateTopic(String topic) throws IOException { messageReaderProps.put("key.schema", keySchema); messageReaderProps.put("value.schema", valueSchema); InputStream dataStream = getClassLoader().getResourceAsStream(getTestCaseDir() + "/data.json"); - MessageReader messageReader = new AvroMessageReader(); + SchemaMessageReader messageReader = new AvroMessageReader(); messageReader.init(dataStream, messageReaderProps); Producer kafkaProducer = getKafkaProducer(); From 6a70a16610b534b178324761954ce1eb8656361f Mon Sep 17 00:00:00 2001 From: Adam Czajkowski Date: Mon, 10 Mar 2025 16:48:45 +0100 Subject: [PATCH 69/76] Upgrade JUnit to 4.13.2 --- google-cloud-bigtable-kafka-connect-sink/pom.xml | 2 +- .../cloud/kafka/connect/bigtable/BigtableSinkTaskTest.java | 2 +- .../connect/bigtable/autocreate/BigtableSchemaManagerTest.java | 2 +- .../kafka/connect/bigtable/config/BigtableSinkConfigTest.java | 2 +- .../kafka/connect/bigtable/integration/ErrorHandlingIT.java | 2 +- .../connect/bigtable/integration/ResourceAutoCreationIT.java | 2 +- .../cloud/kafka/connect/bigtable/mapping/KeyMapperTest.java | 2 +- 7 files changed, 7 insertions(+), 7 deletions(-) diff --git a/google-cloud-bigtable-kafka-connect-sink/pom.xml b/google-cloud-bigtable-kafka-connect-sink/pom.xml index 61681a2f81..c23d34e578 100644 --- a/google-cloud-bigtable-kafka-connect-sink/pom.xml +++ b/google-cloud-bigtable-kafka-connect-sink/pom.xml @@ -21,7 +21,7 @@ 2.12.7 5.15.2 - 4.8.2 + 4.13.2 + 5.11.3 + 7.7.0 + 9.4.57.v20241219 + + 3.25.6 + 3.4.1 + + ${project.basedir}/integration_test_plugins + https://hub-downloads.confluent.io/api/plugins/confluentinc/kafka-connect-gcp-bigtable/versions/2.0.28/confluentinc-kafka-connect-gcp-bigtable-2.0.28.zip + confluentsink.zip + ${project.basedir}/../sink/target/sink-${project.version}-package + ${integration.test.plugin.path}/google-sink + + + + org.apache.kafka + connect-api + ${kafka.version} + + + org.apache.kafka + connect-runtime + ${kafka.version} + + + org.apache.kafka + kafka-clients + ${kafka.version} + + + com.google.protobuf + protobuf-java + ${protobuf3.version} + + + com.google.cloud + google-cloud-bigtable + ${google.bigtable.version} + + + org.apache.hbase + hbase-common + ${hbase.version} + + + org.slf4j + slf4j-api + ${slf4j.api.version} + + + org.slf4j + slf4j-reload4j + ${slf4j.reload4j.version} + + + com.fasterxml.jackson.core + jackson-databind + ${jackson.version} + + + junit + junit + ${junit.version} + + + org.apache.kafka + connect-runtime + ${kafka.version} + test + + + org.apache.kafka + kafka-clients + ${kafka.version} + test + test-jar + + + org.apache.kafka + kafka_${kafka.scala.version} + ${kafka.version} + + + org.apache.kafka + kafka_${kafka.scala.version} + ${kafka.version} + test + test-jar + + + org.mockito + mockito-core + ${mockito.version} + + + org.junit.jupiter + junit-jupiter-api + ${junit.jupiter.version} + + + io.confluent + kafka-schema-registry + ${confluent.version} + + + io.confluent + kafka-schema-registry + tests + test-jar + ${confluent.version} + + + io.confluent + kafka-avro-serializer + ${confluent.version} + + + io.confluent + kafka-connect-avro-converter + ${confluent.version} + + + io.confluent + kafka-connect-protobuf-converter + ${confluent.version} + + + io.confluent + kafka-connect-json-schema-converter + ${confluent.version} + + + org.eclipse.jetty + jetty-server + ${jetty.version} + + + org.eclipse.jetty + jetty-servlet + ${jetty.version} + + + org.eclipse.jetty + jetty-servlets + ${jetty.version} + + + org.eclipse.jetty + jetty-client + ${jetty.version} + + + + + + org.apache.maven.plugins + maven-failsafe-plugin + ${failsafe.version} + + + ${skipIntegrationTests} + false + + classes + true + + + target/test-classes/fake_service_key.json + + localhost:8086 + ${integration.test.plugin.path} + + + + + + verify + integration-test + + + + + + org.apache.maven.plugins + maven-antrun-plugin + ${antrun.version} + + + download-and-unzip-confluent-sink + pre-integration-test + + run + + + + + + + + + + + + + + + + maven-clean-plugin + ${maven.clean.version} + + + + ${integration.test.plugin.path} + + + + + + + + + confluent + https://packages.confluent.io/maven/ + + + \ No newline at end of file diff --git a/google-cloud-bigtable-kafka-connect-sink/integration-tests/src/test/java/com/google/cloud/kafka/connect/bigtable/config/BigtableErrorMode.java b/google-cloud-bigtable-kafka-connect-sink/integration-tests/src/test/java/com/google/cloud/kafka/connect/bigtable/config/BigtableErrorMode.java new file mode 120000 index 0000000000..407f5c71e8 --- /dev/null +++ b/google-cloud-bigtable-kafka-connect-sink/integration-tests/src/test/java/com/google/cloud/kafka/connect/bigtable/config/BigtableErrorMode.java @@ -0,0 +1 @@ +../../../../../../../../../../../sink/src/main/java/com/google/cloud/kafka/connect/bigtable/config/BigtableErrorMode.java \ No newline at end of file diff --git a/google-cloud-bigtable-kafka-connect-sink/integration-tests/src/test/java/com/google/cloud/kafka/connect/bigtable/config/BigtableSinkConfig.java b/google-cloud-bigtable-kafka-connect-sink/integration-tests/src/test/java/com/google/cloud/kafka/connect/bigtable/config/BigtableSinkConfig.java new file mode 120000 index 0000000000..b7509e2c81 --- /dev/null +++ b/google-cloud-bigtable-kafka-connect-sink/integration-tests/src/test/java/com/google/cloud/kafka/connect/bigtable/config/BigtableSinkConfig.java @@ -0,0 +1 @@ +../../../../../../../../../../../sink/src/main/java/com/google/cloud/kafka/connect/bigtable/config/BigtableSinkConfig.java \ No newline at end of file diff --git a/google-cloud-bigtable-kafka-connect-sink/integration-tests/src/test/java/com/google/cloud/kafka/connect/bigtable/config/BigtableSinkTaskConfig.java b/google-cloud-bigtable-kafka-connect-sink/integration-tests/src/test/java/com/google/cloud/kafka/connect/bigtable/config/BigtableSinkTaskConfig.java new file mode 120000 index 0000000000..06092b9f05 --- /dev/null +++ b/google-cloud-bigtable-kafka-connect-sink/integration-tests/src/test/java/com/google/cloud/kafka/connect/bigtable/config/BigtableSinkTaskConfig.java @@ -0,0 +1 @@ +../../../../../../../../../../../sink/src/main/java/com/google/cloud/kafka/connect/bigtable/config/BigtableSinkTaskConfig.java \ No newline at end of file diff --git a/google-cloud-bigtable-kafka-connect-sink/integration-tests/src/test/java/com/google/cloud/kafka/connect/bigtable/config/ConfigInterpolation.java b/google-cloud-bigtable-kafka-connect-sink/integration-tests/src/test/java/com/google/cloud/kafka/connect/bigtable/config/ConfigInterpolation.java new file mode 120000 index 0000000000..5eb72c027f --- /dev/null +++ b/google-cloud-bigtable-kafka-connect-sink/integration-tests/src/test/java/com/google/cloud/kafka/connect/bigtable/config/ConfigInterpolation.java @@ -0,0 +1 @@ +../../../../../../../../../../../sink/src/main/java/com/google/cloud/kafka/connect/bigtable/config/ConfigInterpolation.java \ No newline at end of file diff --git a/google-cloud-bigtable-kafka-connect-sink/integration-tests/src/test/java/com/google/cloud/kafka/connect/bigtable/config/InsertMode.java b/google-cloud-bigtable-kafka-connect-sink/integration-tests/src/test/java/com/google/cloud/kafka/connect/bigtable/config/InsertMode.java new file mode 120000 index 0000000000..ef27286c97 --- /dev/null +++ b/google-cloud-bigtable-kafka-connect-sink/integration-tests/src/test/java/com/google/cloud/kafka/connect/bigtable/config/InsertMode.java @@ -0,0 +1 @@ +../../../../../../../../../../../sink/src/main/java/com/google/cloud/kafka/connect/bigtable/config/InsertMode.java \ No newline at end of file diff --git a/google-cloud-bigtable-kafka-connect-sink/integration-tests/src/test/java/com/google/cloud/kafka/connect/bigtable/config/NullValueMode.java b/google-cloud-bigtable-kafka-connect-sink/integration-tests/src/test/java/com/google/cloud/kafka/connect/bigtable/config/NullValueMode.java new file mode 120000 index 0000000000..82e2851fa6 --- /dev/null +++ b/google-cloud-bigtable-kafka-connect-sink/integration-tests/src/test/java/com/google/cloud/kafka/connect/bigtable/config/NullValueMode.java @@ -0,0 +1 @@ +../../../../../../../../../../../sink/src/main/java/com/google/cloud/kafka/connect/bigtable/config/NullValueMode.java \ No newline at end of file diff --git a/google-cloud-bigtable-kafka-connect-sink/integration-tests/src/test/java/com/google/cloud/kafka/connect/bigtable/exception/InvalidBigtableSchemaModificationException.java b/google-cloud-bigtable-kafka-connect-sink/integration-tests/src/test/java/com/google/cloud/kafka/connect/bigtable/exception/InvalidBigtableSchemaModificationException.java new file mode 120000 index 0000000000..24c65448d2 --- /dev/null +++ b/google-cloud-bigtable-kafka-connect-sink/integration-tests/src/test/java/com/google/cloud/kafka/connect/bigtable/exception/InvalidBigtableSchemaModificationException.java @@ -0,0 +1 @@ +../../../../../../../../../../../sink/src/main/java/com/google/cloud/kafka/connect/bigtable/exception/InvalidBigtableSchemaModificationException.java \ No newline at end of file diff --git a/google-cloud-bigtable-kafka-connect-sink/src/test/java/com/google/cloud/kafka/connect/bigtable/integration/BaseDataGeneratorIT.java b/google-cloud-bigtable-kafka-connect-sink/integration-tests/src/test/java/com/google/cloud/kafka/connect/bigtable/integration/BaseDataGeneratorIT.java similarity index 100% rename from google-cloud-bigtable-kafka-connect-sink/src/test/java/com/google/cloud/kafka/connect/bigtable/integration/BaseDataGeneratorIT.java rename to google-cloud-bigtable-kafka-connect-sink/integration-tests/src/test/java/com/google/cloud/kafka/connect/bigtable/integration/BaseDataGeneratorIT.java diff --git a/google-cloud-bigtable-kafka-connect-sink/src/test/java/com/google/cloud/kafka/connect/bigtable/integration/BaseIT.java b/google-cloud-bigtable-kafka-connect-sink/integration-tests/src/test/java/com/google/cloud/kafka/connect/bigtable/integration/BaseIT.java similarity index 93% rename from google-cloud-bigtable-kafka-connect-sink/src/test/java/com/google/cloud/kafka/connect/bigtable/integration/BaseIT.java rename to google-cloud-bigtable-kafka-connect-sink/integration-tests/src/test/java/com/google/cloud/kafka/connect/bigtable/integration/BaseIT.java index 81e9ea7ea4..c5639773e0 100644 --- a/google-cloud-bigtable-kafka-connect-sink/src/test/java/com/google/cloud/kafka/connect/bigtable/integration/BaseIT.java +++ b/google-cloud-bigtable-kafka-connect-sink/integration-tests/src/test/java/com/google/cloud/kafka/connect/bigtable/integration/BaseIT.java @@ -24,7 +24,6 @@ import com.google.cloud.bigtable.admin.v2.BigtableTableAdminClient; import com.google.cloud.bigtable.data.v2.BigtableDataClient; -import com.google.cloud.kafka.connect.bigtable.BigtableSinkConnector; import com.google.cloud.kafka.connect.bigtable.config.BigtableSinkConfig; import com.google.cloud.kafka.connect.bigtable.util.TestId; import java.util.HashMap; @@ -37,7 +36,9 @@ public abstract class BaseIT { // The Confluent's sink cannot use Application Default Credentials. We set the credentials // explicitly (rather than use ADC) to avoid having a special case for ConfluentCompatibilityIT. - public String CREDENTIALS_PATH_ENV_VAR = "GOOGLE_APPLICATION_CREDENTIALS"; + public static final String CREDENTIALS_PATH_ENV_VAR = "GOOGLE_APPLICATION_CREDENTIALS"; + public static final String CONNECTOR_CLASS_NAME = + "com.google.cloud.kafka.connect.bigtable.BigtableSinkConnector"; public int numTasks = 1; public int maxKafkaMessageSizeBytes = 300 * 1024 * 1024; @@ -45,7 +46,7 @@ public abstract class BaseIT { public Map baseConnectorProps() { Map result = new HashMap<>(); - result.put(CONNECTOR_CLASS_CONFIG, BigtableSinkConnector.class.getName()); + result.put(CONNECTOR_CLASS_CONFIG, CONNECTOR_CLASS_NAME); result.put(TASKS_MAX_CONFIG, Integer.toString(numTasks)); result.put(KEY_CONVERTER_CLASS_CONFIG, StringConverter.class.getName()); result.put(VALUE_CONVERTER_CLASS_CONFIG, StringConverter.class.getName()); diff --git a/google-cloud-bigtable-kafka-connect-sink/src/test/java/com/google/cloud/kafka/connect/bigtable/integration/BaseKafkaConnectBigtableIT.java b/google-cloud-bigtable-kafka-connect-sink/integration-tests/src/test/java/com/google/cloud/kafka/connect/bigtable/integration/BaseKafkaConnectBigtableIT.java similarity index 100% rename from google-cloud-bigtable-kafka-connect-sink/src/test/java/com/google/cloud/kafka/connect/bigtable/integration/BaseKafkaConnectBigtableIT.java rename to google-cloud-bigtable-kafka-connect-sink/integration-tests/src/test/java/com/google/cloud/kafka/connect/bigtable/integration/BaseKafkaConnectBigtableIT.java diff --git a/google-cloud-bigtable-kafka-connect-sink/src/test/java/com/google/cloud/kafka/connect/bigtable/integration/BaseKafkaConnectBigtableSchemaRegistryIT.java b/google-cloud-bigtable-kafka-connect-sink/integration-tests/src/test/java/com/google/cloud/kafka/connect/bigtable/integration/BaseKafkaConnectBigtableSchemaRegistryIT.java similarity index 100% rename from google-cloud-bigtable-kafka-connect-sink/src/test/java/com/google/cloud/kafka/connect/bigtable/integration/BaseKafkaConnectBigtableSchemaRegistryIT.java rename to google-cloud-bigtable-kafka-connect-sink/integration-tests/src/test/java/com/google/cloud/kafka/connect/bigtable/integration/BaseKafkaConnectBigtableSchemaRegistryIT.java diff --git a/google-cloud-bigtable-kafka-connect-sink/src/test/java/com/google/cloud/kafka/connect/bigtable/integration/BaseKafkaConnectIT.java b/google-cloud-bigtable-kafka-connect-sink/integration-tests/src/test/java/com/google/cloud/kafka/connect/bigtable/integration/BaseKafkaConnectIT.java similarity index 97% rename from google-cloud-bigtable-kafka-connect-sink/src/test/java/com/google/cloud/kafka/connect/bigtable/integration/BaseKafkaConnectIT.java rename to google-cloud-bigtable-kafka-connect-sink/integration-tests/src/test/java/com/google/cloud/kafka/connect/bigtable/integration/BaseKafkaConnectIT.java index d2ce0a1704..3fdb1f171b 100644 --- a/google-cloud-bigtable-kafka-connect-sink/src/test/java/com/google/cloud/kafka/connect/bigtable/integration/BaseKafkaConnectIT.java +++ b/google-cloud-bigtable-kafka-connect-sink/integration-tests/src/test/java/com/google/cloud/kafka/connect/bigtable/integration/BaseKafkaConnectIT.java @@ -19,6 +19,7 @@ import static org.junit.Assert.assertEquals; import static org.junit.Assert.assertTrue; +import java.io.File; import java.nio.charset.StandardCharsets; import java.time.Duration; import java.util.ArrayList; @@ -28,6 +29,7 @@ import java.util.HashMap; import java.util.List; import java.util.Map; +import java.util.Objects; import java.util.Optional; import java.util.Properties; import java.util.Set; @@ -97,13 +99,21 @@ public void startConnect() { ProducerConfig.BUFFER_MEMORY_CONFIG, String.valueOf(maxKafkaMessageSizeBytes)); clientConfigs.put( ConsumerConfig.MAX_PARTITION_FETCH_BYTES_CONFIG, String.valueOf(maxKafkaMessageSizeBytes)); + + Map workerProps = new HashMap<>(); + String pluginPath = Objects.requireNonNull(System.getenv(PLUGIN_PATH_ENV_VAR_NAME)); + assertTrue(new File(pluginPath).isDirectory()); + workerProps.put(WorkerConfig.PLUGIN_PATH_CONFIG, pluginPath); + workerProps.put(WorkerConfig.OFFSET_COMMIT_INTERVAL_MS_CONFIG, Long.toString(10000)); + workerProps.put(WorkerConfig.PLUGIN_DISCOVERY_CONFIG, PluginDiscoveryMode.HYBRID_WARN.name()); + connect = new EmbeddedConnectCluster.Builder() .name("kcbt-connect-cluster-" + getTestClassId()) .numWorkers(numWorkers) .numBrokers(numBrokers) .brokerProps(brokerProps) - .workerProps(workerProps()) + .workerProps(workerProps) .clientProps(clientConfigs) .build(); @@ -122,13 +132,6 @@ public void startConnect() { connect.kafka().bootstrapServers()); } - public Map workerProps() { - Map workerProps = new HashMap<>(); - workerProps.put(WorkerConfig.OFFSET_COMMIT_INTERVAL_MS_CONFIG, Long.toString(10000)); - workerProps.put(WorkerConfig.PLUGIN_DISCOVERY_CONFIG, PluginDiscoveryMode.HYBRID_WARN.name()); - return workerProps; - } - public void stopConnect() { logger.info( "Stopping embedded Kafka Connect cluster using bootstrap servers: {}", diff --git a/google-cloud-bigtable-kafka-connect-sink/src/test/java/com/google/cloud/kafka/connect/bigtable/integration/BasicIT.java b/google-cloud-bigtable-kafka-connect-sink/integration-tests/src/test/java/com/google/cloud/kafka/connect/bigtable/integration/BasicIT.java similarity index 100% rename from google-cloud-bigtable-kafka-connect-sink/src/test/java/com/google/cloud/kafka/connect/bigtable/integration/BasicIT.java rename to google-cloud-bigtable-kafka-connect-sink/integration-tests/src/test/java/com/google/cloud/kafka/connect/bigtable/integration/BasicIT.java diff --git a/google-cloud-bigtable-kafka-connect-sink/src/test/java/com/google/cloud/kafka/connect/bigtable/integration/BigtableClientIT.java b/google-cloud-bigtable-kafka-connect-sink/integration-tests/src/test/java/com/google/cloud/kafka/connect/bigtable/integration/BigtableClientIT.java similarity index 100% rename from google-cloud-bigtable-kafka-connect-sink/src/test/java/com/google/cloud/kafka/connect/bigtable/integration/BigtableClientIT.java rename to google-cloud-bigtable-kafka-connect-sink/integration-tests/src/test/java/com/google/cloud/kafka/connect/bigtable/integration/BigtableClientIT.java diff --git a/google-cloud-bigtable-kafka-connect-sink/src/test/java/com/google/cloud/kafka/connect/bigtable/integration/ConfigIT.java b/google-cloud-bigtable-kafka-connect-sink/integration-tests/src/test/java/com/google/cloud/kafka/connect/bigtable/integration/ConfigIT.java similarity index 100% rename from google-cloud-bigtable-kafka-connect-sink/src/test/java/com/google/cloud/kafka/connect/bigtable/integration/ConfigIT.java rename to google-cloud-bigtable-kafka-connect-sink/integration-tests/src/test/java/com/google/cloud/kafka/connect/bigtable/integration/ConfigIT.java diff --git a/google-cloud-bigtable-kafka-connect-sink/src/test/java/com/google/cloud/kafka/connect/bigtable/integration/ConfluentCompatibilityIT.java b/google-cloud-bigtable-kafka-connect-sink/integration-tests/src/test/java/com/google/cloud/kafka/connect/bigtable/integration/ConfluentCompatibilityIT.java similarity index 96% rename from google-cloud-bigtable-kafka-connect-sink/src/test/java/com/google/cloud/kafka/connect/bigtable/integration/ConfluentCompatibilityIT.java rename to google-cloud-bigtable-kafka-connect-sink/integration-tests/src/test/java/com/google/cloud/kafka/connect/bigtable/integration/ConfluentCompatibilityIT.java index 8e9ea58018..e67c9881fd 100644 --- a/google-cloud-bigtable-kafka-connect-sink/src/test/java/com/google/cloud/kafka/connect/bigtable/integration/ConfluentCompatibilityIT.java +++ b/google-cloud-bigtable-kafka-connect-sink/integration-tests/src/test/java/com/google/cloud/kafka/connect/bigtable/integration/ConfluentCompatibilityIT.java @@ -42,7 +42,6 @@ import java.util.HashMap; import java.util.List; import java.util.Map; -import java.util.Objects; import java.util.Properties; import java.util.Set; import java.util.concurrent.ExecutionException; @@ -53,7 +52,6 @@ import org.apache.kafka.clients.producer.Producer; import org.apache.kafka.clients.producer.ProducerRecord; import org.apache.kafka.connect.runtime.ConnectorConfig; -import org.apache.kafka.connect.runtime.WorkerConfig; import org.junit.Test; import org.junit.runner.RunWith; import org.junit.runners.Parameterized; @@ -203,15 +201,6 @@ public String startConnector(Map connectorProps) throws Interrup return topic; } - @Override - public Map workerProps() { - Map props = super.workerProps(); - String pluginPath = Objects.requireNonNull(System.getenv(PLUGIN_PATH_ENV_VAR_NAME)); - // Enabling embedded Kafka Connect to use the Confluent's sink. - props.put(WorkerConfig.PLUGIN_PATH_CONFIG, pluginPath); - return props; - } - public void populateTopic(String topic) throws IOException { String keySchema = readStringResource(getTestCaseDir() + "/key-schema.json"); String valueSchema = readStringResource(getTestCaseDir() + "/value-schema.json"); diff --git a/google-cloud-bigtable-kafka-connect-sink/src/test/java/com/google/cloud/kafka/connect/bigtable/integration/DifferentConvertersIT.java b/google-cloud-bigtable-kafka-connect-sink/integration-tests/src/test/java/com/google/cloud/kafka/connect/bigtable/integration/DifferentConvertersIT.java similarity index 95% rename from google-cloud-bigtable-kafka-connect-sink/src/test/java/com/google/cloud/kafka/connect/bigtable/integration/DifferentConvertersIT.java rename to google-cloud-bigtable-kafka-connect-sink/integration-tests/src/test/java/com/google/cloud/kafka/connect/bigtable/integration/DifferentConvertersIT.java index f906199a3e..d4a3837389 100644 --- a/google-cloud-bigtable-kafka-connect-sink/src/test/java/com/google/cloud/kafka/connect/bigtable/integration/DifferentConvertersIT.java +++ b/google-cloud-bigtable-kafka-connect-sink/integration-tests/src/test/java/com/google/cloud/kafka/connect/bigtable/integration/DifferentConvertersIT.java @@ -17,6 +17,7 @@ import io.confluent.connect.avro.AvroConverter; import io.confluent.connect.json.JsonSchemaConverter; +import io.confluent.connect.protobuf.ProtobufConverter; import io.confluent.kafka.serializers.AbstractKafkaSchemaSerDeConfig; import java.util.Arrays; import java.util.Collection; @@ -43,9 +44,7 @@ public static Collection testCases() { return Arrays.asList( new Object[][] { {(Supplier) AvroConverter::new, Map.of(), true}, - // TODO: uncomment once it starts using java-protobuf 4.x. Broken due to a dependency - // version mismatch for now. - // {(Supplier) ProtobufConverter::new, Map.of(), true}, + {(Supplier) ProtobufConverter::new, Map.of(), true}, {(Supplier) JsonSchemaConverter::new, Map.of(), true}, { (Supplier) JsonConverter::new, diff --git a/google-cloud-bigtable-kafka-connect-sink/src/test/java/com/google/cloud/kafka/connect/bigtable/integration/ErrorHandlingIT.java b/google-cloud-bigtable-kafka-connect-sink/integration-tests/src/test/java/com/google/cloud/kafka/connect/bigtable/integration/ErrorHandlingIT.java similarity index 100% rename from google-cloud-bigtable-kafka-connect-sink/src/test/java/com/google/cloud/kafka/connect/bigtable/integration/ErrorHandlingIT.java rename to google-cloud-bigtable-kafka-connect-sink/integration-tests/src/test/java/com/google/cloud/kafka/connect/bigtable/integration/ErrorHandlingIT.java diff --git a/google-cloud-bigtable-kafka-connect-sink/src/test/java/com/google/cloud/kafka/connect/bigtable/integration/ErrorReportingIT.java b/google-cloud-bigtable-kafka-connect-sink/integration-tests/src/test/java/com/google/cloud/kafka/connect/bigtable/integration/ErrorReportingIT.java similarity index 100% rename from google-cloud-bigtable-kafka-connect-sink/src/test/java/com/google/cloud/kafka/connect/bigtable/integration/ErrorReportingIT.java rename to google-cloud-bigtable-kafka-connect-sink/integration-tests/src/test/java/com/google/cloud/kafka/connect/bigtable/integration/ErrorReportingIT.java diff --git a/google-cloud-bigtable-kafka-connect-sink/src/test/java/com/google/cloud/kafka/connect/bigtable/integration/InsertUpsertIT.java b/google-cloud-bigtable-kafka-connect-sink/integration-tests/src/test/java/com/google/cloud/kafka/connect/bigtable/integration/InsertUpsertIT.java similarity index 100% rename from google-cloud-bigtable-kafka-connect-sink/src/test/java/com/google/cloud/kafka/connect/bigtable/integration/InsertUpsertIT.java rename to google-cloud-bigtable-kafka-connect-sink/integration-tests/src/test/java/com/google/cloud/kafka/connect/bigtable/integration/InsertUpsertIT.java diff --git a/google-cloud-bigtable-kafka-connect-sink/src/test/java/com/google/cloud/kafka/connect/bigtable/integration/MultipleConnectorTasksIT.java b/google-cloud-bigtable-kafka-connect-sink/integration-tests/src/test/java/com/google/cloud/kafka/connect/bigtable/integration/MultipleConnectorTasksIT.java similarity index 100% rename from google-cloud-bigtable-kafka-connect-sink/src/test/java/com/google/cloud/kafka/connect/bigtable/integration/MultipleConnectorTasksIT.java rename to google-cloud-bigtable-kafka-connect-sink/integration-tests/src/test/java/com/google/cloud/kafka/connect/bigtable/integration/MultipleConnectorTasksIT.java diff --git a/google-cloud-bigtable-kafka-connect-sink/src/test/java/com/google/cloud/kafka/connect/bigtable/integration/NullHandlingIT.java b/google-cloud-bigtable-kafka-connect-sink/integration-tests/src/test/java/com/google/cloud/kafka/connect/bigtable/integration/NullHandlingIT.java similarity index 100% rename from google-cloud-bigtable-kafka-connect-sink/src/test/java/com/google/cloud/kafka/connect/bigtable/integration/NullHandlingIT.java rename to google-cloud-bigtable-kafka-connect-sink/integration-tests/src/test/java/com/google/cloud/kafka/connect/bigtable/integration/NullHandlingIT.java diff --git a/google-cloud-bigtable-kafka-connect-sink/src/test/java/com/google/cloud/kafka/connect/bigtable/integration/ResourceAutoCreationIT.java b/google-cloud-bigtable-kafka-connect-sink/integration-tests/src/test/java/com/google/cloud/kafka/connect/bigtable/integration/ResourceAutoCreationIT.java similarity index 98% rename from google-cloud-bigtable-kafka-connect-sink/src/test/java/com/google/cloud/kafka/connect/bigtable/integration/ResourceAutoCreationIT.java rename to google-cloud-bigtable-kafka-connect-sink/integration-tests/src/test/java/com/google/cloud/kafka/connect/bigtable/integration/ResourceAutoCreationIT.java index 6ad82bbeff..6957765020 100644 --- a/google-cloud-bigtable-kafka-connect-sink/src/test/java/com/google/cloud/kafka/connect/bigtable/integration/ResourceAutoCreationIT.java +++ b/google-cloud-bigtable-kafka-connect-sink/integration-tests/src/test/java/com/google/cloud/kafka/connect/bigtable/integration/ResourceAutoCreationIT.java @@ -22,7 +22,6 @@ import com.google.cloud.bigtable.admin.v2.models.ModifyColumnFamiliesRequest; import com.google.cloud.bigtable.data.v2.models.Range; import com.google.cloud.kafka.connect.bigtable.config.BigtableSinkConfig; -import com.google.cloud.kafka.connect.bigtable.config.BigtableSinkTaskConfig; import com.google.cloud.kafka.connect.bigtable.config.InsertMode; import com.google.cloud.kafka.connect.bigtable.config.NullValueMode; import com.google.cloud.kafka.connect.bigtable.exception.InvalidBigtableSchemaModificationException; @@ -127,7 +126,7 @@ public void testTableAutoCreationDisabledColumnFamilyAutoCreationEnabled() String dlqTopic = createDlq(); Map props = baseConnectorProps(); configureDlq(props, dlqTopic); - props.put(BigtableSinkTaskConfig.AUTO_CREATE_COLUMN_FAMILIES_CONFIG, String.valueOf(true)); + props.put(BigtableSinkConfig.AUTO_CREATE_COLUMN_FAMILIES_CONFIG, String.valueOf(true)); props.put(ConnectorConfig.VALUE_CONVERTER_CLASS_CONFIG, JsonConverter.class.getName()); props.put( ConnectorConfig.VALUE_CONVERTER_CLASS_CONFIG diff --git a/google-cloud-bigtable-kafka-connect-sink/src/test/java/com/google/cloud/kafka/connect/bigtable/integration/VersionIT.java b/google-cloud-bigtable-kafka-connect-sink/integration-tests/src/test/java/com/google/cloud/kafka/connect/bigtable/integration/VersionIT.java similarity index 93% rename from google-cloud-bigtable-kafka-connect-sink/src/test/java/com/google/cloud/kafka/connect/bigtable/integration/VersionIT.java rename to google-cloud-bigtable-kafka-connect-sink/integration-tests/src/test/java/com/google/cloud/kafka/connect/bigtable/integration/VersionIT.java index 7f3453781c..d253b7bc95 100644 --- a/google-cloud-bigtable-kafka-connect-sink/src/test/java/com/google/cloud/kafka/connect/bigtable/integration/VersionIT.java +++ b/google-cloud-bigtable-kafka-connect-sink/integration-tests/src/test/java/com/google/cloud/kafka/connect/bigtable/integration/VersionIT.java @@ -19,7 +19,6 @@ import com.fasterxml.jackson.databind.MapperFeature; import com.fasterxml.jackson.databind.ObjectMapper; -import com.google.cloud.kafka.connect.bigtable.BigtableSinkConnector; import com.google.cloud.kafka.connect.bigtable.version.PackageMetadata; import java.io.IOException; import java.net.URI; @@ -49,7 +48,7 @@ public void testVersionIsSet() throws IOException, InterruptedException { PluginInfo[] pluginInfos = mapper.readValue(response.body(), PluginInfo[].class); PluginInfo pluginInfo = Arrays.stream(pluginInfos) - .filter(i -> i.className().equals(BigtableSinkConnector.class.getName())) + .filter(i -> CONNECTOR_CLASS_NAME.equals(i.className())) .findFirst() .get(); assertNotEquals(PackageMetadata.UNKNOWN_VERSION, pluginInfo.version()); diff --git a/google-cloud-bigtable-kafka-connect-sink/integration-tests/src/test/java/com/google/cloud/kafka/connect/bigtable/util/JsonConverterFactory.java b/google-cloud-bigtable-kafka-connect-sink/integration-tests/src/test/java/com/google/cloud/kafka/connect/bigtable/util/JsonConverterFactory.java new file mode 120000 index 0000000000..96b9c8a98b --- /dev/null +++ b/google-cloud-bigtable-kafka-connect-sink/integration-tests/src/test/java/com/google/cloud/kafka/connect/bigtable/util/JsonConverterFactory.java @@ -0,0 +1 @@ +../../../../../../../../../../../sink/src/test/java/com/google/cloud/kafka/connect/bigtable/util/JsonConverterFactory.java \ No newline at end of file diff --git a/google-cloud-bigtable-kafka-connect-sink/integration-tests/src/test/java/com/google/cloud/kafka/connect/bigtable/util/NestedNullStructFactory.java b/google-cloud-bigtable-kafka-connect-sink/integration-tests/src/test/java/com/google/cloud/kafka/connect/bigtable/util/NestedNullStructFactory.java new file mode 120000 index 0000000000..6a9f76dfef --- /dev/null +++ b/google-cloud-bigtable-kafka-connect-sink/integration-tests/src/test/java/com/google/cloud/kafka/connect/bigtable/util/NestedNullStructFactory.java @@ -0,0 +1 @@ +../../../../../../../../../../../sink/src/test/java/com/google/cloud/kafka/connect/bigtable/util/NestedNullStructFactory.java \ No newline at end of file diff --git a/google-cloud-bigtable-kafka-connect-sink/src/test/java/com/google/cloud/kafka/connect/bigtable/util/SchemaRegistryTestUtil.java b/google-cloud-bigtable-kafka-connect-sink/integration-tests/src/test/java/com/google/cloud/kafka/connect/bigtable/util/SchemaRegistryTestUtil.java similarity index 100% rename from google-cloud-bigtable-kafka-connect-sink/src/test/java/com/google/cloud/kafka/connect/bigtable/util/SchemaRegistryTestUtil.java rename to google-cloud-bigtable-kafka-connect-sink/integration-tests/src/test/java/com/google/cloud/kafka/connect/bigtable/util/SchemaRegistryTestUtil.java diff --git a/google-cloud-bigtable-kafka-connect-sink/src/test/java/com/google/cloud/kafka/connect/bigtable/util/TestId.java b/google-cloud-bigtable-kafka-connect-sink/integration-tests/src/test/java/com/google/cloud/kafka/connect/bigtable/util/TestId.java similarity index 100% rename from google-cloud-bigtable-kafka-connect-sink/src/test/java/com/google/cloud/kafka/connect/bigtable/util/TestId.java rename to google-cloud-bigtable-kafka-connect-sink/integration-tests/src/test/java/com/google/cloud/kafka/connect/bigtable/util/TestId.java diff --git a/google-cloud-bigtable-kafka-connect-sink/integration-tests/src/test/java/com/google/cloud/kafka/connect/bigtable/version/PackageMetadata.java b/google-cloud-bigtable-kafka-connect-sink/integration-tests/src/test/java/com/google/cloud/kafka/connect/bigtable/version/PackageMetadata.java new file mode 120000 index 0000000000..945bfd1877 --- /dev/null +++ b/google-cloud-bigtable-kafka-connect-sink/integration-tests/src/test/java/com/google/cloud/kafka/connect/bigtable/version/PackageMetadata.java @@ -0,0 +1 @@ +../../../../../../../../../../../sink/src/main/java/com/google/cloud/kafka/connect/bigtable/version/PackageMetadata.java \ No newline at end of file diff --git a/google-cloud-bigtable-kafka-connect-sink/src/test/resources/compatibility_test_cases/key_bytes/column-families.strings b/google-cloud-bigtable-kafka-connect-sink/integration-tests/src/test/resources/compatibility_test_cases/key_bytes/column-families.strings similarity index 100% rename from google-cloud-bigtable-kafka-connect-sink/src/test/resources/compatibility_test_cases/key_bytes/column-families.strings rename to google-cloud-bigtable-kafka-connect-sink/integration-tests/src/test/resources/compatibility_test_cases/key_bytes/column-families.strings diff --git a/google-cloud-bigtable-kafka-connect-sink/src/test/resources/compatibility_test_cases/key_bytes/data.json b/google-cloud-bigtable-kafka-connect-sink/integration-tests/src/test/resources/compatibility_test_cases/key_bytes/data.json similarity index 100% rename from google-cloud-bigtable-kafka-connect-sink/src/test/resources/compatibility_test_cases/key_bytes/data.json rename to google-cloud-bigtable-kafka-connect-sink/integration-tests/src/test/resources/compatibility_test_cases/key_bytes/data.json diff --git a/google-cloud-bigtable-kafka-connect-sink/src/test/resources/compatibility_test_cases/key_bytes/key-schema.json b/google-cloud-bigtable-kafka-connect-sink/integration-tests/src/test/resources/compatibility_test_cases/key_bytes/key-schema.json similarity index 100% rename from google-cloud-bigtable-kafka-connect-sink/src/test/resources/compatibility_test_cases/key_bytes/key-schema.json rename to google-cloud-bigtable-kafka-connect-sink/integration-tests/src/test/resources/compatibility_test_cases/key_bytes/key-schema.json diff --git a/google-cloud-bigtable-kafka-connect-sink/src/test/resources/compatibility_test_cases/key_bytes/value-schema.json b/google-cloud-bigtable-kafka-connect-sink/integration-tests/src/test/resources/compatibility_test_cases/key_bytes/value-schema.json similarity index 100% rename from google-cloud-bigtable-kafka-connect-sink/src/test/resources/compatibility_test_cases/key_bytes/value-schema.json rename to google-cloud-bigtable-kafka-connect-sink/integration-tests/src/test/resources/compatibility_test_cases/key_bytes/value-schema.json diff --git a/google-cloud-bigtable-kafka-connect-sink/src/test/resources/compatibility_test_cases/key_containers/column-families.strings b/google-cloud-bigtable-kafka-connect-sink/integration-tests/src/test/resources/compatibility_test_cases/key_containers/column-families.strings similarity index 100% rename from google-cloud-bigtable-kafka-connect-sink/src/test/resources/compatibility_test_cases/key_containers/column-families.strings rename to google-cloud-bigtable-kafka-connect-sink/integration-tests/src/test/resources/compatibility_test_cases/key_containers/column-families.strings diff --git a/google-cloud-bigtable-kafka-connect-sink/src/test/resources/compatibility_test_cases/key_containers/data.json b/google-cloud-bigtable-kafka-connect-sink/integration-tests/src/test/resources/compatibility_test_cases/key_containers/data.json similarity index 100% rename from google-cloud-bigtable-kafka-connect-sink/src/test/resources/compatibility_test_cases/key_containers/data.json rename to google-cloud-bigtable-kafka-connect-sink/integration-tests/src/test/resources/compatibility_test_cases/key_containers/data.json diff --git a/google-cloud-bigtable-kafka-connect-sink/src/test/resources/compatibility_test_cases/key_containers/key-schema.json b/google-cloud-bigtable-kafka-connect-sink/integration-tests/src/test/resources/compatibility_test_cases/key_containers/key-schema.json similarity index 100% rename from google-cloud-bigtable-kafka-connect-sink/src/test/resources/compatibility_test_cases/key_containers/key-schema.json rename to google-cloud-bigtable-kafka-connect-sink/integration-tests/src/test/resources/compatibility_test_cases/key_containers/key-schema.json diff --git a/google-cloud-bigtable-kafka-connect-sink/src/test/resources/compatibility_test_cases/key_containers/value-schema.json b/google-cloud-bigtable-kafka-connect-sink/integration-tests/src/test/resources/compatibility_test_cases/key_containers/value-schema.json similarity index 100% rename from google-cloud-bigtable-kafka-connect-sink/src/test/resources/compatibility_test_cases/key_containers/value-schema.json rename to google-cloud-bigtable-kafka-connect-sink/integration-tests/src/test/resources/compatibility_test_cases/key_containers/value-schema.json diff --git a/google-cloud-bigtable-kafka-connect-sink/src/test/resources/compatibility_test_cases/key_logicals/column-families.strings b/google-cloud-bigtable-kafka-connect-sink/integration-tests/src/test/resources/compatibility_test_cases/key_logicals/column-families.strings similarity index 100% rename from google-cloud-bigtable-kafka-connect-sink/src/test/resources/compatibility_test_cases/key_logicals/column-families.strings rename to google-cloud-bigtable-kafka-connect-sink/integration-tests/src/test/resources/compatibility_test_cases/key_logicals/column-families.strings diff --git a/google-cloud-bigtable-kafka-connect-sink/src/test/resources/compatibility_test_cases/key_logicals/data.json b/google-cloud-bigtable-kafka-connect-sink/integration-tests/src/test/resources/compatibility_test_cases/key_logicals/data.json similarity index 100% rename from google-cloud-bigtable-kafka-connect-sink/src/test/resources/compatibility_test_cases/key_logicals/data.json rename to google-cloud-bigtable-kafka-connect-sink/integration-tests/src/test/resources/compatibility_test_cases/key_logicals/data.json diff --git a/google-cloud-bigtable-kafka-connect-sink/src/test/resources/compatibility_test_cases/key_logicals/key-schema.json b/google-cloud-bigtable-kafka-connect-sink/integration-tests/src/test/resources/compatibility_test_cases/key_logicals/key-schema.json similarity index 100% rename from google-cloud-bigtable-kafka-connect-sink/src/test/resources/compatibility_test_cases/key_logicals/key-schema.json rename to google-cloud-bigtable-kafka-connect-sink/integration-tests/src/test/resources/compatibility_test_cases/key_logicals/key-schema.json diff --git a/google-cloud-bigtable-kafka-connect-sink/src/test/resources/compatibility_test_cases/key_logicals/value-schema.json b/google-cloud-bigtable-kafka-connect-sink/integration-tests/src/test/resources/compatibility_test_cases/key_logicals/value-schema.json similarity index 100% rename from google-cloud-bigtable-kafka-connect-sink/src/test/resources/compatibility_test_cases/key_logicals/value-schema.json rename to google-cloud-bigtable-kafka-connect-sink/integration-tests/src/test/resources/compatibility_test_cases/key_logicals/value-schema.json diff --git a/google-cloud-bigtable-kafka-connect-sink/src/test/resources/compatibility_test_cases/key_matryoshkas/column-families.strings b/google-cloud-bigtable-kafka-connect-sink/integration-tests/src/test/resources/compatibility_test_cases/key_matryoshkas/column-families.strings similarity index 100% rename from google-cloud-bigtable-kafka-connect-sink/src/test/resources/compatibility_test_cases/key_matryoshkas/column-families.strings rename to google-cloud-bigtable-kafka-connect-sink/integration-tests/src/test/resources/compatibility_test_cases/key_matryoshkas/column-families.strings diff --git a/google-cloud-bigtable-kafka-connect-sink/src/test/resources/compatibility_test_cases/key_matryoshkas/data.json b/google-cloud-bigtable-kafka-connect-sink/integration-tests/src/test/resources/compatibility_test_cases/key_matryoshkas/data.json similarity index 100% rename from google-cloud-bigtable-kafka-connect-sink/src/test/resources/compatibility_test_cases/key_matryoshkas/data.json rename to google-cloud-bigtable-kafka-connect-sink/integration-tests/src/test/resources/compatibility_test_cases/key_matryoshkas/data.json diff --git a/google-cloud-bigtable-kafka-connect-sink/src/test/resources/compatibility_test_cases/key_matryoshkas/key-schema.json b/google-cloud-bigtable-kafka-connect-sink/integration-tests/src/test/resources/compatibility_test_cases/key_matryoshkas/key-schema.json similarity index 100% rename from google-cloud-bigtable-kafka-connect-sink/src/test/resources/compatibility_test_cases/key_matryoshkas/key-schema.json rename to google-cloud-bigtable-kafka-connect-sink/integration-tests/src/test/resources/compatibility_test_cases/key_matryoshkas/key-schema.json diff --git a/google-cloud-bigtable-kafka-connect-sink/src/test/resources/compatibility_test_cases/key_matryoshkas/value-schema.json b/google-cloud-bigtable-kafka-connect-sink/integration-tests/src/test/resources/compatibility_test_cases/key_matryoshkas/value-schema.json similarity index 100% rename from google-cloud-bigtable-kafka-connect-sink/src/test/resources/compatibility_test_cases/key_matryoshkas/value-schema.json rename to google-cloud-bigtable-kafka-connect-sink/integration-tests/src/test/resources/compatibility_test_cases/key_matryoshkas/value-schema.json diff --git a/google-cloud-bigtable-kafka-connect-sink/src/test/resources/compatibility_test_cases/key_nestedlogicals/column-families.strings b/google-cloud-bigtable-kafka-connect-sink/integration-tests/src/test/resources/compatibility_test_cases/key_nestedlogicals/column-families.strings similarity index 100% rename from google-cloud-bigtable-kafka-connect-sink/src/test/resources/compatibility_test_cases/key_nestedlogicals/column-families.strings rename to google-cloud-bigtable-kafka-connect-sink/integration-tests/src/test/resources/compatibility_test_cases/key_nestedlogicals/column-families.strings diff --git a/google-cloud-bigtable-kafka-connect-sink/src/test/resources/compatibility_test_cases/key_nestedlogicals/data.json b/google-cloud-bigtable-kafka-connect-sink/integration-tests/src/test/resources/compatibility_test_cases/key_nestedlogicals/data.json similarity index 100% rename from google-cloud-bigtable-kafka-connect-sink/src/test/resources/compatibility_test_cases/key_nestedlogicals/data.json rename to google-cloud-bigtable-kafka-connect-sink/integration-tests/src/test/resources/compatibility_test_cases/key_nestedlogicals/data.json diff --git a/google-cloud-bigtable-kafka-connect-sink/src/test/resources/compatibility_test_cases/key_nestedlogicals/key-schema.json b/google-cloud-bigtable-kafka-connect-sink/integration-tests/src/test/resources/compatibility_test_cases/key_nestedlogicals/key-schema.json similarity index 100% rename from google-cloud-bigtable-kafka-connect-sink/src/test/resources/compatibility_test_cases/key_nestedlogicals/key-schema.json rename to google-cloud-bigtable-kafka-connect-sink/integration-tests/src/test/resources/compatibility_test_cases/key_nestedlogicals/key-schema.json diff --git a/google-cloud-bigtable-kafka-connect-sink/src/test/resources/compatibility_test_cases/key_nestedlogicals/value-schema.json b/google-cloud-bigtable-kafka-connect-sink/integration-tests/src/test/resources/compatibility_test_cases/key_nestedlogicals/value-schema.json similarity index 100% rename from google-cloud-bigtable-kafka-connect-sink/src/test/resources/compatibility_test_cases/key_nestedlogicals/value-schema.json rename to google-cloud-bigtable-kafka-connect-sink/integration-tests/src/test/resources/compatibility_test_cases/key_nestedlogicals/value-schema.json diff --git a/google-cloud-bigtable-kafka-connect-sink/src/test/resources/compatibility_test_cases/key_primitives/column-families.strings b/google-cloud-bigtable-kafka-connect-sink/integration-tests/src/test/resources/compatibility_test_cases/key_primitives/column-families.strings similarity index 100% rename from google-cloud-bigtable-kafka-connect-sink/src/test/resources/compatibility_test_cases/key_primitives/column-families.strings rename to google-cloud-bigtable-kafka-connect-sink/integration-tests/src/test/resources/compatibility_test_cases/key_primitives/column-families.strings diff --git a/google-cloud-bigtable-kafka-connect-sink/src/test/resources/compatibility_test_cases/key_primitives/data.json b/google-cloud-bigtable-kafka-connect-sink/integration-tests/src/test/resources/compatibility_test_cases/key_primitives/data.json similarity index 100% rename from google-cloud-bigtable-kafka-connect-sink/src/test/resources/compatibility_test_cases/key_primitives/data.json rename to google-cloud-bigtable-kafka-connect-sink/integration-tests/src/test/resources/compatibility_test_cases/key_primitives/data.json diff --git a/google-cloud-bigtable-kafka-connect-sink/src/test/resources/compatibility_test_cases/key_primitives/key-schema.json b/google-cloud-bigtable-kafka-connect-sink/integration-tests/src/test/resources/compatibility_test_cases/key_primitives/key-schema.json similarity index 100% rename from google-cloud-bigtable-kafka-connect-sink/src/test/resources/compatibility_test_cases/key_primitives/key-schema.json rename to google-cloud-bigtable-kafka-connect-sink/integration-tests/src/test/resources/compatibility_test_cases/key_primitives/key-schema.json diff --git a/google-cloud-bigtable-kafka-connect-sink/src/test/resources/compatibility_test_cases/key_primitives/value-schema.json b/google-cloud-bigtable-kafka-connect-sink/integration-tests/src/test/resources/compatibility_test_cases/key_primitives/value-schema.json similarity index 100% rename from google-cloud-bigtable-kafka-connect-sink/src/test/resources/compatibility_test_cases/key_primitives/value-schema.json rename to google-cloud-bigtable-kafka-connect-sink/integration-tests/src/test/resources/compatibility_test_cases/key_primitives/value-schema.json diff --git a/google-cloud-bigtable-kafka-connect-sink/src/test/resources/compatibility_test_cases/key_root_primitives/column-families.strings b/google-cloud-bigtable-kafka-connect-sink/integration-tests/src/test/resources/compatibility_test_cases/key_root_primitives/column-families.strings similarity index 100% rename from google-cloud-bigtable-kafka-connect-sink/src/test/resources/compatibility_test_cases/key_root_primitives/column-families.strings rename to google-cloud-bigtable-kafka-connect-sink/integration-tests/src/test/resources/compatibility_test_cases/key_root_primitives/column-families.strings diff --git a/google-cloud-bigtable-kafka-connect-sink/src/test/resources/compatibility_test_cases/key_root_primitives/data.json b/google-cloud-bigtable-kafka-connect-sink/integration-tests/src/test/resources/compatibility_test_cases/key_root_primitives/data.json similarity index 100% rename from google-cloud-bigtable-kafka-connect-sink/src/test/resources/compatibility_test_cases/key_root_primitives/data.json rename to google-cloud-bigtable-kafka-connect-sink/integration-tests/src/test/resources/compatibility_test_cases/key_root_primitives/data.json diff --git a/google-cloud-bigtable-kafka-connect-sink/src/test/resources/compatibility_test_cases/key_root_primitives/key-schema.json b/google-cloud-bigtable-kafka-connect-sink/integration-tests/src/test/resources/compatibility_test_cases/key_root_primitives/key-schema.json similarity index 100% rename from google-cloud-bigtable-kafka-connect-sink/src/test/resources/compatibility_test_cases/key_root_primitives/key-schema.json rename to google-cloud-bigtable-kafka-connect-sink/integration-tests/src/test/resources/compatibility_test_cases/key_root_primitives/key-schema.json diff --git a/google-cloud-bigtable-kafka-connect-sink/src/test/resources/compatibility_test_cases/key_root_primitives/value-schema.json b/google-cloud-bigtable-kafka-connect-sink/integration-tests/src/test/resources/compatibility_test_cases/key_root_primitives/value-schema.json similarity index 100% rename from google-cloud-bigtable-kafka-connect-sink/src/test/resources/compatibility_test_cases/key_root_primitives/value-schema.json rename to google-cloud-bigtable-kafka-connect-sink/integration-tests/src/test/resources/compatibility_test_cases/key_root_primitives/value-schema.json diff --git a/google-cloud-bigtable-kafka-connect-sink/src/test/resources/compatibility_test_cases/key_union/column-families.strings b/google-cloud-bigtable-kafka-connect-sink/integration-tests/src/test/resources/compatibility_test_cases/key_union/column-families.strings similarity index 100% rename from google-cloud-bigtable-kafka-connect-sink/src/test/resources/compatibility_test_cases/key_union/column-families.strings rename to google-cloud-bigtable-kafka-connect-sink/integration-tests/src/test/resources/compatibility_test_cases/key_union/column-families.strings diff --git a/google-cloud-bigtable-kafka-connect-sink/src/test/resources/compatibility_test_cases/key_union/data.json b/google-cloud-bigtable-kafka-connect-sink/integration-tests/src/test/resources/compatibility_test_cases/key_union/data.json similarity index 100% rename from google-cloud-bigtable-kafka-connect-sink/src/test/resources/compatibility_test_cases/key_union/data.json rename to google-cloud-bigtable-kafka-connect-sink/integration-tests/src/test/resources/compatibility_test_cases/key_union/data.json diff --git a/google-cloud-bigtable-kafka-connect-sink/src/test/resources/compatibility_test_cases/key_union/key-schema.json b/google-cloud-bigtable-kafka-connect-sink/integration-tests/src/test/resources/compatibility_test_cases/key_union/key-schema.json similarity index 100% rename from google-cloud-bigtable-kafka-connect-sink/src/test/resources/compatibility_test_cases/key_union/key-schema.json rename to google-cloud-bigtable-kafka-connect-sink/integration-tests/src/test/resources/compatibility_test_cases/key_union/key-schema.json diff --git a/google-cloud-bigtable-kafka-connect-sink/src/test/resources/compatibility_test_cases/key_union/value-schema.json b/google-cloud-bigtable-kafka-connect-sink/integration-tests/src/test/resources/compatibility_test_cases/key_union/value-schema.json similarity index 100% rename from google-cloud-bigtable-kafka-connect-sink/src/test/resources/compatibility_test_cases/key_union/value-schema.json rename to google-cloud-bigtable-kafka-connect-sink/integration-tests/src/test/resources/compatibility_test_cases/key_union/value-schema.json diff --git a/google-cloud-bigtable-kafka-connect-sink/src/test/resources/compatibility_test_cases/value_bytes/column-families.strings b/google-cloud-bigtable-kafka-connect-sink/integration-tests/src/test/resources/compatibility_test_cases/value_bytes/column-families.strings similarity index 100% rename from google-cloud-bigtable-kafka-connect-sink/src/test/resources/compatibility_test_cases/value_bytes/column-families.strings rename to google-cloud-bigtable-kafka-connect-sink/integration-tests/src/test/resources/compatibility_test_cases/value_bytes/column-families.strings diff --git a/google-cloud-bigtable-kafka-connect-sink/src/test/resources/compatibility_test_cases/value_bytes/data.json b/google-cloud-bigtable-kafka-connect-sink/integration-tests/src/test/resources/compatibility_test_cases/value_bytes/data.json similarity index 100% rename from google-cloud-bigtable-kafka-connect-sink/src/test/resources/compatibility_test_cases/value_bytes/data.json rename to google-cloud-bigtable-kafka-connect-sink/integration-tests/src/test/resources/compatibility_test_cases/value_bytes/data.json diff --git a/google-cloud-bigtable-kafka-connect-sink/src/test/resources/compatibility_test_cases/value_bytes/key-schema.json b/google-cloud-bigtable-kafka-connect-sink/integration-tests/src/test/resources/compatibility_test_cases/value_bytes/key-schema.json similarity index 100% rename from google-cloud-bigtable-kafka-connect-sink/src/test/resources/compatibility_test_cases/value_bytes/key-schema.json rename to google-cloud-bigtable-kafka-connect-sink/integration-tests/src/test/resources/compatibility_test_cases/value_bytes/key-schema.json diff --git a/google-cloud-bigtable-kafka-connect-sink/src/test/resources/compatibility_test_cases/value_bytes/value-schema.json b/google-cloud-bigtable-kafka-connect-sink/integration-tests/src/test/resources/compatibility_test_cases/value_bytes/value-schema.json similarity index 100% rename from google-cloud-bigtable-kafka-connect-sink/src/test/resources/compatibility_test_cases/value_bytes/value-schema.json rename to google-cloud-bigtable-kafka-connect-sink/integration-tests/src/test/resources/compatibility_test_cases/value_bytes/value-schema.json diff --git a/google-cloud-bigtable-kafka-connect-sink/src/test/resources/compatibility_test_cases/value_containers/column-families.strings b/google-cloud-bigtable-kafka-connect-sink/integration-tests/src/test/resources/compatibility_test_cases/value_containers/column-families.strings similarity index 100% rename from google-cloud-bigtable-kafka-connect-sink/src/test/resources/compatibility_test_cases/value_containers/column-families.strings rename to google-cloud-bigtable-kafka-connect-sink/integration-tests/src/test/resources/compatibility_test_cases/value_containers/column-families.strings diff --git a/google-cloud-bigtable-kafka-connect-sink/src/test/resources/compatibility_test_cases/value_containers/data.json b/google-cloud-bigtable-kafka-connect-sink/integration-tests/src/test/resources/compatibility_test_cases/value_containers/data.json similarity index 100% rename from google-cloud-bigtable-kafka-connect-sink/src/test/resources/compatibility_test_cases/value_containers/data.json rename to google-cloud-bigtable-kafka-connect-sink/integration-tests/src/test/resources/compatibility_test_cases/value_containers/data.json diff --git a/google-cloud-bigtable-kafka-connect-sink/src/test/resources/compatibility_test_cases/value_containers/key-schema.json b/google-cloud-bigtable-kafka-connect-sink/integration-tests/src/test/resources/compatibility_test_cases/value_containers/key-schema.json similarity index 100% rename from google-cloud-bigtable-kafka-connect-sink/src/test/resources/compatibility_test_cases/value_containers/key-schema.json rename to google-cloud-bigtable-kafka-connect-sink/integration-tests/src/test/resources/compatibility_test_cases/value_containers/key-schema.json diff --git a/google-cloud-bigtable-kafka-connect-sink/src/test/resources/compatibility_test_cases/value_containers/value-schema.json b/google-cloud-bigtable-kafka-connect-sink/integration-tests/src/test/resources/compatibility_test_cases/value_containers/value-schema.json similarity index 100% rename from google-cloud-bigtable-kafka-connect-sink/src/test/resources/compatibility_test_cases/value_containers/value-schema.json rename to google-cloud-bigtable-kafka-connect-sink/integration-tests/src/test/resources/compatibility_test_cases/value_containers/value-schema.json diff --git a/google-cloud-bigtable-kafka-connect-sink/src/test/resources/compatibility_test_cases/value_logicals/column-families.strings b/google-cloud-bigtable-kafka-connect-sink/integration-tests/src/test/resources/compatibility_test_cases/value_logicals/column-families.strings similarity index 100% rename from google-cloud-bigtable-kafka-connect-sink/src/test/resources/compatibility_test_cases/value_logicals/column-families.strings rename to google-cloud-bigtable-kafka-connect-sink/integration-tests/src/test/resources/compatibility_test_cases/value_logicals/column-families.strings diff --git a/google-cloud-bigtable-kafka-connect-sink/src/test/resources/compatibility_test_cases/value_logicals/data.json b/google-cloud-bigtable-kafka-connect-sink/integration-tests/src/test/resources/compatibility_test_cases/value_logicals/data.json similarity index 100% rename from google-cloud-bigtable-kafka-connect-sink/src/test/resources/compatibility_test_cases/value_logicals/data.json rename to google-cloud-bigtable-kafka-connect-sink/integration-tests/src/test/resources/compatibility_test_cases/value_logicals/data.json diff --git a/google-cloud-bigtable-kafka-connect-sink/src/test/resources/compatibility_test_cases/value_logicals/key-schema.json b/google-cloud-bigtable-kafka-connect-sink/integration-tests/src/test/resources/compatibility_test_cases/value_logicals/key-schema.json similarity index 100% rename from google-cloud-bigtable-kafka-connect-sink/src/test/resources/compatibility_test_cases/value_logicals/key-schema.json rename to google-cloud-bigtable-kafka-connect-sink/integration-tests/src/test/resources/compatibility_test_cases/value_logicals/key-schema.json diff --git a/google-cloud-bigtable-kafka-connect-sink/src/test/resources/compatibility_test_cases/value_logicals/value-schema.json b/google-cloud-bigtable-kafka-connect-sink/integration-tests/src/test/resources/compatibility_test_cases/value_logicals/value-schema.json similarity index 100% rename from google-cloud-bigtable-kafka-connect-sink/src/test/resources/compatibility_test_cases/value_logicals/value-schema.json rename to google-cloud-bigtable-kafka-connect-sink/integration-tests/src/test/resources/compatibility_test_cases/value_logicals/value-schema.json diff --git a/google-cloud-bigtable-kafka-connect-sink/src/test/resources/compatibility_test_cases/value_matryoshkas/column-families.strings b/google-cloud-bigtable-kafka-connect-sink/integration-tests/src/test/resources/compatibility_test_cases/value_matryoshkas/column-families.strings similarity index 100% rename from google-cloud-bigtable-kafka-connect-sink/src/test/resources/compatibility_test_cases/value_matryoshkas/column-families.strings rename to google-cloud-bigtable-kafka-connect-sink/integration-tests/src/test/resources/compatibility_test_cases/value_matryoshkas/column-families.strings diff --git a/google-cloud-bigtable-kafka-connect-sink/src/test/resources/compatibility_test_cases/value_matryoshkas/data.json b/google-cloud-bigtable-kafka-connect-sink/integration-tests/src/test/resources/compatibility_test_cases/value_matryoshkas/data.json similarity index 100% rename from google-cloud-bigtable-kafka-connect-sink/src/test/resources/compatibility_test_cases/value_matryoshkas/data.json rename to google-cloud-bigtable-kafka-connect-sink/integration-tests/src/test/resources/compatibility_test_cases/value_matryoshkas/data.json diff --git a/google-cloud-bigtable-kafka-connect-sink/src/test/resources/compatibility_test_cases/value_matryoshkas/key-schema.json b/google-cloud-bigtable-kafka-connect-sink/integration-tests/src/test/resources/compatibility_test_cases/value_matryoshkas/key-schema.json similarity index 100% rename from google-cloud-bigtable-kafka-connect-sink/src/test/resources/compatibility_test_cases/value_matryoshkas/key-schema.json rename to google-cloud-bigtable-kafka-connect-sink/integration-tests/src/test/resources/compatibility_test_cases/value_matryoshkas/key-schema.json diff --git a/google-cloud-bigtable-kafka-connect-sink/src/test/resources/compatibility_test_cases/value_matryoshkas/value-schema.json b/google-cloud-bigtable-kafka-connect-sink/integration-tests/src/test/resources/compatibility_test_cases/value_matryoshkas/value-schema.json similarity index 100% rename from google-cloud-bigtable-kafka-connect-sink/src/test/resources/compatibility_test_cases/value_matryoshkas/value-schema.json rename to google-cloud-bigtable-kafka-connect-sink/integration-tests/src/test/resources/compatibility_test_cases/value_matryoshkas/value-schema.json diff --git a/google-cloud-bigtable-kafka-connect-sink/src/test/resources/compatibility_test_cases/value_nestedlogicals/column-families.strings b/google-cloud-bigtable-kafka-connect-sink/integration-tests/src/test/resources/compatibility_test_cases/value_nestedlogicals/column-families.strings similarity index 100% rename from google-cloud-bigtable-kafka-connect-sink/src/test/resources/compatibility_test_cases/value_nestedlogicals/column-families.strings rename to google-cloud-bigtable-kafka-connect-sink/integration-tests/src/test/resources/compatibility_test_cases/value_nestedlogicals/column-families.strings diff --git a/google-cloud-bigtable-kafka-connect-sink/src/test/resources/compatibility_test_cases/value_nestedlogicals/data.json b/google-cloud-bigtable-kafka-connect-sink/integration-tests/src/test/resources/compatibility_test_cases/value_nestedlogicals/data.json similarity index 100% rename from google-cloud-bigtable-kafka-connect-sink/src/test/resources/compatibility_test_cases/value_nestedlogicals/data.json rename to google-cloud-bigtable-kafka-connect-sink/integration-tests/src/test/resources/compatibility_test_cases/value_nestedlogicals/data.json diff --git a/google-cloud-bigtable-kafka-connect-sink/src/test/resources/compatibility_test_cases/value_nestedlogicals/key-schema.json b/google-cloud-bigtable-kafka-connect-sink/integration-tests/src/test/resources/compatibility_test_cases/value_nestedlogicals/key-schema.json similarity index 100% rename from google-cloud-bigtable-kafka-connect-sink/src/test/resources/compatibility_test_cases/value_nestedlogicals/key-schema.json rename to google-cloud-bigtable-kafka-connect-sink/integration-tests/src/test/resources/compatibility_test_cases/value_nestedlogicals/key-schema.json diff --git a/google-cloud-bigtable-kafka-connect-sink/src/test/resources/compatibility_test_cases/value_nestedlogicals/value-schema.json b/google-cloud-bigtable-kafka-connect-sink/integration-tests/src/test/resources/compatibility_test_cases/value_nestedlogicals/value-schema.json similarity index 100% rename from google-cloud-bigtable-kafka-connect-sink/src/test/resources/compatibility_test_cases/value_nestedlogicals/value-schema.json rename to google-cloud-bigtable-kafka-connect-sink/integration-tests/src/test/resources/compatibility_test_cases/value_nestedlogicals/value-schema.json diff --git a/google-cloud-bigtable-kafka-connect-sink/src/test/resources/compatibility_test_cases/value_nulls/column-families.strings b/google-cloud-bigtable-kafka-connect-sink/integration-tests/src/test/resources/compatibility_test_cases/value_nulls/column-families.strings similarity index 100% rename from google-cloud-bigtable-kafka-connect-sink/src/test/resources/compatibility_test_cases/value_nulls/column-families.strings rename to google-cloud-bigtable-kafka-connect-sink/integration-tests/src/test/resources/compatibility_test_cases/value_nulls/column-families.strings diff --git a/google-cloud-bigtable-kafka-connect-sink/src/test/resources/compatibility_test_cases/value_nulls/data.json b/google-cloud-bigtable-kafka-connect-sink/integration-tests/src/test/resources/compatibility_test_cases/value_nulls/data.json similarity index 100% rename from google-cloud-bigtable-kafka-connect-sink/src/test/resources/compatibility_test_cases/value_nulls/data.json rename to google-cloud-bigtable-kafka-connect-sink/integration-tests/src/test/resources/compatibility_test_cases/value_nulls/data.json diff --git a/google-cloud-bigtable-kafka-connect-sink/src/test/resources/compatibility_test_cases/value_nulls/key-schema.json b/google-cloud-bigtable-kafka-connect-sink/integration-tests/src/test/resources/compatibility_test_cases/value_nulls/key-schema.json similarity index 100% rename from google-cloud-bigtable-kafka-connect-sink/src/test/resources/compatibility_test_cases/value_nulls/key-schema.json rename to google-cloud-bigtable-kafka-connect-sink/integration-tests/src/test/resources/compatibility_test_cases/value_nulls/key-schema.json diff --git a/google-cloud-bigtable-kafka-connect-sink/src/test/resources/compatibility_test_cases/value_nulls/value-schema.json b/google-cloud-bigtable-kafka-connect-sink/integration-tests/src/test/resources/compatibility_test_cases/value_nulls/value-schema.json similarity index 100% rename from google-cloud-bigtable-kafka-connect-sink/src/test/resources/compatibility_test_cases/value_nulls/value-schema.json rename to google-cloud-bigtable-kafka-connect-sink/integration-tests/src/test/resources/compatibility_test_cases/value_nulls/value-schema.json diff --git a/google-cloud-bigtable-kafka-connect-sink/src/test/resources/compatibility_test_cases/value_primitives/column-families.strings b/google-cloud-bigtable-kafka-connect-sink/integration-tests/src/test/resources/compatibility_test_cases/value_primitives/column-families.strings similarity index 100% rename from google-cloud-bigtable-kafka-connect-sink/src/test/resources/compatibility_test_cases/value_primitives/column-families.strings rename to google-cloud-bigtable-kafka-connect-sink/integration-tests/src/test/resources/compatibility_test_cases/value_primitives/column-families.strings diff --git a/google-cloud-bigtable-kafka-connect-sink/src/test/resources/compatibility_test_cases/value_primitives/data.json b/google-cloud-bigtable-kafka-connect-sink/integration-tests/src/test/resources/compatibility_test_cases/value_primitives/data.json similarity index 100% rename from google-cloud-bigtable-kafka-connect-sink/src/test/resources/compatibility_test_cases/value_primitives/data.json rename to google-cloud-bigtable-kafka-connect-sink/integration-tests/src/test/resources/compatibility_test_cases/value_primitives/data.json diff --git a/google-cloud-bigtable-kafka-connect-sink/src/test/resources/compatibility_test_cases/value_primitives/key-schema.json b/google-cloud-bigtable-kafka-connect-sink/integration-tests/src/test/resources/compatibility_test_cases/value_primitives/key-schema.json similarity index 100% rename from google-cloud-bigtable-kafka-connect-sink/src/test/resources/compatibility_test_cases/value_primitives/key-schema.json rename to google-cloud-bigtable-kafka-connect-sink/integration-tests/src/test/resources/compatibility_test_cases/value_primitives/key-schema.json diff --git a/google-cloud-bigtable-kafka-connect-sink/src/test/resources/compatibility_test_cases/value_primitives/value-schema.json b/google-cloud-bigtable-kafka-connect-sink/integration-tests/src/test/resources/compatibility_test_cases/value_primitives/value-schema.json similarity index 100% rename from google-cloud-bigtable-kafka-connect-sink/src/test/resources/compatibility_test_cases/value_primitives/value-schema.json rename to google-cloud-bigtable-kafka-connect-sink/integration-tests/src/test/resources/compatibility_test_cases/value_primitives/value-schema.json diff --git a/google-cloud-bigtable-kafka-connect-sink/src/test/resources/compatibility_test_cases/value_root_primitives/column-families.strings b/google-cloud-bigtable-kafka-connect-sink/integration-tests/src/test/resources/compatibility_test_cases/value_root_primitives/column-families.strings similarity index 100% rename from google-cloud-bigtable-kafka-connect-sink/src/test/resources/compatibility_test_cases/value_root_primitives/column-families.strings rename to google-cloud-bigtable-kafka-connect-sink/integration-tests/src/test/resources/compatibility_test_cases/value_root_primitives/column-families.strings diff --git a/google-cloud-bigtable-kafka-connect-sink/src/test/resources/compatibility_test_cases/value_root_primitives/data.json b/google-cloud-bigtable-kafka-connect-sink/integration-tests/src/test/resources/compatibility_test_cases/value_root_primitives/data.json similarity index 100% rename from google-cloud-bigtable-kafka-connect-sink/src/test/resources/compatibility_test_cases/value_root_primitives/data.json rename to google-cloud-bigtable-kafka-connect-sink/integration-tests/src/test/resources/compatibility_test_cases/value_root_primitives/data.json diff --git a/google-cloud-bigtable-kafka-connect-sink/src/test/resources/compatibility_test_cases/value_root_primitives/key-schema.json b/google-cloud-bigtable-kafka-connect-sink/integration-tests/src/test/resources/compatibility_test_cases/value_root_primitives/key-schema.json similarity index 100% rename from google-cloud-bigtable-kafka-connect-sink/src/test/resources/compatibility_test_cases/value_root_primitives/key-schema.json rename to google-cloud-bigtable-kafka-connect-sink/integration-tests/src/test/resources/compatibility_test_cases/value_root_primitives/key-schema.json diff --git a/google-cloud-bigtable-kafka-connect-sink/src/test/resources/compatibility_test_cases/value_root_primitives/value-schema.json b/google-cloud-bigtable-kafka-connect-sink/integration-tests/src/test/resources/compatibility_test_cases/value_root_primitives/value-schema.json similarity index 100% rename from google-cloud-bigtable-kafka-connect-sink/src/test/resources/compatibility_test_cases/value_root_primitives/value-schema.json rename to google-cloud-bigtable-kafka-connect-sink/integration-tests/src/test/resources/compatibility_test_cases/value_root_primitives/value-schema.json diff --git a/google-cloud-bigtable-kafka-connect-sink/src/test/resources/compatibility_test_cases/value_union/column-families.strings b/google-cloud-bigtable-kafka-connect-sink/integration-tests/src/test/resources/compatibility_test_cases/value_union/column-families.strings similarity index 100% rename from google-cloud-bigtable-kafka-connect-sink/src/test/resources/compatibility_test_cases/value_union/column-families.strings rename to google-cloud-bigtable-kafka-connect-sink/integration-tests/src/test/resources/compatibility_test_cases/value_union/column-families.strings diff --git a/google-cloud-bigtable-kafka-connect-sink/src/test/resources/compatibility_test_cases/value_union/data.json b/google-cloud-bigtable-kafka-connect-sink/integration-tests/src/test/resources/compatibility_test_cases/value_union/data.json similarity index 100% rename from google-cloud-bigtable-kafka-connect-sink/src/test/resources/compatibility_test_cases/value_union/data.json rename to google-cloud-bigtable-kafka-connect-sink/integration-tests/src/test/resources/compatibility_test_cases/value_union/data.json diff --git a/google-cloud-bigtable-kafka-connect-sink/src/test/resources/compatibility_test_cases/value_union/key-schema.json b/google-cloud-bigtable-kafka-connect-sink/integration-tests/src/test/resources/compatibility_test_cases/value_union/key-schema.json similarity index 100% rename from google-cloud-bigtable-kafka-connect-sink/src/test/resources/compatibility_test_cases/value_union/key-schema.json rename to google-cloud-bigtable-kafka-connect-sink/integration-tests/src/test/resources/compatibility_test_cases/value_union/key-schema.json diff --git a/google-cloud-bigtable-kafka-connect-sink/src/test/resources/compatibility_test_cases/value_union/value-schema.json b/google-cloud-bigtable-kafka-connect-sink/integration-tests/src/test/resources/compatibility_test_cases/value_union/value-schema.json similarity index 100% rename from google-cloud-bigtable-kafka-connect-sink/src/test/resources/compatibility_test_cases/value_union/value-schema.json rename to google-cloud-bigtable-kafka-connect-sink/integration-tests/src/test/resources/compatibility_test_cases/value_union/value-schema.json diff --git a/google-cloud-bigtable-kafka-connect-sink/integration-tests/src/test/resources/fake_service_key.json b/google-cloud-bigtable-kafka-connect-sink/integration-tests/src/test/resources/fake_service_key.json new file mode 120000 index 0000000000..cb0e7d8a6a --- /dev/null +++ b/google-cloud-bigtable-kafka-connect-sink/integration-tests/src/test/resources/fake_service_key.json @@ -0,0 +1 @@ +../../../../sink/src/test/resources/fake_service_key.json \ No newline at end of file diff --git a/google-cloud-bigtable-kafka-connect-sink/integration-tests/src/test/resources/log4j.properties b/google-cloud-bigtable-kafka-connect-sink/integration-tests/src/test/resources/log4j.properties new file mode 120000 index 0000000000..2191152ffd --- /dev/null +++ b/google-cloud-bigtable-kafka-connect-sink/integration-tests/src/test/resources/log4j.properties @@ -0,0 +1 @@ +../../../../sink/src/test/resources/log4j.properties \ No newline at end of file diff --git a/google-cloud-bigtable-kafka-connect-sink/pom.xml b/google-cloud-bigtable-kafka-connect-sink/pom.xml index f311ba2137..1c949129c5 100644 --- a/google-cloud-bigtable-kafka-connect-sink/pom.xml +++ b/google-cloud-bigtable-kafka-connect-sink/pom.xml @@ -3,13 +3,17 @@ xsi:schemaLocation="http://maven.apache.org/POM/4.0.0 http://maven.apache.org/xsd/maven-4.0.0.xsd"> 4.0.0 com.google.cloud.kafka.connect.bigtable - sink + sink-parent 1.0.0-SNAPSHOT - jar - kafka-connect-bigtable-sink + pom + kafka-connect-bigtable-sink-parent - Google Bigtable sink connector for Apache Kafka Connect + Parent of Google Bigtable sink connector for Apache Kafka Connect + + sink + integration-tests + 26.53.0 2.52.0 @@ -17,18 +21,12 @@ 2.12 2.6.1 1.7.33 - 1.7.36 2.12.7 - 5.15.2 4.13.2 - - 5.11.3 - 7.7.0 + 5.15.2 + 1.7.36 + 2.43.0 1.19.2 3.11.2 @@ -37,15 +35,10 @@ 3.5.2 3.5.2 3.4.2 - 9.4.57.v20241219 11 11 UTF-8 - - ${project.basedir}/integration_test_plugins - https://hub-downloads.confluent.io/api/plugins/confluentinc/kafka-connect-gcp-bigtable/versions/2.0.28/confluentinc-kafka-connect-gcp-bigtable-2.0.28.zip - confluentsink.zip @@ -58,186 +51,8 @@ - - - org.apache.kafka - connect-api - ${kafka.version} - provided - - - org.apache.kafka - connect-runtime - ${kafka.version} - provided - - - org.apache.kafka - kafka-clients - ${kafka.version} - provided - - - com.google.cloud - google-cloud-bigtable - ${google.bigtable.version} - - - org.apache.hbase - hbase-common - ${hbase.version} - - - org.slf4j - slf4j-api - ${slf4j.api.version} - - - org.slf4j - slf4j-reload4j - ${slf4j.reload4j.version} - test - - - com.fasterxml.jackson.core - jackson-databind - ${jackson.version} - - - junit - junit - ${junit.version} - test - - - org.apache.kafka - connect-runtime - ${kafka.version} - test - test - - - org.apache.kafka - kafka-clients - ${kafka.version} - test - test-jar - test - - - org.apache.kafka - kafka_${kafka.scala.version} - ${kafka.version} - test - - - org.apache.kafka - kafka_${kafka.scala.version} - ${kafka.version} - test - test-jar - test - - - org.mockito - mockito-core - ${mockito.version} - test - - - org.junit.jupiter - junit-jupiter-api - ${junit.jupiter.version} - test - - - io.confluent - kafka-schema-registry - ${confluent.version} - test - - - io.confluent - kafka-schema-registry - tests - test-jar - ${confluent.version} - test - - - io.confluent - kafka-avro-serializer - ${confluent.version} - test - - - io.confluent - kafka-connect-avro-converter - ${confluent.version} - test - - - io.confluent - kafka-connect-protobuf-converter - ${confluent.version} - test - - - io.confluent - kafka-connect-json-schema-converter - ${confluent.version} - test - - - org.eclipse.jetty - jetty-server - ${jetty.version} - test - - - org.eclipse.jetty - jetty-servlet - ${jetty.version} - test - - - org.eclipse.jetty - jetty-servlets - ${jetty.version} - test - - - org.eclipse.jetty - jetty-client - ${jetty.version} - test - - - - - org.apache.maven.plugins - maven-assembly-plugin - - - src/main/assembly/package.xml - - - - - make-assembly - package - - single - - - - com.diffplug.spotless spotless-maven-plugin @@ -264,53 +79,20 @@ true - ${project.basedir}/license.header + license.header - org.apache.maven.plugins - maven-surefire-plugin - ${surefire.version} - - - ${skipUnitTests} - false - - classes - true - - - target/test-classes/fake_service_key.json - - - - - org.apache.maven.plugins - maven-failsafe-plugin - ${failsafe.version} - - - ${skipIntegrationTests} - false - - classes - true - - - target/test-classes/fake_service_key.json - - localhost:8086 - ${integration.test.plugin.path} - - + org.jacoco + jacoco-maven-plugin + ${jacoco.version} - verify - integration-test + prepare-agent @@ -327,43 +109,6 @@ - - org.jacoco - jacoco-maven-plugin - ${jacoco.version} - - - - prepare-agent - - - - - - org.apache.maven.plugins - maven-antrun-plugin - ${antrun.version} - - - download-and-unzip-confluent-sink - pre-integration-test - - run - - - - - - - - - - @@ -380,11 +125,4 @@ - - - - confluent - https://packages.confluent.io/maven/ - - \ No newline at end of file diff --git a/google-cloud-bigtable-kafka-connect-sink/sink/pom.xml b/google-cloud-bigtable-kafka-connect-sink/sink/pom.xml new file mode 100644 index 0000000000..a7656c4483 --- /dev/null +++ b/google-cloud-bigtable-kafka-connect-sink/sink/pom.xml @@ -0,0 +1,120 @@ + + + 4.0.0 + com.google.cloud.kafka.connect.bigtable + sink + 1.0.0-SNAPSHOT + jar + kafka-connect-bigtable-sink + + Google Bigtable sink connector for Apache Kafka Connect + + + com.google.cloud.kafka.connect.bigtable + sink-parent + 1.0.0-SNAPSHOT + + + + org.apache.kafka + connect-api + ${kafka.version} + provided + + + org.apache.kafka + connect-runtime + ${kafka.version} + provided + + + org.apache.kafka + kafka-clients + ${kafka.version} + provided + + + com.google.cloud + google-cloud-bigtable + ${google.bigtable.version} + + + org.apache.hbase + hbase-common + ${hbase.version} + + + org.slf4j + slf4j-api + ${slf4j.api.version} + + + com.fasterxml.jackson.core + jackson-databind + ${jackson.version} + + + org.slf4j + slf4j-reload4j + ${slf4j.reload4j.version} + test + + + junit + junit + ${junit.version} + test + + + org.mockito + mockito-core + ${mockito.version} + test + + + + + + + org.apache.maven.plugins + maven-assembly-plugin + + + src/main/assembly/package.xml + + + + + make-assembly + package + + single + + + + + + org.apache.maven.plugins + maven-surefire-plugin + ${surefire.version} + + + ${skipUnitTests} + false + + classes + true + + + target/test-classes/fake_service_key.json + + + + + + \ No newline at end of file diff --git a/google-cloud-bigtable-kafka-connect-sink/src/main/assembly/package.xml b/google-cloud-bigtable-kafka-connect-sink/sink/src/main/assembly/package.xml similarity index 100% rename from google-cloud-bigtable-kafka-connect-sink/src/main/assembly/package.xml rename to google-cloud-bigtable-kafka-connect-sink/sink/src/main/assembly/package.xml diff --git a/google-cloud-bigtable-kafka-connect-sink/src/main/java/com/google/cloud/kafka/connect/bigtable/BigtableSinkConnector.java b/google-cloud-bigtable-kafka-connect-sink/sink/src/main/java/com/google/cloud/kafka/connect/bigtable/BigtableSinkConnector.java similarity index 100% rename from google-cloud-bigtable-kafka-connect-sink/src/main/java/com/google/cloud/kafka/connect/bigtable/BigtableSinkConnector.java rename to google-cloud-bigtable-kafka-connect-sink/sink/src/main/java/com/google/cloud/kafka/connect/bigtable/BigtableSinkConnector.java diff --git a/google-cloud-bigtable-kafka-connect-sink/src/main/java/com/google/cloud/kafka/connect/bigtable/BigtableSinkTask.java b/google-cloud-bigtable-kafka-connect-sink/sink/src/main/java/com/google/cloud/kafka/connect/bigtable/BigtableSinkTask.java similarity index 100% rename from google-cloud-bigtable-kafka-connect-sink/src/main/java/com/google/cloud/kafka/connect/bigtable/BigtableSinkTask.java rename to google-cloud-bigtable-kafka-connect-sink/sink/src/main/java/com/google/cloud/kafka/connect/bigtable/BigtableSinkTask.java diff --git a/google-cloud-bigtable-kafka-connect-sink/src/main/java/com/google/cloud/kafka/connect/bigtable/autocreate/BigtableSchemaManager.java b/google-cloud-bigtable-kafka-connect-sink/sink/src/main/java/com/google/cloud/kafka/connect/bigtable/autocreate/BigtableSchemaManager.java similarity index 100% rename from google-cloud-bigtable-kafka-connect-sink/src/main/java/com/google/cloud/kafka/connect/bigtable/autocreate/BigtableSchemaManager.java rename to google-cloud-bigtable-kafka-connect-sink/sink/src/main/java/com/google/cloud/kafka/connect/bigtable/autocreate/BigtableSchemaManager.java diff --git a/google-cloud-bigtable-kafka-connect-sink/src/main/java/com/google/cloud/kafka/connect/bigtable/autocreate/ResourceCreationResult.java b/google-cloud-bigtable-kafka-connect-sink/sink/src/main/java/com/google/cloud/kafka/connect/bigtable/autocreate/ResourceCreationResult.java similarity index 100% rename from google-cloud-bigtable-kafka-connect-sink/src/main/java/com/google/cloud/kafka/connect/bigtable/autocreate/ResourceCreationResult.java rename to google-cloud-bigtable-kafka-connect-sink/sink/src/main/java/com/google/cloud/kafka/connect/bigtable/autocreate/ResourceCreationResult.java diff --git a/google-cloud-bigtable-kafka-connect-sink/src/main/java/com/google/cloud/kafka/connect/bigtable/config/BigtableErrorMode.java b/google-cloud-bigtable-kafka-connect-sink/sink/src/main/java/com/google/cloud/kafka/connect/bigtable/config/BigtableErrorMode.java similarity index 100% rename from google-cloud-bigtable-kafka-connect-sink/src/main/java/com/google/cloud/kafka/connect/bigtable/config/BigtableErrorMode.java rename to google-cloud-bigtable-kafka-connect-sink/sink/src/main/java/com/google/cloud/kafka/connect/bigtable/config/BigtableErrorMode.java diff --git a/google-cloud-bigtable-kafka-connect-sink/src/main/java/com/google/cloud/kafka/connect/bigtable/config/BigtableSinkConfig.java b/google-cloud-bigtable-kafka-connect-sink/sink/src/main/java/com/google/cloud/kafka/connect/bigtable/config/BigtableSinkConfig.java similarity index 100% rename from google-cloud-bigtable-kafka-connect-sink/src/main/java/com/google/cloud/kafka/connect/bigtable/config/BigtableSinkConfig.java rename to google-cloud-bigtable-kafka-connect-sink/sink/src/main/java/com/google/cloud/kafka/connect/bigtable/config/BigtableSinkConfig.java diff --git a/google-cloud-bigtable-kafka-connect-sink/src/main/java/com/google/cloud/kafka/connect/bigtable/config/BigtableSinkTaskConfig.java b/google-cloud-bigtable-kafka-connect-sink/sink/src/main/java/com/google/cloud/kafka/connect/bigtable/config/BigtableSinkTaskConfig.java similarity index 100% rename from google-cloud-bigtable-kafka-connect-sink/src/main/java/com/google/cloud/kafka/connect/bigtable/config/BigtableSinkTaskConfig.java rename to google-cloud-bigtable-kafka-connect-sink/sink/src/main/java/com/google/cloud/kafka/connect/bigtable/config/BigtableSinkTaskConfig.java diff --git a/google-cloud-bigtable-kafka-connect-sink/src/main/java/com/google/cloud/kafka/connect/bigtable/config/ConfigInterpolation.java b/google-cloud-bigtable-kafka-connect-sink/sink/src/main/java/com/google/cloud/kafka/connect/bigtable/config/ConfigInterpolation.java similarity index 100% rename from google-cloud-bigtable-kafka-connect-sink/src/main/java/com/google/cloud/kafka/connect/bigtable/config/ConfigInterpolation.java rename to google-cloud-bigtable-kafka-connect-sink/sink/src/main/java/com/google/cloud/kafka/connect/bigtable/config/ConfigInterpolation.java diff --git a/google-cloud-bigtable-kafka-connect-sink/src/main/java/com/google/cloud/kafka/connect/bigtable/config/InsertMode.java b/google-cloud-bigtable-kafka-connect-sink/sink/src/main/java/com/google/cloud/kafka/connect/bigtable/config/InsertMode.java similarity index 100% rename from google-cloud-bigtable-kafka-connect-sink/src/main/java/com/google/cloud/kafka/connect/bigtable/config/InsertMode.java rename to google-cloud-bigtable-kafka-connect-sink/sink/src/main/java/com/google/cloud/kafka/connect/bigtable/config/InsertMode.java diff --git a/google-cloud-bigtable-kafka-connect-sink/src/main/java/com/google/cloud/kafka/connect/bigtable/config/NullValueMode.java b/google-cloud-bigtable-kafka-connect-sink/sink/src/main/java/com/google/cloud/kafka/connect/bigtable/config/NullValueMode.java similarity index 100% rename from google-cloud-bigtable-kafka-connect-sink/src/main/java/com/google/cloud/kafka/connect/bigtable/config/NullValueMode.java rename to google-cloud-bigtable-kafka-connect-sink/sink/src/main/java/com/google/cloud/kafka/connect/bigtable/config/NullValueMode.java diff --git a/google-cloud-bigtable-kafka-connect-sink/src/main/java/com/google/cloud/kafka/connect/bigtable/exception/BatchException.java b/google-cloud-bigtable-kafka-connect-sink/sink/src/main/java/com/google/cloud/kafka/connect/bigtable/exception/BatchException.java similarity index 100% rename from google-cloud-bigtable-kafka-connect-sink/src/main/java/com/google/cloud/kafka/connect/bigtable/exception/BatchException.java rename to google-cloud-bigtable-kafka-connect-sink/sink/src/main/java/com/google/cloud/kafka/connect/bigtable/exception/BatchException.java diff --git a/google-cloud-bigtable-kafka-connect-sink/src/main/java/com/google/cloud/kafka/connect/bigtable/exception/BigtableSinkLogicError.java b/google-cloud-bigtable-kafka-connect-sink/sink/src/main/java/com/google/cloud/kafka/connect/bigtable/exception/BigtableSinkLogicError.java similarity index 100% rename from google-cloud-bigtable-kafka-connect-sink/src/main/java/com/google/cloud/kafka/connect/bigtable/exception/BigtableSinkLogicError.java rename to google-cloud-bigtable-kafka-connect-sink/sink/src/main/java/com/google/cloud/kafka/connect/bigtable/exception/BigtableSinkLogicError.java diff --git a/google-cloud-bigtable-kafka-connect-sink/src/main/java/com/google/cloud/kafka/connect/bigtable/exception/InvalidBigtableSchemaModificationException.java b/google-cloud-bigtable-kafka-connect-sink/sink/src/main/java/com/google/cloud/kafka/connect/bigtable/exception/InvalidBigtableSchemaModificationException.java similarity index 100% rename from google-cloud-bigtable-kafka-connect-sink/src/main/java/com/google/cloud/kafka/connect/bigtable/exception/InvalidBigtableSchemaModificationException.java rename to google-cloud-bigtable-kafka-connect-sink/sink/src/main/java/com/google/cloud/kafka/connect/bigtable/exception/InvalidBigtableSchemaModificationException.java diff --git a/google-cloud-bigtable-kafka-connect-sink/src/main/java/com/google/cloud/kafka/connect/bigtable/mapping/KeyMapper.java b/google-cloud-bigtable-kafka-connect-sink/sink/src/main/java/com/google/cloud/kafka/connect/bigtable/mapping/KeyMapper.java similarity index 100% rename from google-cloud-bigtable-kafka-connect-sink/src/main/java/com/google/cloud/kafka/connect/bigtable/mapping/KeyMapper.java rename to google-cloud-bigtable-kafka-connect-sink/sink/src/main/java/com/google/cloud/kafka/connect/bigtable/mapping/KeyMapper.java diff --git a/google-cloud-bigtable-kafka-connect-sink/src/main/java/com/google/cloud/kafka/connect/bigtable/mapping/LogicalTypeUtils.java b/google-cloud-bigtable-kafka-connect-sink/sink/src/main/java/com/google/cloud/kafka/connect/bigtable/mapping/LogicalTypeUtils.java similarity index 100% rename from google-cloud-bigtable-kafka-connect-sink/src/main/java/com/google/cloud/kafka/connect/bigtable/mapping/LogicalTypeUtils.java rename to google-cloud-bigtable-kafka-connect-sink/sink/src/main/java/com/google/cloud/kafka/connect/bigtable/mapping/LogicalTypeUtils.java diff --git a/google-cloud-bigtable-kafka-connect-sink/src/main/java/com/google/cloud/kafka/connect/bigtable/mapping/MutationData.java b/google-cloud-bigtable-kafka-connect-sink/sink/src/main/java/com/google/cloud/kafka/connect/bigtable/mapping/MutationData.java similarity index 100% rename from google-cloud-bigtable-kafka-connect-sink/src/main/java/com/google/cloud/kafka/connect/bigtable/mapping/MutationData.java rename to google-cloud-bigtable-kafka-connect-sink/sink/src/main/java/com/google/cloud/kafka/connect/bigtable/mapping/MutationData.java diff --git a/google-cloud-bigtable-kafka-connect-sink/src/main/java/com/google/cloud/kafka/connect/bigtable/mapping/MutationDataBuilder.java b/google-cloud-bigtable-kafka-connect-sink/sink/src/main/java/com/google/cloud/kafka/connect/bigtable/mapping/MutationDataBuilder.java similarity index 100% rename from google-cloud-bigtable-kafka-connect-sink/src/main/java/com/google/cloud/kafka/connect/bigtable/mapping/MutationDataBuilder.java rename to google-cloud-bigtable-kafka-connect-sink/sink/src/main/java/com/google/cloud/kafka/connect/bigtable/mapping/MutationDataBuilder.java diff --git a/google-cloud-bigtable-kafka-connect-sink/src/main/java/com/google/cloud/kafka/connect/bigtable/mapping/SchemaErrorReporter.java b/google-cloud-bigtable-kafka-connect-sink/sink/src/main/java/com/google/cloud/kafka/connect/bigtable/mapping/SchemaErrorReporter.java similarity index 100% rename from google-cloud-bigtable-kafka-connect-sink/src/main/java/com/google/cloud/kafka/connect/bigtable/mapping/SchemaErrorReporter.java rename to google-cloud-bigtable-kafka-connect-sink/sink/src/main/java/com/google/cloud/kafka/connect/bigtable/mapping/SchemaErrorReporter.java diff --git a/google-cloud-bigtable-kafka-connect-sink/src/main/java/com/google/cloud/kafka/connect/bigtable/mapping/SchemaUtils.java b/google-cloud-bigtable-kafka-connect-sink/sink/src/main/java/com/google/cloud/kafka/connect/bigtable/mapping/SchemaUtils.java similarity index 100% rename from google-cloud-bigtable-kafka-connect-sink/src/main/java/com/google/cloud/kafka/connect/bigtable/mapping/SchemaUtils.java rename to google-cloud-bigtable-kafka-connect-sink/sink/src/main/java/com/google/cloud/kafka/connect/bigtable/mapping/SchemaUtils.java diff --git a/google-cloud-bigtable-kafka-connect-sink/src/main/java/com/google/cloud/kafka/connect/bigtable/mapping/ValueMapper.java b/google-cloud-bigtable-kafka-connect-sink/sink/src/main/java/com/google/cloud/kafka/connect/bigtable/mapping/ValueMapper.java similarity index 100% rename from google-cloud-bigtable-kafka-connect-sink/src/main/java/com/google/cloud/kafka/connect/bigtable/mapping/ValueMapper.java rename to google-cloud-bigtable-kafka-connect-sink/sink/src/main/java/com/google/cloud/kafka/connect/bigtable/mapping/ValueMapper.java diff --git a/google-cloud-bigtable-kafka-connect-sink/src/main/java/com/google/cloud/kafka/connect/bigtable/version/PackageMetadata.java b/google-cloud-bigtable-kafka-connect-sink/sink/src/main/java/com/google/cloud/kafka/connect/bigtable/version/PackageMetadata.java similarity index 100% rename from google-cloud-bigtable-kafka-connect-sink/src/main/java/com/google/cloud/kafka/connect/bigtable/version/PackageMetadata.java rename to google-cloud-bigtable-kafka-connect-sink/sink/src/main/java/com/google/cloud/kafka/connect/bigtable/version/PackageMetadata.java diff --git a/google-cloud-bigtable-kafka-connect-sink/src/main/resources/META-INF/services/org.apache.kafka.connect.sink.SinkConnector b/google-cloud-bigtable-kafka-connect-sink/sink/src/main/resources/META-INF/services/org.apache.kafka.connect.sink.SinkConnector similarity index 100% rename from google-cloud-bigtable-kafka-connect-sink/src/main/resources/META-INF/services/org.apache.kafka.connect.sink.SinkConnector rename to google-cloud-bigtable-kafka-connect-sink/sink/src/main/resources/META-INF/services/org.apache.kafka.connect.sink.SinkConnector diff --git a/google-cloud-bigtable-kafka-connect-sink/src/test/java/com/google/cloud/kafka/connect/bigtable/BigtableSinkConnectorTest.java b/google-cloud-bigtable-kafka-connect-sink/sink/src/test/java/com/google/cloud/kafka/connect/bigtable/BigtableSinkConnectorTest.java similarity index 100% rename from google-cloud-bigtable-kafka-connect-sink/src/test/java/com/google/cloud/kafka/connect/bigtable/BigtableSinkConnectorTest.java rename to google-cloud-bigtable-kafka-connect-sink/sink/src/test/java/com/google/cloud/kafka/connect/bigtable/BigtableSinkConnectorTest.java diff --git a/google-cloud-bigtable-kafka-connect-sink/src/test/java/com/google/cloud/kafka/connect/bigtable/BigtableSinkTaskTest.java b/google-cloud-bigtable-kafka-connect-sink/sink/src/test/java/com/google/cloud/kafka/connect/bigtable/BigtableSinkTaskTest.java similarity index 98% rename from google-cloud-bigtable-kafka-connect-sink/src/test/java/com/google/cloud/kafka/connect/bigtable/BigtableSinkTaskTest.java rename to google-cloud-bigtable-kafka-connect-sink/sink/src/test/java/com/google/cloud/kafka/connect/bigtable/BigtableSinkTaskTest.java index ff2d573737..79050066d8 100644 --- a/google-cloud-bigtable-kafka-connect-sink/src/test/java/com/google/cloud/kafka/connect/bigtable/BigtableSinkTaskTest.java +++ b/google-cloud-bigtable-kafka-connect-sink/sink/src/test/java/com/google/cloud/kafka/connect/bigtable/BigtableSinkTaskTest.java @@ -65,6 +65,7 @@ import com.google.cloud.kafka.connect.bigtable.util.ApiExceptionFactory; import com.google.cloud.kafka.connect.bigtable.util.BasicPropertiesFactory; import com.google.cloud.kafka.connect.bigtable.util.FutureUtil; +import com.google.common.collect.Collections2; import com.google.protobuf.ByteString; import java.nio.charset.StandardCharsets; import java.util.AbstractMap; @@ -82,7 +83,6 @@ import java.util.concurrent.Future; import java.util.stream.Collectors; import java.util.stream.IntStream; -import org.apache.commons.collections4.iterators.PermutationIterator; import org.apache.kafka.common.record.TimestampType; import org.apache.kafka.connect.errors.ConnectException; import org.apache.kafka.connect.sink.ErrantRecordReporter; @@ -351,10 +351,11 @@ public void testOrderMapSuccesses() { BigtableSinkTask.orderMap(map1, List.of(-1, key1, -2, key2, -3, key3, -4, key4, -5)) .keySet())); - PermutationIterator permutations = - new PermutationIterator<>(List.of(key1, key2, key3, key4)); - permutations.forEachRemaining( - p -> assertEquals(p, new ArrayList<>(BigtableSinkTask.orderMap(map1, p).keySet()))); + Collections2.permutations(List.of(key1, key2, key3, key4)) + .forEach( + p -> { + assertEquals(p, new ArrayList<>(BigtableSinkTask.orderMap(map1, p).keySet())); + }); } @Test diff --git a/google-cloud-bigtable-kafka-connect-sink/src/test/java/com/google/cloud/kafka/connect/bigtable/autocreate/BigtableSchemaManagerTest.java b/google-cloud-bigtable-kafka-connect-sink/sink/src/test/java/com/google/cloud/kafka/connect/bigtable/autocreate/BigtableSchemaManagerTest.java similarity index 100% rename from google-cloud-bigtable-kafka-connect-sink/src/test/java/com/google/cloud/kafka/connect/bigtable/autocreate/BigtableSchemaManagerTest.java rename to google-cloud-bigtable-kafka-connect-sink/sink/src/test/java/com/google/cloud/kafka/connect/bigtable/autocreate/BigtableSchemaManagerTest.java diff --git a/google-cloud-bigtable-kafka-connect-sink/src/test/java/com/google/cloud/kafka/connect/bigtable/autocreate/SchemaApiExceptionsTest.java b/google-cloud-bigtable-kafka-connect-sink/sink/src/test/java/com/google/cloud/kafka/connect/bigtable/autocreate/SchemaApiExceptionsTest.java similarity index 100% rename from google-cloud-bigtable-kafka-connect-sink/src/test/java/com/google/cloud/kafka/connect/bigtable/autocreate/SchemaApiExceptionsTest.java rename to google-cloud-bigtable-kafka-connect-sink/sink/src/test/java/com/google/cloud/kafka/connect/bigtable/autocreate/SchemaApiExceptionsTest.java diff --git a/google-cloud-bigtable-kafka-connect-sink/src/test/java/com/google/cloud/kafka/connect/bigtable/config/BigtableSinkConfigTest.java b/google-cloud-bigtable-kafka-connect-sink/sink/src/test/java/com/google/cloud/kafka/connect/bigtable/config/BigtableSinkConfigTest.java similarity index 100% rename from google-cloud-bigtable-kafka-connect-sink/src/test/java/com/google/cloud/kafka/connect/bigtable/config/BigtableSinkConfigTest.java rename to google-cloud-bigtable-kafka-connect-sink/sink/src/test/java/com/google/cloud/kafka/connect/bigtable/config/BigtableSinkConfigTest.java diff --git a/google-cloud-bigtable-kafka-connect-sink/src/test/java/com/google/cloud/kafka/connect/bigtable/config/ConfigInterpolationTest.java b/google-cloud-bigtable-kafka-connect-sink/sink/src/test/java/com/google/cloud/kafka/connect/bigtable/config/ConfigInterpolationTest.java similarity index 100% rename from google-cloud-bigtable-kafka-connect-sink/src/test/java/com/google/cloud/kafka/connect/bigtable/config/ConfigInterpolationTest.java rename to google-cloud-bigtable-kafka-connect-sink/sink/src/test/java/com/google/cloud/kafka/connect/bigtable/config/ConfigInterpolationTest.java diff --git a/google-cloud-bigtable-kafka-connect-sink/src/test/java/com/google/cloud/kafka/connect/bigtable/mapping/KeyMapperTest.java b/google-cloud-bigtable-kafka-connect-sink/sink/src/test/java/com/google/cloud/kafka/connect/bigtable/mapping/KeyMapperTest.java similarity index 100% rename from google-cloud-bigtable-kafka-connect-sink/src/test/java/com/google/cloud/kafka/connect/bigtable/mapping/KeyMapperTest.java rename to google-cloud-bigtable-kafka-connect-sink/sink/src/test/java/com/google/cloud/kafka/connect/bigtable/mapping/KeyMapperTest.java diff --git a/google-cloud-bigtable-kafka-connect-sink/src/test/java/com/google/cloud/kafka/connect/bigtable/mapping/LogicalTypeUtilsTest.java b/google-cloud-bigtable-kafka-connect-sink/sink/src/test/java/com/google/cloud/kafka/connect/bigtable/mapping/LogicalTypeUtilsTest.java similarity index 100% rename from google-cloud-bigtable-kafka-connect-sink/src/test/java/com/google/cloud/kafka/connect/bigtable/mapping/LogicalTypeUtilsTest.java rename to google-cloud-bigtable-kafka-connect-sink/sink/src/test/java/com/google/cloud/kafka/connect/bigtable/mapping/LogicalTypeUtilsTest.java diff --git a/google-cloud-bigtable-kafka-connect-sink/src/test/java/com/google/cloud/kafka/connect/bigtable/mapping/MutationDataBuilderTest.java b/google-cloud-bigtable-kafka-connect-sink/sink/src/test/java/com/google/cloud/kafka/connect/bigtable/mapping/MutationDataBuilderTest.java similarity index 100% rename from google-cloud-bigtable-kafka-connect-sink/src/test/java/com/google/cloud/kafka/connect/bigtable/mapping/MutationDataBuilderTest.java rename to google-cloud-bigtable-kafka-connect-sink/sink/src/test/java/com/google/cloud/kafka/connect/bigtable/mapping/MutationDataBuilderTest.java diff --git a/google-cloud-bigtable-kafka-connect-sink/src/test/java/com/google/cloud/kafka/connect/bigtable/mapping/ValueMapperTest.java b/google-cloud-bigtable-kafka-connect-sink/sink/src/test/java/com/google/cloud/kafka/connect/bigtable/mapping/ValueMapperTest.java similarity index 100% rename from google-cloud-bigtable-kafka-connect-sink/src/test/java/com/google/cloud/kafka/connect/bigtable/mapping/ValueMapperTest.java rename to google-cloud-bigtable-kafka-connect-sink/sink/src/test/java/com/google/cloud/kafka/connect/bigtable/mapping/ValueMapperTest.java diff --git a/google-cloud-bigtable-kafka-connect-sink/src/test/java/com/google/cloud/kafka/connect/bigtable/util/ApiExceptionFactory.java b/google-cloud-bigtable-kafka-connect-sink/sink/src/test/java/com/google/cloud/kafka/connect/bigtable/util/ApiExceptionFactory.java similarity index 100% rename from google-cloud-bigtable-kafka-connect-sink/src/test/java/com/google/cloud/kafka/connect/bigtable/util/ApiExceptionFactory.java rename to google-cloud-bigtable-kafka-connect-sink/sink/src/test/java/com/google/cloud/kafka/connect/bigtable/util/ApiExceptionFactory.java diff --git a/google-cloud-bigtable-kafka-connect-sink/src/test/java/com/google/cloud/kafka/connect/bigtable/util/BasicPropertiesFactory.java b/google-cloud-bigtable-kafka-connect-sink/sink/src/test/java/com/google/cloud/kafka/connect/bigtable/util/BasicPropertiesFactory.java similarity index 100% rename from google-cloud-bigtable-kafka-connect-sink/src/test/java/com/google/cloud/kafka/connect/bigtable/util/BasicPropertiesFactory.java rename to google-cloud-bigtable-kafka-connect-sink/sink/src/test/java/com/google/cloud/kafka/connect/bigtable/util/BasicPropertiesFactory.java diff --git a/google-cloud-bigtable-kafka-connect-sink/src/test/java/com/google/cloud/kafka/connect/bigtable/util/FutureUtil.java b/google-cloud-bigtable-kafka-connect-sink/sink/src/test/java/com/google/cloud/kafka/connect/bigtable/util/FutureUtil.java similarity index 100% rename from google-cloud-bigtable-kafka-connect-sink/src/test/java/com/google/cloud/kafka/connect/bigtable/util/FutureUtil.java rename to google-cloud-bigtable-kafka-connect-sink/sink/src/test/java/com/google/cloud/kafka/connect/bigtable/util/FutureUtil.java diff --git a/google-cloud-bigtable-kafka-connect-sink/src/test/java/com/google/cloud/kafka/connect/bigtable/util/JsonConverterFactory.java b/google-cloud-bigtable-kafka-connect-sink/sink/src/test/java/com/google/cloud/kafka/connect/bigtable/util/JsonConverterFactory.java similarity index 100% rename from google-cloud-bigtable-kafka-connect-sink/src/test/java/com/google/cloud/kafka/connect/bigtable/util/JsonConverterFactory.java rename to google-cloud-bigtable-kafka-connect-sink/sink/src/test/java/com/google/cloud/kafka/connect/bigtable/util/JsonConverterFactory.java diff --git a/google-cloud-bigtable-kafka-connect-sink/src/test/java/com/google/cloud/kafka/connect/bigtable/util/MockUtil.java b/google-cloud-bigtable-kafka-connect-sink/sink/src/test/java/com/google/cloud/kafka/connect/bigtable/util/MockUtil.java similarity index 100% rename from google-cloud-bigtable-kafka-connect-sink/src/test/java/com/google/cloud/kafka/connect/bigtable/util/MockUtil.java rename to google-cloud-bigtable-kafka-connect-sink/sink/src/test/java/com/google/cloud/kafka/connect/bigtable/util/MockUtil.java diff --git a/google-cloud-bigtable-kafka-connect-sink/src/test/java/com/google/cloud/kafka/connect/bigtable/util/NestedNullStructFactory.java b/google-cloud-bigtable-kafka-connect-sink/sink/src/test/java/com/google/cloud/kafka/connect/bigtable/util/NestedNullStructFactory.java similarity index 100% rename from google-cloud-bigtable-kafka-connect-sink/src/test/java/com/google/cloud/kafka/connect/bigtable/util/NestedNullStructFactory.java rename to google-cloud-bigtable-kafka-connect-sink/sink/src/test/java/com/google/cloud/kafka/connect/bigtable/util/NestedNullStructFactory.java diff --git a/google-cloud-bigtable-kafka-connect-sink/src/test/resources/fake_service_key.json b/google-cloud-bigtable-kafka-connect-sink/sink/src/test/resources/fake_service_key.json similarity index 100% rename from google-cloud-bigtable-kafka-connect-sink/src/test/resources/fake_service_key.json rename to google-cloud-bigtable-kafka-connect-sink/sink/src/test/resources/fake_service_key.json diff --git a/google-cloud-bigtable-kafka-connect-sink/src/test/resources/log4j.properties b/google-cloud-bigtable-kafka-connect-sink/sink/src/test/resources/log4j.properties similarity index 100% rename from google-cloud-bigtable-kafka-connect-sink/src/test/resources/log4j.properties rename to google-cloud-bigtable-kafka-connect-sink/sink/src/test/resources/log4j.properties From e20d4ae33bc3032d18e9787913fa351f96379cc3 Mon Sep 17 00:00:00 2001 From: Adam Czajkowski Date: Mon, 17 Mar 2025 09:44:59 +0100 Subject: [PATCH 74/76] Provide a simple README describing code organization and linking to the other READMEs --- google-cloud-bigtable-kafka-connect-sink/README.md | 14 ++++++++++++++ 1 file changed, 14 insertions(+) create mode 100644 google-cloud-bigtable-kafka-connect-sink/README.md diff --git a/google-cloud-bigtable-kafka-connect-sink/README.md b/google-cloud-bigtable-kafka-connect-sink/README.md new file mode 100644 index 0000000000..c8b1e620ac --- /dev/null +++ b/google-cloud-bigtable-kafka-connect-sink/README.md @@ -0,0 +1,14 @@ +# Code organization +The maven project is split into two modules: +- [sink](sink) - the sink and its unit tests +- [integration-tests](integration-tests) - the integration tests + +This split enables two desirable properties for the integration tests: +- the versions of dependencies used by the integration tests and the sink may be different (since Kafka Connect isolates connectors' class loaders automatically), +- the sink is provided to the integration tests with a directory of jars just like in a real Kafka Connect deployment. + +# Tests +For details on running the tests, please see [doc/tests.md](doc/tests.md). + +# Performance test setup +The performance test setup is described in detail in [doc/performance/README.md](doc/performance/README.md). From b22d97e2e147968acb6908d85b74add1e224125a Mon Sep 17 00:00:00 2001 From: Adam Czajkowski Date: Tue, 1 Apr 2025 14:15:17 +0200 Subject: [PATCH 75/76] Set headers on requests to Bigtable API --- .../bigtable/config/BigtableSinkConfig.java | 14 ++++++++++++++ 1 file changed, 14 insertions(+) diff --git a/google-cloud-bigtable-kafka-connect-sink/sink/src/main/java/com/google/cloud/kafka/connect/bigtable/config/BigtableSinkConfig.java b/google-cloud-bigtable-kafka-connect-sink/sink/src/main/java/com/google/cloud/kafka/connect/bigtable/config/BigtableSinkConfig.java index 2036d19b16..457792ff57 100644 --- a/google-cloud-bigtable-kafka-connect-sink/sink/src/main/java/com/google/cloud/kafka/connect/bigtable/config/BigtableSinkConfig.java +++ b/google-cloud-bigtable-kafka-connect-sink/sink/src/main/java/com/google/cloud/kafka/connect/bigtable/config/BigtableSinkConfig.java @@ -18,6 +18,8 @@ import com.google.api.gax.core.CredentialsProvider; import com.google.api.gax.core.FixedCredentialsProvider; import com.google.api.gax.retrying.RetrySettings; +import com.google.api.gax.rpc.FixedHeaderProvider; +import com.google.api.gax.rpc.HeaderProvider; import com.google.api.gax.rpc.StatusCode; import com.google.auth.oauth2.GoogleCredentials; import com.google.cloud.bigtable.admin.v2.BigtableTableAdminClient; @@ -26,6 +28,7 @@ import com.google.cloud.bigtable.data.v2.BigtableDataClient; import com.google.cloud.bigtable.data.v2.BigtableDataSettings; import com.google.cloud.bigtable.data.v2.stub.EnhancedBigtableStubSettings; +import com.google.cloud.kafka.connect.bigtable.version.PackageMetadata; import com.google.common.annotations.VisibleForTesting; import com.google.common.collect.Sets; import java.io.ByteArrayInputStream; @@ -465,6 +468,7 @@ BigtableTableAdminClient getBigtableAdminClient( } BigtableTableAdminStubSettings.Builder adminStubSettings = adminSettingsBuilder.stubSettings(); + adminStubSettings.setHeaderProvider(getHeaderProvider()); adminStubSettings.listTablesSettings().setRetrySettings(defaultRetrySettings); adminStubSettings.getTableSettings().setRetrySettings(defaultRetrySettings); adminStubSettings @@ -524,6 +528,7 @@ public BigtableDataClient getBigtableDataClient() { } EnhancedBigtableStubSettings.Builder dataStubSettings = dataSettingsBuilder.stubSettings(); + dataStubSettings.setHeaderProvider(getHeaderProvider()); dataStubSettings.mutateRowSettings().setRetrySettings(retrySettings); dataStubSettings.checkAndMutateRowSettings().setRetrySettings(retrySettings); dataStubSettings.bulkMutateRowsSettings().setRetrySettings(retrySettings); @@ -682,4 +687,13 @@ protected Optional getUserConfiguredBigtableCredentialsProv throw new ConfigException("Cloud Bigtable credentials creation failed."); } } + + /** + * @return {@link HeaderProvider} allowing the service provider to monitor the usage of this + * connector. + */ + private static HeaderProvider getHeaderProvider() { + return FixedHeaderProvider.create( + "user-agent", "bigtable-java/kafka-connect-sink/" + PackageMetadata.getVersion()); + } } From 50d72cf78e109a69af39a40b0a61ae170fd39c95 Mon Sep 17 00:00:00 2001 From: Adam Czajkowski Date: Tue, 1 Apr 2025 15:15:38 +0200 Subject: [PATCH 76/76] Remove an unused file --- .../kafka/connect/bigtable/config/BigtableSinkTaskConfig.java | 1 - 1 file changed, 1 deletion(-) delete mode 120000 google-cloud-bigtable-kafka-connect-sink/integration-tests/src/test/java/com/google/cloud/kafka/connect/bigtable/config/BigtableSinkTaskConfig.java diff --git a/google-cloud-bigtable-kafka-connect-sink/integration-tests/src/test/java/com/google/cloud/kafka/connect/bigtable/config/BigtableSinkTaskConfig.java b/google-cloud-bigtable-kafka-connect-sink/integration-tests/src/test/java/com/google/cloud/kafka/connect/bigtable/config/BigtableSinkTaskConfig.java deleted file mode 120000 index 06092b9f05..0000000000 --- a/google-cloud-bigtable-kafka-connect-sink/integration-tests/src/test/java/com/google/cloud/kafka/connect/bigtable/config/BigtableSinkTaskConfig.java +++ /dev/null @@ -1 +0,0 @@ -../../../../../../../../../../../sink/src/main/java/com/google/cloud/kafka/connect/bigtable/config/BigtableSinkTaskConfig.java \ No newline at end of file